mirror of https://github.com/apache/druid.git
Rewrite exotic LAST_VALUE/FIRST_VALUE to self-reference. (#16063)
* Rewrite exotic LAST_VALUE/FIRST_VALUE to self-reference. * rewrite `LAST_VALUE(x) OVER (ORDER BY y)` to `LAG(x,0) OVER (ORDER BY y)` * not directly to `x` because some queries get unplannable that way * restrict `NTILE` from framing - as its not supported * add test to ensure that all of the `KNOWN_WINDOW_FNS`'s framing is accounted for * checkstyle/etc * add test * apidoc * add assume to avoid MSQ fail
This commit is contained in:
parent
8370db106c
commit
a16092b16a
|
@ -62,6 +62,7 @@ import org.apache.druid.sql.calcite.rule.FilterJoinExcludePushToChildRule;
|
|||
import org.apache.druid.sql.calcite.rule.FlattenConcatRule;
|
||||
import org.apache.druid.sql.calcite.rule.ProjectAggregatePruneUnusedCallRule;
|
||||
import org.apache.druid.sql.calcite.rule.ReverseLookupRule;
|
||||
import org.apache.druid.sql.calcite.rule.RewriteFirstValueLastValueRule;
|
||||
import org.apache.druid.sql.calcite.rule.SortCollapseRule;
|
||||
import org.apache.druid.sql.calcite.rule.logical.DruidLogicalRules;
|
||||
import org.apache.druid.sql.calcite.run.EngineFeature;
|
||||
|
@ -330,6 +331,7 @@ public class CalciteRulesManager
|
|||
// make it impossible to convert to COALESCE.
|
||||
builder.addRuleInstance(new CaseToCoalesceRule());
|
||||
builder.addRuleInstance(new CoalesceLookupRule());
|
||||
builder.addRuleInstance(new RewriteFirstValueLastValueRule());
|
||||
}
|
||||
|
||||
// Remaining rules run as a single group until fixpoint.
|
||||
|
|
|
@ -137,9 +137,27 @@ public class DruidSqlValidator extends BaseDruidSqlValidator
|
|||
);
|
||||
}
|
||||
|
||||
boolean hasBounds = lowerBound != null || upperBound != null;
|
||||
if (call.getKind() == SqlKind.NTILE && hasBounds) {
|
||||
throw buildCalciteContextException(
|
||||
"Framing of NTILE is not supported.",
|
||||
call
|
||||
);
|
||||
}
|
||||
|
||||
if (call.getKind() == SqlKind.FIRST_VALUE || call.getKind() == SqlKind.LAST_VALUE) {
|
||||
if (!isUnboundedOrCurrent(lowerBound) || !isUnboundedOrCurrent(upperBound)) {
|
||||
throw buildCalciteContextException(
|
||||
"Framing of FIRST_VALUE/LAST_VALUE is only allowed with UNBOUNDED or CURRENT ROW.",
|
||||
call
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
if (plannerContext.queryContext().isWindowingStrictValidation()) {
|
||||
if (!targetWindow.isRows() &&
|
||||
(!isValidRangeEndpoint(lowerBound) || !isValidRangeEndpoint(upperBound))) {
|
||||
(!isUnboundedOrCurrent(lowerBound) || !isUnboundedOrCurrent(upperBound))) {
|
||||
// this limitation can be lifted when https://github.com/apache/druid/issues/15767 is addressed
|
||||
throw buildCalciteContextException(
|
||||
StringUtils.format(
|
||||
|
@ -578,7 +596,7 @@ public class DruidSqlValidator extends BaseDruidSqlValidator
|
|||
*/
|
||||
private boolean isValidEndpoint(@Nullable SqlNode bound)
|
||||
{
|
||||
if (isValidRangeEndpoint(bound)) {
|
||||
if (isUnboundedOrCurrent(bound)) {
|
||||
return true;
|
||||
}
|
||||
if (bound.getKind() == SqlKind.FOLLOWING || bound.getKind() == SqlKind.PRECEDING) {
|
||||
|
@ -593,7 +611,7 @@ public class DruidSqlValidator extends BaseDruidSqlValidator
|
|||
/**
|
||||
* Checks if the given endpoint is valid for a RANGE window frame.
|
||||
*/
|
||||
private boolean isValidRangeEndpoint(@Nullable SqlNode bound)
|
||||
private boolean isUnboundedOrCurrent(@Nullable SqlNode bound)
|
||||
{
|
||||
return bound == null
|
||||
|| SqlWindow.isCurrentRow(bound)
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.apache.druid.sql.calcite.rel;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
|
@ -85,7 +86,8 @@ import java.util.List;
|
|||
*/
|
||||
public class Windowing
|
||||
{
|
||||
private static final ImmutableMap<String, ProcessorMaker> KNOWN_WINDOW_FNS = ImmutableMap
|
||||
@VisibleForTesting
|
||||
public static final ImmutableMap<String, ProcessorMaker> KNOWN_WINDOW_FNS = ImmutableMap
|
||||
.<String, ProcessorMaker>builder()
|
||||
.put("LAG", (agg) ->
|
||||
new WindowOffsetProcessor(agg.getColumn(0), agg.getOutputName(), -agg.getConstantInt(1, 1)))
|
||||
|
|
|
@ -0,0 +1,138 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.sql.calcite.rule;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import org.apache.calcite.plan.RelOptRule;
|
||||
import org.apache.calcite.plan.RelOptRuleCall;
|
||||
import org.apache.calcite.rel.RelNode;
|
||||
import org.apache.calcite.rel.rules.SubstitutionRule;
|
||||
import org.apache.calcite.rex.RexBuilder;
|
||||
import org.apache.calcite.rex.RexNode;
|
||||
import org.apache.calcite.rex.RexOver;
|
||||
import org.apache.calcite.rex.RexShuttle;
|
||||
import org.apache.calcite.rex.RexWindow;
|
||||
import org.apache.calcite.rex.RexWindowBound;
|
||||
import org.apache.calcite.sql.SqlAggFunction;
|
||||
import org.apache.calcite.sql.SqlKind;
|
||||
import org.apache.calcite.sql.SqlOperator;
|
||||
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
|
||||
|
||||
import java.math.BigDecimal;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Rewrites exotic cases of FIRST_VALUE/LAST_VALUE to simpler plans.
|
||||
*
|
||||
* LAST_VALUE(x) OVER (ORDER BY Y)
|
||||
* implicitly means:
|
||||
* LAST_VALUE(x) OVER (ORDER BY Y RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)
|
||||
* which is equiv to
|
||||
* LAST_VALUE(x) OVER (ORDER BY Y ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)
|
||||
* since it will take the last value from the window; the value of the window will be:
|
||||
* X at the CURRENT ROW.
|
||||
*
|
||||
* This rule does this and a symmetric one for FIRST_VALUE.
|
||||
*/
|
||||
public class RewriteFirstValueLastValueRule extends RelOptRule implements SubstitutionRule
|
||||
{
|
||||
public RewriteFirstValueLastValueRule()
|
||||
{
|
||||
super(operand(RelNode.class, any()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onMatch(RelOptRuleCall call)
|
||||
{
|
||||
final RelNode oldNode = call.rel(0);
|
||||
final RewriteShuttle shuttle = new RewriteShuttle(oldNode.getCluster().getRexBuilder());
|
||||
final RelNode newNode = oldNode.accept(shuttle);
|
||||
|
||||
// noinspection ObjectEquality
|
||||
if (newNode != oldNode) {
|
||||
call.transformTo(newNode);
|
||||
call.getPlanner().prune(oldNode);
|
||||
}
|
||||
}
|
||||
|
||||
private static class RewriteShuttle extends RexShuttle
|
||||
{
|
||||
private final RexBuilder rexBuilder;
|
||||
|
||||
public RewriteShuttle(RexBuilder rexBuilder)
|
||||
{
|
||||
this.rexBuilder = rexBuilder;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RexNode visitOver(RexOver over)
|
||||
{
|
||||
SqlOperator operator = over.getOperator();
|
||||
RexWindow window = over.getWindow();
|
||||
RexWindowBound upperBound = window.getUpperBound();
|
||||
RexWindowBound lowerBound = window.getLowerBound();
|
||||
|
||||
if (window.orderKeys.size() > 0) {
|
||||
if (operator.getKind() == SqlKind.LAST_VALUE && !upperBound.isUnbounded()) {
|
||||
if (upperBound.isCurrentRow()) {
|
||||
return rewriteToReferenceCurrentRow(over);
|
||||
}
|
||||
}
|
||||
if (operator.getKind() == SqlKind.FIRST_VALUE && !lowerBound.isUnbounded()) {
|
||||
if (lowerBound.isCurrentRow()) {
|
||||
return rewriteToReferenceCurrentRow(over);
|
||||
}
|
||||
}
|
||||
}
|
||||
return super.visitOver(over);
|
||||
}
|
||||
|
||||
private RexNode rewriteToReferenceCurrentRow(RexOver over)
|
||||
{
|
||||
// could remove `last_value( x ) over ( .... order by y )`
|
||||
// best would be to: return over.getOperands().get(0);
|
||||
// however that make some queries too good
|
||||
return makeOver(
|
||||
over,
|
||||
over.getWindow(),
|
||||
SqlStdOperatorTable.LAG,
|
||||
ImmutableList.of(over.getOperands().get(0), rexBuilder.makeBigintLiteral(BigDecimal.ZERO))
|
||||
);
|
||||
}
|
||||
|
||||
private RexNode makeOver(RexOver over, RexWindow window, SqlAggFunction aggFunction, List<RexNode> operands)
|
||||
{
|
||||
return rexBuilder.makeOver(
|
||||
over.type,
|
||||
aggFunction,
|
||||
operands,
|
||||
window.partitionKeys,
|
||||
window.orderKeys,
|
||||
window.getLowerBound(),
|
||||
window.getUpperBound(),
|
||||
window.isRows(),
|
||||
true,
|
||||
false,
|
||||
over.isDistinct(),
|
||||
over.ignoreNulls()
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -326,6 +326,13 @@ public class BaseCalciteQueryTest extends CalciteTestBase
|
|||
OUTER_LIMIT_CONTEXT.put(PlannerContext.CTX_SQL_OUTER_LIMIT, 2);
|
||||
}
|
||||
|
||||
public static boolean developerIDEdetected()
|
||||
{
|
||||
String javaCmd = System.getProperties().getProperty("sun.java.command", "");
|
||||
boolean isEclipse = javaCmd.contains("org.eclipse.jdt.internal.junit.runner.RemoteTestRunner");
|
||||
return isEclipse;
|
||||
}
|
||||
|
||||
// Generate timestamps for expected results
|
||||
public static long timestamp(final String timeString)
|
||||
{
|
||||
|
|
|
@ -14998,6 +14998,22 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
assertThat(e, invalidSqlIs("The query contains window functions; To run these window functions, specify [enableWindowing] in query context. (line [1], column [13])"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNtileNotSupportedWithFrame()
|
||||
{
|
||||
assumeFeatureAvailable(EngineFeature.WINDOW_FUNCTIONS);
|
||||
|
||||
DruidException e = assertThrows(
|
||||
DruidException.class,
|
||||
() -> testBuilder()
|
||||
.queryContext(ImmutableMap.of(PlannerContext.CTX_ENABLE_WINDOW_FNS, true))
|
||||
.sql("SELECT ntile(4) OVER (ORDER BY dim1 ROWS BETWEEN 1 FOLLOWING AND CURRENT ROW) from druid.foo")
|
||||
.run()
|
||||
);
|
||||
|
||||
assertThat(e, invalidSqlContains("Framing of NTILE is not supported"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInGroupByLimitOutGroupByOrderBy()
|
||||
{
|
||||
|
|
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.RE;
|
||||
|
@ -61,11 +60,7 @@ public class CalciteWindowQueryTest extends BaseCalciteQueryTest
|
|||
|
||||
public static final boolean DUMP_ACTUAL_RESULTS = Boolean.parseBoolean(
|
||||
System.getProperty("druid.tests.sql.dumpActualResults")
|
||||
);
|
||||
|
||||
static {
|
||||
NullHandling.initializeForTests();
|
||||
}
|
||||
) || developerIDEdetected();
|
||||
|
||||
private static final ObjectMapper YAML_JACKSON = new DefaultObjectMapper(new YAMLFactory(), "tests");
|
||||
|
||||
|
|
|
@ -4512,7 +4512,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/defaultFrame/RBUPACR_int7")
|
||||
@Test
|
||||
public void test_frameclause_defaultFrame_RBUPACR_int7()
|
||||
|
@ -4534,7 +4533,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("last_val/lastValFn_34")
|
||||
@Test
|
||||
public void test_last_val_lastValFn_34()
|
||||
|
@ -4542,7 +4540,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("last_val/lastValFn_35")
|
||||
@Test
|
||||
public void test_last_val_lastValFn_35()
|
||||
|
@ -4550,7 +4547,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("last_val/lastValFn_38")
|
||||
@Test
|
||||
public void test_last_val_lastValFn_38()
|
||||
|
@ -4558,7 +4554,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("last_val/lastValFn_39")
|
||||
@Test
|
||||
public void test_last_val_lastValFn_39()
|
||||
|
@ -6040,7 +6035,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_COUNT_MISMATCH)
|
||||
@DrillTest("last_val/lastValFn_33")
|
||||
@Test
|
||||
public void test_last_val_lastValFn_33()
|
||||
|
@ -6128,7 +6122,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("aggregates/aggOWnFn_14")
|
||||
@Test
|
||||
public void test_aggregates_aggOWnFn_14()
|
||||
|
@ -6136,7 +6129,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("aggregates/aggOWnFn_15")
|
||||
@Test
|
||||
public void test_aggregates_aggOWnFn_15()
|
||||
|
@ -6183,7 +6175,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("aggregates/aggOWnFn_39")
|
||||
@Test
|
||||
public void test_aggregates_aggOWnFn_39()
|
||||
|
@ -6191,7 +6182,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("aggregates/aggOWnFn_40")
|
||||
@Test
|
||||
public void test_aggregates_aggOWnFn_40()
|
||||
|
@ -6616,7 +6606,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/defaultFrame/RBUPACR_bgint_7")
|
||||
@Test
|
||||
public void test_frameclause_defaultFrame_RBUPACR_bgint_7()
|
||||
|
@ -6631,7 +6620,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/defaultFrame/RBUPACR_chr_5")
|
||||
@Test
|
||||
public void test_frameclause_defaultFrame_RBUPACR_chr_5()
|
||||
|
@ -6667,7 +6655,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/defaultFrame/RBUPACR_dbl_7")
|
||||
@Test
|
||||
public void test_frameclause_defaultFrame_RBUPACR_dbl_7()
|
||||
|
@ -6696,7 +6683,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/defaultFrame/RBUPACR_dt_5")
|
||||
@Test
|
||||
public void test_frameclause_defaultFrame_RBUPACR_dt_5()
|
||||
|
@ -6718,7 +6704,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/defaultFrame/RBUPACR_int14")
|
||||
@Test
|
||||
public void test_frameclause_defaultFrame_RBUPACR_int14()
|
||||
|
@ -6747,7 +6732,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/defaultFrame/RBUPACR_vchr_5")
|
||||
@Test
|
||||
public void test_frameclause_defaultFrame_RBUPACR_vchr_5()
|
||||
|
@ -6769,7 +6753,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/multipl_wnwds/fval_mulwds")
|
||||
@Test
|
||||
public void test_frameclause_multipl_wnwds_fval_mulwds()
|
||||
|
@ -6777,7 +6760,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/multipl_wnwds/lval_mulwds")
|
||||
@Test
|
||||
public void test_frameclause_multipl_wnwds_lval_mulwds()
|
||||
|
@ -6785,7 +6767,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/multipl_wnwds/mulwind_08")
|
||||
@Test
|
||||
public void test_frameclause_multipl_wnwds_mulwind_08()
|
||||
|
@ -6793,7 +6774,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/multipl_wnwds/mulwind_09")
|
||||
@Test
|
||||
public void test_frameclause_multipl_wnwds_mulwind_09()
|
||||
|
@ -6808,7 +6788,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/RBCRACR/RBCRACR_bgint_6")
|
||||
@Test
|
||||
public void test_frameclause_RBCRACR_RBCRACR_bgint_6()
|
||||
|
@ -6816,7 +6795,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/RBCRACR/RBCRACR_bgint_7")
|
||||
@Test
|
||||
public void test_frameclause_RBCRACR_RBCRACR_bgint_7()
|
||||
|
@ -6831,7 +6809,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/RBCRACR/RBCRACR_char_4")
|
||||
@Test
|
||||
public void test_frameclause_RBCRACR_RBCRACR_char_4()
|
||||
|
@ -6839,7 +6816,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/RBCRACR/RBCRACR_char_5")
|
||||
@Test
|
||||
public void test_frameclause_RBCRACR_RBCRACR_char_5()
|
||||
|
@ -6847,7 +6823,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/RBCRACR/RBCRACR_dbl_6")
|
||||
@Test
|
||||
public void test_frameclause_RBCRACR_RBCRACR_dbl_6()
|
||||
|
@ -6855,7 +6830,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/RBCRACR/RBCRACR_dbl_7")
|
||||
@Test
|
||||
public void test_frameclause_RBCRACR_RBCRACR_dbl_7()
|
||||
|
@ -6877,7 +6851,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/RBCRACR/RBCRACR_dt_4")
|
||||
@Test
|
||||
public void test_frameclause_RBCRACR_RBCRACR_dt_4()
|
||||
|
@ -6885,7 +6858,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/RBCRACR/RBCRACR_dt_5")
|
||||
@Test
|
||||
public void test_frameclause_RBCRACR_RBCRACR_dt_5()
|
||||
|
@ -6893,7 +6865,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/RBCRACR/RBCRACR_int_13")
|
||||
@Test
|
||||
public void test_frameclause_RBCRACR_RBCRACR_int_13()
|
||||
|
@ -6901,7 +6872,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/RBCRACR/RBCRACR_int_14")
|
||||
@Test
|
||||
public void test_frameclause_RBCRACR_RBCRACR_int_14()
|
||||
|
@ -6909,7 +6879,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/RBCRACR/RBCRACR_int_6")
|
||||
@Test
|
||||
public void test_frameclause_RBCRACR_RBCRACR_int_6()
|
||||
|
@ -6917,7 +6886,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/RBCRACR/RBCRACR_int_7")
|
||||
@Test
|
||||
public void test_frameclause_RBCRACR_RBCRACR_int_7()
|
||||
|
@ -6932,7 +6900,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/RBCRACR/RBCRACR_vchar_4")
|
||||
@Test
|
||||
public void test_frameclause_RBCRACR_RBCRACR_vchar_4()
|
||||
|
@ -6940,7 +6907,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/RBCRACR/RBCRACR_vchar_5")
|
||||
@Test
|
||||
public void test_frameclause_RBCRACR_RBCRACR_vchar_5()
|
||||
|
@ -6983,7 +6949,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/RBUPACR/RBUPACR_bgint_7")
|
||||
@Test
|
||||
public void test_frameclause_RBUPACR_RBUPACR_bgint_7()
|
||||
|
@ -6998,7 +6963,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/RBUPACR/RBUPACR_chr_5")
|
||||
@Test
|
||||
public void test_frameclause_RBUPACR_RBUPACR_chr_5()
|
||||
|
@ -7034,7 +6998,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/RBUPACR/RBUPACR_dbl_7")
|
||||
@Test
|
||||
public void test_frameclause_RBUPACR_RBUPACR_dbl_7()
|
||||
|
@ -7056,7 +7019,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/RBUPACR/RBUPACR_int14")
|
||||
@Test
|
||||
public void test_frameclause_RBUPACR_RBUPACR_int14()
|
||||
|
@ -7071,7 +7033,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/RBUPACR/RBUPACR_vchr_5")
|
||||
@Test
|
||||
public void test_frameclause_RBUPACR_RBUPACR_vchr_5()
|
||||
|
@ -7205,7 +7166,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/subQueries/frmInSubQry_61")
|
||||
@Test
|
||||
public void test_frameclause_subQueries_frmInSubQry_61()
|
||||
|
@ -7220,7 +7180,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/subQueries/frmInSubQry_63")
|
||||
@Test
|
||||
public void test_frameclause_subQueries_frmInSubQry_63()
|
||||
|
@ -7228,7 +7187,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("frameclause/subQueries/frmInSubQry_64")
|
||||
@Test
|
||||
public void test_frameclause_subQueries_frmInSubQry_64()
|
||||
|
@ -7250,7 +7208,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("last_val/lastValFn_10")
|
||||
@Test
|
||||
public void test_last_val_lastValFn_10()
|
||||
|
@ -7258,7 +7215,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("last_val/lastValFn_11")
|
||||
@Test
|
||||
public void test_last_val_lastValFn_11()
|
||||
|
@ -7266,7 +7222,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("last_val/lastValFn_12")
|
||||
@Test
|
||||
public void test_last_val_lastValFn_12()
|
||||
|
@ -7274,7 +7229,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("last_val/lastValFn_13")
|
||||
@Test
|
||||
public void test_last_val_lastValFn_13()
|
||||
|
@ -7282,7 +7236,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("last_val/lastValFn_14")
|
||||
@Test
|
||||
public void test_last_val_lastValFn_14()
|
||||
|
@ -7290,7 +7243,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("last_val/lastValFn_15")
|
||||
@Test
|
||||
public void test_last_val_lastValFn_15()
|
||||
|
@ -7298,7 +7250,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("last_val/lastValFn_16")
|
||||
@Test
|
||||
public void test_last_val_lastValFn_16()
|
||||
|
@ -7306,7 +7257,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("last_val/lastValFn_17")
|
||||
@Test
|
||||
public void test_last_val_lastValFn_17()
|
||||
|
@ -7314,7 +7264,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("last_val/lastValFn_1")
|
||||
@Test
|
||||
public void test_last_val_lastValFn_1()
|
||||
|
@ -7322,7 +7271,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("last_val/lastValFn_20")
|
||||
@Test
|
||||
public void test_last_val_lastValFn_20()
|
||||
|
@ -7330,7 +7278,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("last_val/lastValFn_23")
|
||||
@Test
|
||||
public void test_last_val_lastValFn_23()
|
||||
|
@ -7338,7 +7285,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("last_val/lastValFn_2")
|
||||
@Test
|
||||
public void test_last_val_lastValFn_2()
|
||||
|
@ -7346,7 +7292,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("last_val/lastValFn_32")
|
||||
@Test
|
||||
public void test_last_val_lastValFn_32()
|
||||
|
@ -7354,7 +7299,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("last_val/lastValFn_36")
|
||||
@Test
|
||||
public void test_last_val_lastValFn_36()
|
||||
|
@ -7362,7 +7306,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("last_val/lastValFn_3")
|
||||
@Test
|
||||
public void test_last_val_lastValFn_3()
|
||||
|
@ -7370,7 +7313,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("last_val/lastValFn_4")
|
||||
@Test
|
||||
public void test_last_val_lastValFn_4()
|
||||
|
@ -7378,7 +7320,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("last_val/lastValFn_6")
|
||||
@Test
|
||||
public void test_last_val_lastValFn_6()
|
||||
|
@ -7386,7 +7327,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("last_val/lastValFn_7")
|
||||
@Test
|
||||
public void test_last_val_lastValFn_7()
|
||||
|
@ -7394,7 +7334,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("last_val/lastValFn_8")
|
||||
@Test
|
||||
public void test_last_val_lastValFn_8()
|
||||
|
@ -7402,7 +7341,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.RESULT_MISMATCH)
|
||||
@DrillTest("last_val/lastValFn_9")
|
||||
@Test
|
||||
public void test_last_val_lastValFn_9()
|
||||
|
@ -7622,7 +7560,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.T_ALLTYPES_ISSUES)
|
||||
@DrillTest("frameclause/multipl_wnwds/rnkNoFrm01")
|
||||
@Test
|
||||
public void test_frameclause_multipl_wnwds_rnkNoFrm01()
|
||||
|
@ -7630,7 +7567,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.T_ALLTYPES_ISSUES)
|
||||
@DrillTest("frameclause/multipl_wnwds/rnkNoFrm02")
|
||||
@Test
|
||||
public void test_frameclause_multipl_wnwds_rnkNoFrm02()
|
||||
|
@ -7645,7 +7581,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.T_ALLTYPES_ISSUES)
|
||||
@DrillTest("frameclause/multipl_wnwds/rnkNoFrm04")
|
||||
@Test
|
||||
public void test_frameclause_multipl_wnwds_rnkNoFrm04()
|
||||
|
@ -7653,7 +7588,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.T_ALLTYPES_ISSUES)
|
||||
@DrillTest("frameclause/multipl_wnwds/rnkNoFrm05")
|
||||
@Test
|
||||
public void test_frameclause_multipl_wnwds_rnkNoFrm05()
|
||||
|
@ -7661,7 +7595,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.T_ALLTYPES_ISSUES)
|
||||
@DrillTest("frameclause/multipl_wnwds/rnkNoFrm06")
|
||||
@Test
|
||||
public void test_frameclause_multipl_wnwds_rnkNoFrm06()
|
||||
|
@ -7690,7 +7623,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.T_ALLTYPES_ISSUES)
|
||||
@DrillTest("frameclause/RBUPACR/RBUPACR_dt_5")
|
||||
@Test
|
||||
public void test_frameclause_RBUPACR_RBUPACR_dt_5()
|
||||
|
@ -7698,7 +7630,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
windowQueryTest();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.T_ALLTYPES_ISSUES)
|
||||
@DrillTest("frameclause/RBUPACR/RBUPACR_int7")
|
||||
@Test
|
||||
public void test_frameclause_RBUPACR_RBUPACR_int7()
|
||||
|
|
|
@ -89,8 +89,6 @@ public @interface NotYetSupported
|
|||
ALLDATA_CSV(DruidException.class, "allData.csv"),
|
||||
BIGINT_TIME_COMPARE(DruidException.class, "Cannot apply '.' to arguments of type"),
|
||||
INCORRECT_SYNTAX(DruidException.class, "Incorrect syntax near the keyword"),
|
||||
// at least c7 is represented oddly in the parquet file
|
||||
T_ALLTYPES_ISSUES(AssertionError.class, "(t_alltype|allTypsUniq|fewRowsAllData).parquet.*Verifier.verify"),
|
||||
RESULT_MISMATCH(AssertionError.class, "(assertResulEquals|AssertionError: column content mismatch)"),
|
||||
UNSUPPORTED_NULL_ORDERING(DruidException.class, "(A|DE)SCENDING ordering with NULLS (LAST|FIRST)"),
|
||||
WINDOW_OPERATOR_QUERY_ON_UNSUPPORTED_DATASOURCE(DruidException.class, "WindowOperatorQuery must run on top of a query or inline data source"),
|
||||
|
|
|
@ -104,7 +104,7 @@ public class NotYetSupportedUsageTest
|
|||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return " | " + className + " | " + methodNames.size() + " | " + mode + " | ";
|
||||
return " | " + className + " | " + methodNames.size() + " | " + mode + " | " + mode.regex + " | ";
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.druid.segment.column.ColumnType;
|
|||
import org.apache.druid.sql.calcite.expression.DirectOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.OperatorConversions;
|
||||
import org.apache.druid.sql.calcite.expression.SqlOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.rel.Windowing;
|
||||
import org.apache.druid.sql.calcite.table.RowSignatures;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
@ -36,6 +37,9 @@ import org.junit.Test;
|
|||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.junit.Assert.assertFalse;
|
||||
|
||||
public class DruidOperatorTableTest
|
||||
{
|
||||
|
@ -97,4 +101,27 @@ public class DruidOperatorTableTest
|
|||
Assert.assertTrue(DruidOperatorTable.isFunctionSyntax(operator1.getSyntax()));
|
||||
Assert.assertFalse(DruidOperatorTable.isFunctionSyntax(operator2.getSyntax()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBuiltinWindowOperatorsSupportFramingAsExpected()
|
||||
{
|
||||
DruidOperatorTable operatorTable = new DruidOperatorTable(ImmutableSet.of(), ImmutableSet.of());
|
||||
ImmutableSet<String> keySet = Windowing.KNOWN_WINDOW_FNS.keySet();
|
||||
Set<SqlOperator> windowOps = operatorTable.getOperatorList().stream()
|
||||
.filter(o -> keySet.contains(o.getKind().toString())).collect(Collectors.toSet());
|
||||
for (SqlOperator operator : windowOps) {
|
||||
switch (operator.kind) {
|
||||
case FIRST_VALUE:
|
||||
case LAST_VALUE:
|
||||
case NTILE:
|
||||
// These are handled with DruidSqlValidator and a rewrite rule.
|
||||
continue;
|
||||
default:
|
||||
assertFalse(
|
||||
operator + " allows framing; should be supported or rejected and then exclude from this check",
|
||||
operator.allowsFraming()
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,23 @@
|
|||
type: "operatorValidation"
|
||||
|
||||
sql: |
|
||||
SELECT
|
||||
LAST_VALUE(dim1) OVER ()
|
||||
= 'abc',
|
||||
FIRST_VALUE(dim1) OVER ()
|
||||
= '',
|
||||
LAST_VALUE(dim1) OVER (ORDER BY m1 ASC)
|
||||
is not distinct from dim1,
|
||||
LAST_VALUE(dim1) OVER (ORDER BY m1 DESC)
|
||||
is not distinct from dim1,
|
||||
m1,dim1
|
||||
FROM foo
|
||||
ORDER by m1
|
||||
|
||||
expectedResults:
|
||||
- [true,true,true,true,1.0,""]
|
||||
- [true,true,true,true,2.0,"10.1"]
|
||||
- [true,true,true,true,3.0,"2"]
|
||||
- [true,true,true,true,4.0,"1"]
|
||||
- [true,true,true,true,5.0,"def"]
|
||||
- [true,true,true,true,6.0,"abc"]
|
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
Loading…
Reference in New Issue