diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownVirtualColumnsIntoOlapScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownVirtualColumnsIntoOlapScan.java index 131a935db5391c..e7188dbffaefc8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownVirtualColumnsIntoOlapScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownVirtualColumnsIntoOlapScan.java @@ -42,9 +42,38 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.types.ArrayType; +import org.apache.doris.nereids.types.BigIntType; +import org.apache.doris.nereids.types.BitmapType; +import org.apache.doris.nereids.types.BooleanType; +import org.apache.doris.nereids.types.CharType; +import org.apache.doris.nereids.types.DataType; +import org.apache.doris.nereids.types.DateTimeType; +import org.apache.doris.nereids.types.DateTimeV2Type; +import org.apache.doris.nereids.types.DateType; +import org.apache.doris.nereids.types.DateV2Type; +import org.apache.doris.nereids.types.DecimalV2Type; +import org.apache.doris.nereids.types.DecimalV3Type; +import org.apache.doris.nereids.types.DoubleType; +import org.apache.doris.nereids.types.FloatType; +import org.apache.doris.nereids.types.HllType; +import org.apache.doris.nereids.types.IPv4Type; +import org.apache.doris.nereids.types.IPv6Type; +import org.apache.doris.nereids.types.IntegerType; +import org.apache.doris.nereids.types.JsonType; +import org.apache.doris.nereids.types.LargeIntType; +import org.apache.doris.nereids.types.MapType; +import org.apache.doris.nereids.types.QuantileStateType; +import org.apache.doris.nereids.types.SmallIntType; +import org.apache.doris.nereids.types.StringType; +import org.apache.doris.nereids.types.StructType; +import org.apache.doris.nereids.types.TinyIntType; +import org.apache.doris.nereids.types.VarcharType; +import org.apache.doris.nereids.types.VariantType; import org.apache.doris.nereids.util.ExpressionUtils; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Maps; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -107,6 +136,38 @@ public class PushDownVirtualColumnsIntoOlapScan implements RewriteRuleFactory { // Logger for debugging private static final Logger logger = LogManager.getLogger(PushDownVirtualColumnsIntoOlapScan.class); + // Supported data types for virtual columns based on TabletColumn::get_field_length_by_type + // Using Nereids DataType instead of TPrimitiveType for better type safety + private static final ImmutableSet> SUPPORTED_VIRTUAL_COLUMN_TYPES = ImmutableSet.of( + TinyIntType.class, + BooleanType.class, + SmallIntType.class, + IntegerType.class, + BigIntType.class, + LargeIntType.class, + IPv4Type.class, + IPv6Type.class, + DateType.class, + DateV2Type.class, + DateTimeType.class, + DateTimeV2Type.class, + FloatType.class, + DoubleType.class, + QuantileStateType.class, + BitmapType.class, + CharType.class, + VarcharType.class, + HllType.class, + StringType.class, + VariantType.class, + JsonType.class, + StructType.class, + ArrayType.class, + MapType.class, + DecimalV2Type.class, + DecimalV3Type.class + ); + @Override public List buildRules() { return ImmutableList.of( @@ -205,6 +266,7 @@ private void extractRepeatedSubExpressions(LogicalFilter filter expressionCounts.entrySet().stream() .filter(entry -> entry.getValue() >= MIN_OCCURRENCE_COUNT) .filter(entry -> !replaceMap.containsKey(entry.getKey())) + .filter(entry -> isSupportedVirtualColumnType(entry.getKey())) // Check virtual column type support .sorted((e1, e2) -> { // Sort by benefit: (occurrence_count - 1) * expression_complexity int benefit1 = (e1.getValue() - 1) * getExpressionComplexity(e1.getKey()); @@ -217,6 +279,11 @@ private void extractRepeatedSubExpressions(LogicalFilter filter Alias alias = new Alias(expr); replaceMap.put(expr, alias.toSlot()); virtualColumnsBuilder.add(alias); + + if (LOG.isDebugEnabled()) { + LOG.debug("Created virtual column for expression: {} with type: {}", + expr.toSql(), expr.getDataType().simpleString()); + } }); // Logging for debugging @@ -438,6 +505,32 @@ private boolean isIndexPushdownFunction(Expression expr) { } } + /** + * Check if an expression's return type is supported for virtual columns + * Based on TabletColumn::get_field_length_by_type implementation + */ + private boolean isSupportedVirtualColumnType(Expression expr) { + try { + DataType dataType = expr.getDataType(); + Class typeClass = dataType.getClass(); + + boolean isSupported = SUPPORTED_VIRTUAL_COLUMN_TYPES.contains(typeClass); + + if (LOG.isDebugEnabled()) { + LOG.debug("Expression {} has type {} (class: {}), supported: {}", + expr.toSql(), dataType.simpleString(), typeClass.getSimpleName(), isSupported); + } + + return isSupported; + } catch (Exception e) { + // If we can't determine the type, it's safer to not create virtual column + if (LOG.isDebugEnabled()) { + LOG.debug("Failed to determine type for expression {}: {}", expr.toSql(), e.getMessage()); + } + return false; + } + } + /** * Get function name from expression if it's a function call */ diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushDownVirtualColumnsIntoOlapScanTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushDownVirtualColumnsIntoOlapScanTest.java index 3438152698a833..9a1e385261426b 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushDownVirtualColumnsIntoOlapScanTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushDownVirtualColumnsIntoOlapScanTest.java @@ -30,8 +30,10 @@ import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Not; import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.functions.scalar.Concat; import org.apache.doris.nereids.trees.expressions.functions.scalar.IsIpAddressInRange; import org.apache.doris.nereids.trees.expressions.functions.scalar.L2Distance; +import org.apache.doris.nereids.trees.expressions.functions.scalar.Lambda; import org.apache.doris.nereids.trees.expressions.functions.scalar.MultiMatch; import org.apache.doris.nereids.trees.expressions.functions.scalar.MultiMatchAny; import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral; @@ -591,4 +593,242 @@ public void testContainsIndexPushdownFunction_NestedExpression() { throw new RuntimeException("Failed to test containsIndexPushdownFunction", e); } } + + @Test + public void testIsSupportedVirtualColumnType() { + try { + PushDownVirtualColumnsIntoOlapScan rule = new PushDownVirtualColumnsIntoOlapScan(); + + // Use reflection to access the private method + java.lang.reflect.Method method = rule.getClass() + .getDeclaredMethod("isSupportedVirtualColumnType", + org.apache.doris.nereids.trees.expressions.Expression.class); + method.setAccessible(true); + + DataType intType = IntegerType.INSTANCE; + DataType varcharType = VarcharType.createVarcharType(100); + + // Test supported types + SlotReference intSlot = new SlotReference("int_col", intType); + SlotReference varcharSlot = new SlotReference("varchar_col", varcharType); + + // Test basic arithmetic expression with supported types (should return integer) + Add intAddition = new Add(intSlot, new IntegerLiteral(1)); + boolean intSupported = (boolean) method.invoke(rule, intAddition); + assert intSupported : "Integer arithmetic expression should be supported for virtual columns"; + + // Test string concatenation (should return varchar) + Concat stringConcat = new Concat(varcharSlot, new StringLiteral("_suffix")); + boolean stringSupported = (boolean) method.invoke(rule, stringConcat); + assert stringSupported : "String expression should be supported for virtual columns"; + + // Test a complex expression with multiple operations + Multiply complexMath = new Multiply( + new Add(intSlot, new IntegerLiteral(5)), + new IntegerLiteral(2) + ); + boolean complexSupported = (boolean) method.invoke(rule, complexMath); + assert complexSupported : "Complex arithmetic expression should be supported for virtual columns"; + + // Test a CAST expression to string (should be supported) + Cast castToString = new Cast(intSlot, VarcharType.createVarcharType(50)); + boolean castSupported = (boolean) method.invoke(rule, castToString); + assert castSupported : "CAST to supported type should be supported for virtual columns"; + + } catch (Exception e) { + throw new RuntimeException("Failed to test isSupportedVirtualColumnType", e); + } + } + + @Test + public void testUnsupportedVirtualColumnType() { + try { + PushDownVirtualColumnsIntoOlapScan rule = new PushDownVirtualColumnsIntoOlapScan(); + + // Use reflection to access the private method + java.lang.reflect.Method method = rule.getClass() + .getDeclaredMethod("isSupportedVirtualColumnType", + org.apache.doris.nereids.trees.expressions.Expression.class); + method.setAccessible(true); + + // Test expression that might return an unsupported type + // Create a lambda function expression which should not be supported + DataType intType = IntegerType.INSTANCE; + SlotReference intSlot = new SlotReference("int_col", intType); + + // Test expressions that should fail type detection or return false + // We create an expression that might fail during type determination + Lambda lambdaExpr = new Lambda(ImmutableList.of("int_col"), new Add(intSlot, new IntegerLiteral(1))); + + boolean lambdaSupported = (boolean) method.invoke(rule, lambdaExpr); + assert !lambdaSupported : "Lambda expressions should not be supported for virtual columns"; + + } catch (Exception e) { + // Expected for some unsupported expressions + // The test should handle gracefully when expressions cannot be evaluated + } + } + + @Test + public void testVirtualColumnTypeFilteringInExtraction() { + // Test that the extractRepeatedSubExpressions method properly filters out + // expressions with unsupported types during virtual column creation + try { + PushDownVirtualColumnsIntoOlapScan rule = new PushDownVirtualColumnsIntoOlapScan(); + + // Use reflection to access the private extractRepeatedSubExpressions method + java.lang.reflect.Method extractMethod = rule.getClass() + .getDeclaredMethod("extractRepeatedSubExpressions", + org.apache.doris.nereids.trees.plans.logical.LogicalFilter.class, + java.util.Optional.class, + java.util.Map.class, + com.google.common.collect.ImmutableList.Builder.class); + extractMethod.setAccessible(true); + + DataType intType = IntegerType.INSTANCE; + SlotReference x = new SlotReference("x", intType); + SlotReference y = new SlotReference("y", intType); + + // Create expressions that should be supported (arithmetic operations return int) + Add supportedAdd1 = new Add(x, y); + Add supportedAdd2 = new Add(x, y); + Add supportedAdd3 = new Add(x, y); + + // Create filter conditions using the repeated supported expression + GreaterThan gt1 = new GreaterThan(supportedAdd1, new IntegerLiteral(10)); + GreaterThan gt2 = new GreaterThan(supportedAdd2, new IntegerLiteral(20)); + GreaterThan gt3 = new GreaterThan(supportedAdd3, new IntegerLiteral(30)); + + // Create OLAP scan and filter + LogicalOlapScan scan = PlanConstructor.newLogicalOlapScan(0, "t1", 0); + LogicalFilter filter = new LogicalFilter<>( + ImmutableSet.of(gt1, gt2, gt3), scan); + + // Test the extraction method + java.util.Map replaceMap = + new java.util.HashMap<>(); + com.google.common.collect.ImmutableList.Builder + virtualColumnsBuilder = com.google.common.collect.ImmutableList.builder(); + + // Call the extraction method + extractMethod.invoke(rule, filter, java.util.Optional.empty(), replaceMap, virtualColumnsBuilder); + + // Verify that virtual columns were created for supported expressions + java.util.List virtualColumns = + virtualColumnsBuilder.build(); + + // Since Add(x, y) appears 3 times and returns int (supported type), + // it should be included in virtual columns + assert !virtualColumns.isEmpty() : "Should create virtual columns for repeated supported expressions"; + assert replaceMap.size() > 0 : "Should have replacements for supported expressions"; + + // Test that the virtual column expression has a supported type + if (!virtualColumns.isEmpty()) { + org.apache.doris.nereids.trees.expressions.NamedExpression virtualCol = virtualColumns.get(0); + if (virtualCol instanceof org.apache.doris.nereids.trees.expressions.Alias) { + org.apache.doris.nereids.trees.expressions.Alias alias = + (org.apache.doris.nereids.trees.expressions.Alias) virtualCol; + org.apache.doris.nereids.trees.expressions.Expression expr = alias.child(); + + // The expression should be supported by isSupportedVirtualColumnType + java.lang.reflect.Method typeCheckMethod = rule.getClass() + .getDeclaredMethod("isSupportedVirtualColumnType", + org.apache.doris.nereids.trees.expressions.Expression.class); + typeCheckMethod.setAccessible(true); + boolean isSupported = (boolean) typeCheckMethod.invoke(rule, expr); + assert isSupported : "Virtual column expression should have supported type"; + } + } + + } catch (Exception e) { + throw new RuntimeException("Failed to test virtual column type filtering", e); + } + } + + @Test + public void testTypeFilteringWithMixedExpressions() { + // Test extraction with both supported and unsupported expression types + try { + PushDownVirtualColumnsIntoOlapScan rule = new PushDownVirtualColumnsIntoOlapScan(); + + // Use reflection to access private methods + java.lang.reflect.Method extractMethod = rule.getClass() + .getDeclaredMethod("extractRepeatedSubExpressions", + org.apache.doris.nereids.trees.plans.logical.LogicalFilter.class, + java.util.Optional.class, + java.util.Map.class, + com.google.common.collect.ImmutableList.Builder.class); + extractMethod.setAccessible(true); + + java.lang.reflect.Method typeCheckMethod = rule.getClass() + .getDeclaredMethod("isSupportedVirtualColumnType", + org.apache.doris.nereids.trees.expressions.Expression.class); + typeCheckMethod.setAccessible(true); + + DataType intType = IntegerType.INSTANCE; + SlotReference x = new SlotReference("x", intType); + SlotReference y = new SlotReference("y", intType); + + // Create supported repeated expressions (arithmetic operations) + Add supportedExpr1 = new Add(x, y); + Add supportedExpr2 = new Add(x, y); + + // Create potentially unsupported repeated expressions (lambda expressions) + Lambda unsupportedExpr1 = new Lambda(ImmutableList.of("x"), new Add(x, new IntegerLiteral(1))); + Lambda unsupportedExpr2 = new Lambda(ImmutableList.of("x"), new Add(x, new IntegerLiteral(1))); + + // Verify type support status for both supported and unsupported expressions + boolean supportedIsSupported = (boolean) typeCheckMethod.invoke(rule, supportedExpr1); + boolean unsupportedIsSupported1 = (boolean) typeCheckMethod.invoke(rule, unsupportedExpr1); + boolean unsupportedIsSupported2 = (boolean) typeCheckMethod.invoke(rule, unsupportedExpr2); + + assert supportedIsSupported : "Add expression should be supported"; + assert !unsupportedIsSupported1 : "Lambda expression 1 should not be supported"; + assert !unsupportedIsSupported2 : "Lambda expression 2 should not be supported"; + + // Verify that both unsupported expressions have the same type checking result + assert unsupportedIsSupported1 == unsupportedIsSupported2 : + "Both lambda expressions should have the same support status"; + + // Create filter conditions using both types + GreaterThan gt1 = new GreaterThan(supportedExpr1, new IntegerLiteral(10)); + GreaterThan gt2 = new GreaterThan(supportedExpr2, new IntegerLiteral(20)); + // Note: We can't easily create actual filter conditions with lambda expressions + // since they require specific context, so we focus on the type checking + + LogicalOlapScan scan = PlanConstructor.newLogicalOlapScan(0, "t1", 0); + LogicalFilter filter = new LogicalFilter<>( + ImmutableSet.of(gt1, gt2), scan); + + // Test extraction + java.util.Map replaceMap = + new java.util.HashMap<>(); + com.google.common.collect.ImmutableList.Builder + virtualColumnsBuilder = com.google.common.collect.ImmutableList.builder(); + + extractMethod.invoke(rule, filter, java.util.Optional.empty(), replaceMap, virtualColumnsBuilder); + + // Verify results: only supported expressions should create virtual columns + java.util.List virtualColumns = + virtualColumnsBuilder.build(); + + // Should have virtual columns only for supported expressions + for (org.apache.doris.nereids.trees.expressions.NamedExpression virtualCol : virtualColumns) { + if (virtualCol instanceof org.apache.doris.nereids.trees.expressions.Alias) { + org.apache.doris.nereids.trees.expressions.Alias alias = + (org.apache.doris.nereids.trees.expressions.Alias) virtualCol; + org.apache.doris.nereids.trees.expressions.Expression expr = alias.child(); + + boolean isSupported = (boolean) typeCheckMethod.invoke(rule, expr); + assert isSupported : "All virtual column expressions should have supported types"; + } + } + + } catch (Exception e) { + // Expected for lambda expressions or other complex scenarios + // The important thing is that type checking works correctly + } + } } diff --git a/regression-test/data/query_p0/virtual_slot_ref/expr_with_runtime_return_type.out b/regression-test/data/query_p0/virtual_slot_ref/expr_with_runtime_return_type.out new file mode 100644 index 00000000000000..2c4b2881253762 --- /dev/null +++ b/regression-test/data/query_p0/virtual_slot_ref/expr_with_runtime_return_type.out @@ -0,0 +1,3 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !0 -- + diff --git a/regression-test/suites/query_p0/virtual_slot_ref/expr_with_runtime_return_type.groovy b/regression-test/suites/query_p0/virtual_slot_ref/expr_with_runtime_return_type.groovy new file mode 100644 index 00000000000000..80185475304a31 --- /dev/null +++ b/regression-test/suites/query_p0/virtual_slot_ref/expr_with_runtime_return_type.groovy @@ -0,0 +1,60 @@ +// 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. + +suite("expr_with_runtime_return_type") { + sql """ + DROP TABLE IF EXISTS expr_with_runtime_return_type + """ + sql """ + create table expr_with_runtime_return_type ( + col_datetime_6__undef_signed_not_null datetime(6) not null , + pk int, + col_date_undef_signed date null , + col_datetime_undef_signed datetime null , + col_datetime_3__undef_signed_not_null datetime(3) not null , + col_datetime_undef_signed_not_null datetime not null , + col_date_undef_signed_not_null date not null , + col_datetime_3__undef_signed datetime(3) null , + col_datetime_6__undef_signed datetime(6) null + ) engine=olap + DUPLICATE KEY(col_datetime_6__undef_signed_not_null, pk, col_date_undef_signed, col_datetime_undef_signed, col_datetime_3__undef_signed_not_null, col_datetime_undef_signed_not_null, col_date_undef_signed_not_null) + PARTITION BY RANGE(col_datetime_6__undef_signed_not_null) ( + PARTITION p0 VALUES LESS THAN ("1997-01-01 00:00:00"), + PARTITION p1 VALUES LESS THAN ("2010-01-01 00:00:00"), + PARTITION p2 VALUES LESS THAN ("2020-01-01 00:00:00"), + PARTITION p3 VALUES LESS THAN ("2030-01-01 00:00:00"), + PARTITION p4 VALUES LESS THAN ("2040-01-01 00:00:00"), + PARTITION p5 VALUES LESS THAN ("2050-01-01 00:00:00"), + PARTITION p6 VALUES LESS THAN ("2060-01-01 00:00:00"), + PARTITION p7 VALUES LESS THAN ("2070-01-01 00:00:00"), + PARTITION p8 VALUES LESS THAN ("2180-01-01 00:00:00"), + PARTITION p9 VALUES LESS THAN ("9999-01-01 00:00:00"), + PARTITION p10 VALUES LESS THAN (MAXVALUE) + ) + + distributed by hash(pk) buckets 10 + properties("replication_num" = "1"); + """ + sql """ + insert into expr_with_runtime_return_type(pk,col_date_undef_signed,col_date_undef_signed_not_null,col_datetime_undef_signed,col_datetime_undef_signed_not_null,col_datetime_3__undef_signed,col_datetime_3__undef_signed_not_null,col_datetime_6__undef_signed,col_datetime_6__undef_signed_not_null) values (0,'2017-06-25','2000-07-17','2014-04-09','2014-08-12','9999-12-31','2013-10-18','2000-07-12 04:43:12','2024-08-03 13:08:30'),(1,'2012-12-21 09:07:28','2023-01-15 08:32:59','2024-08-03 13:08:30','2024-06-30 12:01:02','2018-04-28 12:57:41','2014-08-12','2024-07-01','2011-11-24 00:42:38'),(2,'2024-08-03 13:08:30','2016-02-21 03:11:25','2014-12-04 19:44:01','2006-01-07','9999-12-31','9999-12-31','2017-01-28 17:43:30','2024-07-01'),(3,'2023-01-15 08:32:59','2012-01-23 22:26:11','2014-08-12','2024-06-30 12:01:02','2003-10-03 19:01:11','2018-05-01','2005-06-04 21:54:42','9999-12-31 23:59:59'),(4,'2016-06-12','9999-12-31 23:59:59','9999-12-31 23:59:59','2024-07-01',null,'9999-12-31 23:59:59','2009-05-13 01:10:11','2019-10-03 17:54:05'),(5,null,'2024-08-03 13:08:30','2014-08-12','2010-03-19 22:55:36','2024-06-30 12:01:02','2013-12-13 18:32:56','2023-01-15 08:32:59','2014-08-12'),(6,'2014-08-12','2005-03-10','2006-07-01 07:56:55','2014-08-12','2011-01-15','9999-12-31 23:59:59','9999-12-31','2011-01-11'),(7,'2007-08-11 00:47:18','2018-11-10 00:41:09','2000-10-27 04:30:20','2018-09-05','2018-01-14 23:15:28','2014-08-12','2010-02-02','2014-08-12'),(8,'2013-02-03','2006-10-20','2005-06-13','2024-06-30 12:01:02','2024-08-03 13:08:30','2014-08-12','2024-06-30 12:01:02','2018-10-17'),(9,null,'2014-08-12','9999-12-31 23:59:59','2002-05-05 05:16:57','2014-08-12','2014-08-12','2024-06-30 12:01:02','2024-07-01'),(10,'2017-09-15','9999-12-31','2024-08-03 13:08:30','2010-12-15','2000-01-22 04:53:17','2016-05-01 12:33:49','2017-01-06','2012-02-04 13:32:35'),(11,'2007-12-21 09:36:57','2009-12-03','9999-12-31 23:59:59','9999-12-31 23:59:59','9999-12-31 23:59:59','2014-08-12','2005-09-08','9999-12-31'),(12,'2005-08-18 13:06:00','9999-12-31 23:59:59','2005-01-12 18:23:14','9999-12-31','9999-12-31 23:59:59','2014-08-12','2002-01-27','2001-11-02'),(13,'9999-12-31 23:59:59','2023-01-15 08:32:59','2024-08-03 13:08:30','2006-06-14 14:07:27','2024-06-30 12:01:02','9999-12-31',null,'9999-12-31'),(14,'2006-05-21 15:51:22','9999-12-31','2016-08-24 04:10:20','2000-02-18 05:35:17',null,'2024-07-01','2023-01-15 08:32:59','2024-08-03 13:08:30'),(15,'2012-01-16 12:14:10','2004-02-11',null,'2024-07-01','2024-07-01','2002-10-27','2024-06-30 12:01:02','9999-12-31'),(16,'2023-01-15 08:32:59','2024-07-01','2023-01-15 08:32:59','9999-12-31 23:59:59','2024-06-30 12:01:02','2003-08-21','2004-01-28','2024-06-30 12:01:02'),(17,'2001-05-12 09:46:54','9999-12-31 23:59:59','9999-12-31 23:59:59','2024-08-03 13:08:30','2023-01-15 08:32:59','2024-08-03 13:08:30','2006-09-05','2009-09-21 06:38:50'),(18,'2019-01-16','2024-07-01','2003-10-04 02:01:40','2024-07-01','2014-05-25','2016-02-28','2014-08-12','2013-12-06 14:36:04'),(19,'2024-07-01','2014-11-01 23:28:51','9999-12-31 23:59:59','9999-12-31 23:59:59','2024-08-03 13:08:30','9999-12-31 23:59:59','2023-01-15 08:32:59','9999-12-31'),(20,'2009-09-24 14:51:17','2024-08-03 13:08:30','2024-08-03 13:08:30','2019-05-06 23:28:43','2011-05-26 19:01:46','2006-07-10','9999-12-31 23:59:59','2023-01-15 08:32:59'),(21,'2008-04-05','2003-01-11 01:00:20','2024-08-03 13:08:30','2013-03-03 21:44:57','2024-07-01','9999-12-31','2024-07-01','9999-12-31'),(22,'2019-09-11 03:52:21','2023-01-15 08:32:59','2011-12-24 14:40:06','2024-06-30 12:01:02','9999-12-31 23:59:59','2009-06-06','2013-06-28 23:17:39','2023-01-15 08:32:59'),(23,'2024-08-03 13:08:30','2001-08-01 03:55:09','2024-06-30 12:01:02','2024-08-03 13:08:30','9999-12-31 23:59:59','2024-06-30 12:01:02','2006-08-26 01:21:12','2002-12-14 00:11:14'),(24,'2024-06-30 12:01:02','2024-07-01','2014-08-12','9999-12-31 23:59:59','2023-01-15 08:32:59','2024-06-30 12:01:02','2024-07-01','2024-08-03 13:08:30'),(25,'2002-02-21','2014-08-12','2014-08-12','2024-06-30 12:01:02','9999-12-31 23:59:59','9999-12-31','2008-02-13 13:14:28','2024-06-30 12:01:02'),(26,'2002-12-17','2006-03-17','2024-08-03 13:08:30','2014-06-10',null,'9999-12-31','2014-08-12','9999-12-31 23:59:59'),(27,'2024-06-30 12:01:02','2023-01-15 08:32:59','9999-12-31 23:59:59','9999-12-31','2006-05-25','9999-12-31','2014-08-12','2023-01-15 08:32:59'),(28,'9999-12-31 23:59:59','2013-11-17 16:51:07','2014-08-12','2006-06-03','2016-05-04','9999-12-31 23:59:59',null,'2024-08-03 13:08:30'),(29,'2024-07-01','2024-06-30 12:01:02','2023-01-15 08:32:59','9999-12-31 23:59:59','2024-07-01','2008-08-25','2010-03-02','2024-07-01'),(30,null,'9999-12-31','2007-08-13 12:49:37','2024-06-30 12:01:02',null,'2023-01-15 08:32:59','2024-06-30 12:01:02','2024-07-01'),(31,'2009-04-16','9999-12-31','2002-04-28','2023-01-15 08:32:59','2024-07-01','2024-08-03 13:08:30',null,'2008-06-18'),(32,'2023-01-15 08:32:59','9999-12-31 23:59:59','9999-12-31','2024-08-03 13:08:30','2004-12-18 16:13:18','2014-08-12','2012-09-24 14:20:23','2008-10-20'),(33,'2014-08-12','2024-07-01','2024-08-03 13:08:30','2002-09-26',null,'2024-06-30 12:01:02','9999-12-31','2023-01-15 08:32:59'),(34,'9999-12-31 23:59:59','2023-01-15 08:32:59','2024-06-30 12:01:02','2014-08-12','2024-06-30 12:01:02','2014-08-12','2000-11-22','2016-05-09'),(35,'2005-10-11','9999-12-31 23:59:59','2018-08-02 09:25:27','2024-08-03 13:08:30','2000-06-11','2014-08-12','2024-06-30 12:01:02','2017-02-26'),(36,'2014-08-12','2024-06-30 12:01:02','9999-12-31 23:59:59','9999-12-31 23:59:59','2004-02-22','9999-12-31','2015-07-17 00:52:24','9999-12-31 23:59:59'),(37,'9999-12-31 23:59:59','2010-06-12 00:46:26','2024-08-03 13:08:30','2007-08-16','9999-12-31','2014-08-12','2024-06-30 12:01:02','2018-03-18'),(38,'2004-03-13','2023-01-15 08:32:59','9999-12-31 23:59:59','9999-12-31','2024-06-30 12:01:02','2013-10-07','9999-12-31','2002-07-24 09:05:49'),(39,'2014-12-26 20:30:21','2019-06-08','2014-05-09 00:48:27','2014-08-12','2008-02-11 20:08:30','9999-12-31 23:59:59',null,'2005-02-12'),(40,'2015-04-20','2013-07-25','2014-04-24 20:12:41','2014-08-12','2019-02-24','2002-10-06 10:17:20','2014-08-12','9999-12-31'),(41,'2024-07-01','2011-12-28','2023-01-15 08:32:59','9999-12-31','2023-01-15 08:32:59','2005-02-28 08:40:52',null,'2019-07-23 23:07:44'),(42,'2024-08-03 13:08:30','2004-02-10','2000-03-22','2024-06-30 12:01:02','2024-07-01','2016-05-17 10:26:03','2023-01-15 08:32:59','2024-08-03 13:08:30'),(43,'2024-07-01','2024-07-01','2003-05-19 21:41:22','2014-08-12','2018-09-25','2009-04-23','2014-08-12','2017-12-03 09:05:03'),(44,'9999-12-31','9999-12-31 23:59:59','2024-06-30 12:01:02','2006-09-13','2014-08-12','2024-08-03 13:08:30','2023-01-15 08:32:59','2023-01-15 08:32:59'),(45,'2024-06-30 12:01:02','9999-12-31',null,'2024-06-30 12:01:02','2011-04-27','2006-01-21 15:02:52','2003-11-18','2014-08-12'),(46,'2011-02-24','9999-12-31 23:59:59','2024-07-01','2024-08-03 13:08:30','2013-12-22','2008-08-21',null,'9999-12-31 23:59:59'),(47,'2014-08-12','9999-12-31','2024-08-03 13:08:30','9999-12-31','2014-08-12','2002-12-23 04:32:14','2024-07-01','2014-08-12'),(48,'2018-12-20','2024-06-30 12:01:02','2023-01-15 08:32:59','2023-01-15 08:32:59','2004-09-21 20:10:42','2019-06-10','2024-08-03 13:08:30','2023-01-15 08:32:59'),(49,'2015-06-19 13:59:51','2024-07-01','2007-11-16 10:58:01','2010-07-17 05:05:44','2005-09-06 01:09:42','9999-12-31 23:59:59','2024-08-03 13:08:30','2018-09-07 01:24:27'),(50,'9999-12-31','2024-06-30 12:01:02','2005-01-03','2018-12-02','2024-08-03 13:08:30','9999-12-31','2024-08-03 13:08:30','2024-08-03 13:08:30'),(51,'2000-04-02','2014-08-12','2014-08-12','2024-07-01','9999-12-31','2024-06-30 12:01:02','2024-07-01','2024-06-30 12:01:02'),(52,'2024-06-30 12:01:02','2024-08-03 13:08:30','2024-07-01','2016-05-06','2023-01-15 08:32:59','2010-08-10 18:52:15','2013-03-19 05:32:41','9999-12-31 23:59:59'),(53,'2002-03-15','9999-12-31','2024-07-01','2024-08-03 13:08:30','2023-01-15 08:32:59','2010-07-25',null,'2013-08-23'),(54,'9999-12-31','2014-08-12','2011-03-18','2000-09-16','2024-06-30 12:01:02','9999-12-31 23:59:59','2010-02-01','2001-06-28 04:47:33'),(55,'2001-06-12 15:07:04','2024-08-03 13:08:30',null,'2003-10-20 07:33:21','2023-01-15 08:32:59','2013-11-13 11:00:01','2006-02-20','9999-12-31'),(56,'2011-04-07','2024-06-30 12:01:02','2024-07-01','2012-09-07 04:07:24','2013-06-11','2024-06-30 12:01:02','2024-08-03 13:08:30','2024-07-01'),(57,'9999-12-31','2001-11-10','2024-06-30 12:01:02','2024-07-01',null,'2018-06-08 23:39:18',null,'2011-11-27 03:05:08'),(58,'2024-08-03 13:08:30','9999-12-31 23:59:59','2005-03-08 13:31:05','2016-08-10','2014-08-12','2024-06-30 12:01:02','2014-08-12','2008-05-01'),(59,'2014-08-12','9999-12-31','2001-11-23','9999-12-31 23:59:59','9999-12-31 23:59:59','2010-02-25 17:05:44','2006-02-13','2012-05-04'),(60,'9999-12-31','2016-12-01 19:57:27','2015-10-06 12:30:44','9999-12-31','2019-06-24 00:03:15','2024-06-30 12:01:02','2012-10-28','2024-08-03 13:08:30'),(61,'9999-12-31 23:59:59','2010-02-25',null,'2014-08-12','2024-07-01','2002-04-16 04:05:24','2024-06-30 12:01:02','9999-12-31 23:59:59'),(62,'2009-05-03','2024-06-30 12:01:02','2002-06-01 11:03:40','2014-08-12','2024-07-01','2012-02-28','2024-07-01','2024-06-30 12:01:02'),(63,'2015-11-07 03:11:34','2014-08-12','9999-12-31 23:59:59','2023-01-15 08:32:59','2001-11-16 14:22:25','9999-12-31 23:59:59','2024-08-03 13:08:30','2002-03-27 21:05:07'),(64,null,'2012-11-13','2013-10-08','2023-01-15 08:32:59','9999-12-31 23:59:59','9999-12-31','2024-06-30 12:01:02','2014-08-12'),(65,'2024-07-01','2014-08-12','2024-07-01','2014-08-12','2014-08-12','2024-08-03 13:08:30','2023-01-15 08:32:59','9999-12-31 23:59:59'),(66,'2011-08-03','2024-07-01','2024-07-01','2017-01-07 21:55:37','2012-09-15 01:25:39','2006-03-24 18:01:18','2003-06-24','9999-12-31'),(67,'2016-07-14 10:09:24','2007-04-05','2023-01-15 08:32:59','2024-06-30 12:01:02','9999-12-31 23:59:59','9999-12-31','2024-06-30 12:01:02','2018-04-05 13:23:12'),(68,'2024-07-01','2002-10-16 06:59:59','2019-07-08','2005-12-05 15:56:05','2011-06-01 00:44:14','2024-07-01','2024-08-03 13:08:30','2024-06-30 12:01:02'),(69,null,'2002-02-10 03:03:16','2024-06-30 12:01:02','9999-12-31 23:59:59','2014-12-15','2017-11-27','9999-12-31','2023-01-15 08:32:59'),(70,'2024-07-01','2024-08-03 13:08:30','2024-08-03 13:08:30','2011-04-04','2011-11-04 20:33:25','2016-05-09','2024-06-30 12:01:02','2011-12-03 20:42:11'),(71,'9999-12-31','2008-02-14 00:25:55','2019-10-02 19:08:46','9999-12-31 23:59:59','2024-08-03 13:08:30','2024-08-03 13:08:30',null,'2006-02-04 07:40:10'),(72,'2017-10-02','2013-03-19',null,'2006-04-25','2010-05-02 01:06:52','2023-01-15 08:32:59','2024-08-03 13:08:30','2024-07-01'),(73,'2024-07-01','9999-12-31',null,'2019-08-02','9999-12-31','2023-01-15 08:32:59','2012-08-18','2000-11-14'),(74,'2016-03-17 09:53:52','9999-12-31 23:59:59','2024-07-01','2001-02-19 03:10:39',null,'2024-06-30 12:01:02','2000-11-24','2023-01-15 08:32:59'),(75,'2001-04-09 03:47:31','2001-10-19','2014-08-12','2023-01-15 08:32:59','9999-12-31','2016-08-24','2024-08-03 13:08:30','2024-08-03 13:08:30'),(76,'2024-08-03 13:08:30','2023-01-15 08:32:59','2024-07-01','9999-12-31 23:59:59','2002-10-01','2024-08-03 13:08:30','9999-12-31 23:59:59','2014-03-14'),(77,'2017-02-13 09:17:52','2004-02-24 16:58:13','2024-08-03 13:08:30','9999-12-31','9999-12-31','2024-08-03 13:08:30','2017-06-10','2024-06-30 12:01:02'),(78,'2017-02-14','2024-07-01','2010-03-08 03:49:40','2017-12-28','2014-08-12','2017-02-07 00:55:15','2024-07-01','2014-08-12'),(79,'2024-07-01','2014-08-12','2024-07-01','2014-08-12','2024-07-01','2002-05-05','2024-06-30 12:01:02','2001-01-02 11:46:17'),(80,'9999-12-31','2002-01-25 13:15:54',null,'2014-08-12','2024-08-03 13:08:30','2005-06-26','2011-11-22','2013-09-06'),(81,'2014-08-12','2024-08-03 13:08:30','9999-12-31','2024-06-30 12:01:02','2003-02-20','2001-04-14 09:16:01','9999-12-31','2006-07-26 10:05:13'),(82,'2024-07-01','2024-06-30 12:01:02','2011-10-11','2005-04-18','2024-07-01','2007-07-13 15:02:14','2024-07-01','9999-12-31 23:59:59'),(83,'2024-06-30 12:01:02','2003-04-12 08:46:49','2014-08-12','2024-08-03 13:08:30','2024-08-03 13:08:30','2017-12-25','2023-01-15 08:32:59','9999-12-31 23:59:59'),(84,'2014-08-12','2023-01-15 08:32:59','2024-08-03 13:08:30','2014-08-12','2002-08-15','2019-12-20','2007-03-13 13:46:13','9999-12-31'),(85,null,'2024-07-01','2014-08-12','2024-06-30 12:01:02','2014-08-12','2016-12-08','2023-01-15 08:32:59','2014-08-12'),(86,'2019-10-11','2006-07-28 01:15:41','2024-08-03 13:08:30','2010-12-01','2024-06-30 12:01:02','2024-06-30 12:01:02','2015-10-28','2009-10-18'),(87,'2024-06-30 12:01:02','2000-10-18 20:18:44','2002-01-05 09:06:25','2023-01-15 08:32:59','2019-11-17','9999-12-31 23:59:59','2008-05-19','2011-03-03 00:14:30'),(88,'2007-12-19','2008-07-01 09:54:51','2004-12-21','2011-01-07 09:11:49','2023-01-15 08:32:59','2023-01-15 08:32:59','2024-08-03 13:08:30','2023-01-15 08:32:59'),(89,'2009-05-24 10:00:43','2002-03-14','2000-11-21 02:27:36','2024-07-01','2016-08-15 19:58:54','9999-12-31','9999-12-31','2024-08-03 13:08:30'),(90,'2023-01-15 08:32:59','2008-08-11','2014-08-12','9999-12-31 23:59:59','9999-12-31 23:59:59','2009-07-18 01:13:11','9999-12-31 23:59:59','2024-07-01'),(91,'2024-08-03 13:08:30','2009-03-25','2008-05-06 12:41:26','9999-12-31','2014-08-12','2024-08-03 13:08:30',null,'2009-09-03 09:28:22'),(92,'2014-08-12','2024-08-03 13:08:30','2024-08-03 13:08:30','2014-08-12','2014-08-12','2013-05-21','2018-11-04 20:27:55','2024-07-01'),(93,'9999-12-31 23:59:59','2019-03-08 01:50:09','2007-10-14 16:30:07','2018-05-28 10:56:06','2024-06-30 12:01:02','2024-06-30 12:01:02','9999-12-31 23:59:59','2023-01-15 08:32:59'),(94,'2024-07-01','2024-07-01',null,'9999-12-31','2024-06-30 12:01:02','2024-08-03 13:08:30','2024-08-03 13:08:30','2024-06-30 12:01:02'),(95,'2024-08-03 13:08:30','9999-12-31 23:59:59','2015-02-09','2023-01-15 08:32:59','2016-03-18','2016-06-08 20:14:10','2003-07-24','2023-01-15 08:32:59'),(96,'9999-12-31 23:59:59','2023-01-15 08:32:59','2004-03-06 05:53:03','2014-08-12','2024-08-03 13:08:30','2024-08-03 13:08:30',null,'2008-05-18 05:25:09'),(97,'2024-08-03 13:08:30','2001-03-18 13:56:32','2016-02-07','2001-01-19 09:08:26','2014-01-11','2000-09-11','9999-12-31','2014-08-12'),(98,'2004-11-01','9999-12-31 23:59:59','2024-07-01','2024-07-01','9999-12-31','2024-08-03 13:08:30','2017-06-23 07:40:05','2024-06-30 12:01:02'),(99,'2023-01-15 08:32:59','2004-07-26 16:25:58','9999-12-31','2023-01-15 08:32:59','2001-05-24 16:30:22','9999-12-31 23:59:59','2004-11-23','2002-07-02'),(100,'2024-07-01','2023-01-15 08:32:59',null,'2014-08-12','2009-03-24 05:56:52','2014-08-12','2011-07-26','2006-07-02'),(101,'2024-07-01','2014-08-12','9999-12-31','2014-12-17','2024-08-03 13:08:30','2023-01-15 08:32:59','2000-10-12 03:26:23','2024-07-01'),(102,'2008-09-04','9999-12-31','2023-01-15 08:32:59','2024-06-30 12:01:02','2014-08-12','2024-06-30 12:01:02','9999-12-31','9999-12-31'),(103,'9999-12-31','2014-08-12','2017-11-23 15:09:58','2024-06-30 12:01:02','9999-12-31','2014-04-17 21:06:31','2006-03-18','2014-08-12'),(104,'2023-01-15 08:32:59','2024-07-01',null,'9999-12-31','9999-12-31 23:59:59','2024-06-30 12:01:02','9999-12-31','9999-12-31 23:59:59'),(105,'2003-12-25 02:49:53','2001-04-12 16:22:52','2024-07-01','2002-05-07','2010-06-15 16:18:05','2024-07-01','9999-12-31 23:59:59','2014-05-10 05:38:40'),(106,'2024-08-03 13:08:30','9999-12-31','9999-12-31 23:59:59','2014-08-12','2013-10-02','2024-07-01','2014-08-12','2016-01-06 17:31:23'),(107,'9999-12-31','2017-05-02','9999-12-31 23:59:59','2003-06-17 20:10:34',null,'9999-12-31 23:59:59',null,'2003-04-12'),(108,'2014-08-12','9999-12-31 23:59:59','2023-01-15 08:32:59','2006-12-18 15:31:07','2024-08-03 13:08:30','2003-05-05',null,'2013-11-03'),(109,'2005-03-20','9999-12-31','2014-08-12','2024-06-30 12:01:02','9999-12-31 23:59:59','2024-07-01','2017-07-16','2008-10-28'),(110,'2001-10-24 14:58:15','2004-02-18','2023-01-15 08:32:59','9999-12-31 23:59:59','9999-12-31','2024-06-30 12:01:02','2024-08-03 13:08:30','2024-08-03 13:08:30'),(111,'2007-09-02','2024-06-30 12:01:02','2014-08-12','9999-12-31 23:59:59','2000-09-02','2008-12-09 08:53:39','9999-12-31 23:59:59','9999-12-31 23:59:59'),(112,'9999-12-31 23:59:59','2009-11-26 09:51:16','2001-04-24 10:54:54','2014-08-12','2008-10-10 06:12:29','2024-07-01','2014-08-12','9999-12-31'),(113,'2023-01-15 08:32:59','2024-06-30 12:01:02','9999-12-31 23:59:59','2024-08-03 13:08:30','2002-10-04 10:37:22','2024-06-30 12:01:02','2002-06-06 08:46:14','2001-10-13 03:29:15'),(114,'2023-01-15 08:32:59','9999-12-31 23:59:59','2024-07-01','2003-12-19 22:44:02',null,'2024-07-01','2023-01-15 08:32:59','2024-06-30 12:01:02'),(115,'9999-12-31 23:59:59','2001-06-21 21:04:13','2024-07-01','2001-01-14 08:40:23','2023-01-15 08:32:59','2012-12-05','9999-12-31','2014-08-12'),(116,'2015-11-19 19:20:00','2014-08-12','2024-07-01','2014-11-08 02:37:13','9999-12-31','2023-01-15 08:32:59','2024-06-30 12:01:02','2024-07-01'),(117,'2024-08-03 13:08:30','2014-08-12','2023-01-15 08:32:59','2001-06-19','9999-12-31','2014-08-12','9999-12-31 23:59:59','9999-12-31'),(118,'2014-08-12','2005-10-04','2003-07-20 21:18:24','2019-02-06 03:04:18','9999-12-31 23:59:59','2024-08-03 13:08:30','2024-07-01','2024-06-30 12:01:02'),(119,'9999-12-31','2019-11-01','2024-08-03 13:08:30','2018-04-06 01:35:26','2024-08-03 13:08:30','2014-08-12',null,'2017-07-14 16:20:14'),(120,'2005-05-14','2024-06-30 12:01:02','2014-04-14','2024-08-03 13:08:30','2024-06-30 12:01:02','9999-12-31 23:59:59','2005-04-26 00:29:06','2023-01-15 08:32:59'),(121,'9999-12-31','2024-06-30 12:01:02','9999-12-31','2024-06-30 12:01:02','2024-06-30 12:01:02','2024-08-03 13:08:30','2024-08-03 13:08:30','9999-12-31 23:59:59'),(122,'2024-08-03 13:08:30','2024-08-03 13:08:30','2005-04-13','2024-07-01','9999-12-31 23:59:59','2014-08-12','2024-08-03 13:08:30','2006-03-07 21:38:09'),(123,'2014-08-12','2002-02-17 20:38:23','2024-07-01','2024-06-30 12:01:02','2024-06-30 12:01:02','9999-12-31 23:59:59',null,'9999-12-31'),(124,'2014-08-12','2014-07-02','2015-08-27','2001-09-02','2024-08-03 13:08:30','2013-03-28 01:33:13',null,'2023-01-15 08:32:59'),(125,'2024-06-30 12:01:02','2024-08-03 13:08:30',null,'9999-12-31 23:59:59','2018-01-28','2005-03-26',null,'2024-08-03 13:08:30'),(126,'2000-02-07','2002-03-09',null,'9999-12-31',null,'2014-08-12','2023-01-15 08:32:59','2014-08-12'),(127,'9999-12-31','2024-06-30 12:01:02','2014-08-12','2024-08-03 13:08:30','2000-11-07','2024-07-01','2024-07-01','9999-12-31 23:59:59'),(128,'2003-02-12','2024-08-03 13:08:30',null,'2006-09-20','2014-08-12','2017-10-24 03:51:47','2000-12-22 00:14:47','2003-01-05'),(129,'2008-11-08','2024-07-01','9999-12-31','9999-12-31','2005-04-27 02:33:16','2006-05-08','2009-02-13 10:12:05','2012-11-18'),(130,null,'2015-06-20 17:09:39','2015-04-04','2013-12-01','2014-08-12','2024-08-03 13:08:30','2014-08-12','9999-12-31 23:59:59'),(131,'2024-08-03 13:08:30','2024-06-30 12:01:02','2007-01-09','2024-07-01','2024-07-01','9999-12-31 23:59:59','2000-01-19 10:02:41','9999-12-31 23:59:59'),(132,'2005-08-10','2023-01-15 08:32:59','2024-08-03 13:08:30','2023-01-15 08:32:59','2010-02-01 13:15:50','2014-06-26','2010-01-03','2023-01-15 08:32:59'),(133,'2024-07-01','2023-01-15 08:32:59','2005-03-13 10:43:15','9999-12-31','2017-09-01 21:01:51','9999-12-31 23:59:59','9999-12-31','2008-05-17'),(134,null,'2012-09-25 15:05:41','9999-12-31','2024-08-03 13:08:30','2008-08-05 11:19:24','2001-10-01','2023-01-15 08:32:59','2024-06-30 12:01:02'),(135,'2015-03-16','2009-12-24 22:23:28','9999-12-31 23:59:59','2013-10-28','9999-12-31','2009-01-12','2024-06-30 12:01:02','2024-08-03 13:08:30'),(136,'2014-08-12','9999-12-31','2003-11-22 08:31:17','2015-02-22 02:52:54','2009-08-01 22:57:59','2006-07-28','2004-04-10 19:19:44','2014-08-12'),(137,'2004-05-27 16:32:06','2024-06-30 12:01:02','2014-08-12','2014-05-27 16:53:48','2024-07-01','2000-05-19 05:07:16','2024-06-30 12:01:02','2023-01-15 08:32:59'),(138,null,'2001-06-08 17:53:07','2000-10-14 08:52:56','2024-07-01','2024-06-30 12:01:02','2015-01-02 12:25:44','2012-04-01 04:37:16','2014-09-20 00:19:12'),(139,'9999-12-31','2024-06-30 12:01:02','2014-10-03 12:59:23','2024-06-30 12:01:02','2005-07-18 03:51:41','2014-10-14','2014-08-12','2023-01-15 08:32:59'),(140,'2024-06-30 12:01:02','2017-07-26','2008-11-14 16:02:40','2008-07-18 08:11:28','9999-12-31','2014-08-12','9999-12-31 23:59:59','2024-07-01'),(141,'2003-09-06','2018-07-01 11:04:26',null,'2024-06-30 12:01:02','2023-01-15 08:32:59','2018-04-02','2024-08-03 13:08:30','2000-05-24'),(142,'2014-08-12','2014-08-12','2014-08-12','2000-08-27','2018-12-18 22:00:13','2023-01-15 08:32:59','2015-02-28','9999-12-31 23:59:59'),(143,'2024-07-01','2007-08-28 15:54:27','2023-01-15 08:32:59','9999-12-31 23:59:59','2024-06-30 12:01:02','2008-07-15','2009-07-19 19:36:32','2024-06-30 12:01:02'),(144,'9999-12-31','2007-02-07 07:47:29','2011-12-09 04:26:59','2016-09-22',null,'2014-08-12','2012-09-05','2014-11-23 21:37:42'),(145,'2007-03-01 11:10:21','2023-01-15 08:32:59','2019-11-11 20:43:48','9999-12-31','2008-09-19 01:32:21','2024-07-01','9999-12-31 23:59:59','2023-01-15 08:32:59'),(146,'2015-05-10','2019-04-21 21:53:00','2024-07-01','9999-12-31','2024-06-30 12:01:02','2014-08-12','2016-09-17 17:07:25','2023-01-15 08:32:59'),(147,'9999-12-31','9999-12-31','2024-07-01','2014-08-12','2013-02-22','2023-01-15 08:32:59','9999-12-31 23:59:59','9999-12-31 23:59:59'),(148,'2012-02-26 21:57:01','2016-08-05 18:29:11','9999-12-31 23:59:59','2014-08-12','2024-08-03 13:08:30','2023-01-15 08:32:59','2005-11-10','2024-07-01'),(149,'2024-07-01','2019-05-11 22:18:09','2018-11-09 21:21:07','2023-01-15 08:32:59','2023-01-15 08:32:59','2024-07-01','2006-05-05 05:56:09','2002-12-06'),(150,'2024-06-30 12:01:02','2001-05-01 08:21:50','2023-01-15 08:32:59','9999-12-31 23:59:59','2024-08-03 13:08:30','2023-01-15 08:32:59','2016-03-16 07:09:52','2017-01-20'),(151,'2007-10-03 21:17:52','2009-06-07 16:46:07','2016-04-13 19:46:06','9999-12-31','2014-08-12','2016-01-14','9999-12-31 23:59:59','2023-01-15 08:32:59'),(152,'2003-09-15 13:45:20','9999-12-31 23:59:59','2018-04-21','2023-01-15 08:32:59','2023-01-15 08:32:59','2024-07-01','9999-12-31 23:59:59','9999-12-31'),(153,'9999-12-31 23:59:59','2023-01-15 08:32:59','2012-11-09 19:58:14','2011-07-24 10:38:26','9999-12-31','2004-09-15 08:02:21','2024-07-01','2014-01-16'),(154,'2014-10-28','2024-06-30 12:01:02','2024-07-01','9999-12-31','2023-01-15 08:32:59','2024-08-03 13:08:30','2016-11-09 12:52:43','9999-12-31'),(155,'2023-01-15 08:32:59','2002-05-21 13:55:55','2015-09-08','9999-12-31','2024-08-03 13:08:30','2023-01-15 08:32:59','2024-06-30 12:01:02','2024-08-03 13:08:30'),(156,'2014-08-12','2024-06-30 12:01:02','2024-07-01','2015-10-02','2005-03-10','2024-08-03 13:08:30','9999-12-31','2015-06-03 13:36:19'),(157,'2024-06-30 12:01:02','2014-08-12','2016-03-08 15:52:04','2024-06-30 12:01:02','2023-01-15 08:32:59','9999-12-31 23:59:59','2006-03-20','2024-06-30 12:01:02'),(158,'2006-04-13','2015-07-08','2024-07-01','2024-06-30 12:01:02','2024-06-30 12:01:02','2007-08-05',null,'2024-07-01'),(159,'2004-09-04','9999-12-31','2019-07-15 18:09:30','2024-08-03 13:08:30','2005-11-04 12:33:57','2014-08-12','9999-12-31 23:59:59','2000-08-25'),(160,'2024-08-03 13:08:30','2002-01-20 07:13:10','2024-07-01','9999-12-31 23:59:59','2014-08-12','2018-10-14','2024-07-01','2014-05-07 07:44:13'),(161,'2024-08-03 13:08:30','2007-10-13 03:06:34','2008-04-26 19:27:03','2019-10-12 06:39:18','9999-12-31 23:59:59','2023-01-15 08:32:59','2012-07-17','2002-11-27'),(162,'9999-12-31 23:59:59','2024-06-30 12:01:02','2001-02-09 20:20:02','2014-08-12','2009-08-23','9999-12-31 23:59:59',null,'2009-08-22'),(163,'2024-08-03 13:08:30','2023-01-15 08:32:59','2024-08-03 13:08:30','2019-07-05','2009-04-02 13:14:42','2024-08-03 13:08:30','2008-01-23','2002-09-08 20:24:09'),(164,'9999-12-31','9999-12-31','2003-06-17','9999-12-31 23:59:59','2024-08-03 13:08:30','2024-07-01','2006-09-19','2000-01-04'),(165,'2023-01-15 08:32:59','2014-08-12','2024-07-01','2014-08-12','2024-08-03 13:08:30','2023-01-15 08:32:59','2024-08-03 13:08:30','9999-12-31 23:59:59'),(166,'9999-12-31 23:59:59','9999-12-31 23:59:59','2003-05-13','9999-12-31 23:59:59','2014-07-20','2023-01-15 08:32:59','2024-07-01','2015-04-08 08:32:36'),(167,'2023-01-15 08:32:59','2008-01-11 15:34:47','2014-08-12','9999-12-31','2016-10-03','2023-01-15 08:32:59',null,'2008-09-27 12:40:25'),(168,'2006-03-20 11:44:06','2024-08-03 13:08:30','2015-02-07','2024-06-30 12:01:02','2024-07-01','2006-04-11','9999-12-31','9999-12-31'),(169,'2023-01-15 08:32:59','2018-08-10 21:40:47','9999-12-31','2012-03-18','2024-07-01','2023-01-15 08:32:59','2014-08-12','2024-06-30 12:01:02'),(170,'2001-08-06 03:18:10','2023-01-15 08:32:59','9999-12-31','2024-06-30 12:01:02','9999-12-31','2023-01-15 08:32:59','2024-06-30 12:01:02','2005-07-03'),(171,'2012-04-11 05:09:36','9999-12-31 23:59:59','2014-08-12','2017-12-08','2002-03-02 10:25:45','2024-07-01','2024-07-01','2018-04-05 05:36:06'),(172,null,'2024-06-30 12:01:02','2014-08-12','2018-04-20 17:46:04','2003-09-27','2007-10-08','9999-12-31','2023-01-15 08:32:59'),(173,'2006-07-16','2023-01-15 08:32:59',null,'2000-03-12','2014-08-12','9999-12-31','2007-02-24','2019-04-19'),(174,'2019-03-09','9999-12-31','2015-10-28','2009-03-15 19:36:41','2001-08-19','2024-08-03 13:08:30','2011-12-03','9999-12-31'),(175,'2023-01-15 08:32:59','2007-04-21 14:37:31','2024-06-30 12:01:02','9999-12-31 23:59:59','2014-10-08','9999-12-31 23:59:59','9999-12-31','2024-06-30 12:01:02'),(176,'2023-01-15 08:32:59','2014-01-01','9999-12-31','2014-08-12','2023-01-15 08:32:59','2024-08-03 13:08:30','2024-07-01','2007-01-10'),(177,'2005-01-28 11:51:34','2010-12-10','2024-07-01','2010-05-22',null,'2024-07-01',null,'2024-07-01'),(178,'2004-05-21 09:06:38','9999-12-31 23:59:59','2005-09-01','2001-10-23',null,'9999-12-31 23:59:59','9999-12-31','2024-08-03 13:08:30'),(179,'9999-12-31','2024-08-03 13:08:30','2024-07-01','2014-08-12','2024-06-30 12:01:02','2001-12-16 04:53:22','9999-12-31','2014-08-12'),(180,'2024-06-30 12:01:02','2012-01-22','2014-08-12','2016-01-10','9999-12-31','9999-12-31 23:59:59',null,'2018-12-20 01:26:29'),(181,'2024-08-03 13:08:30','2004-01-28 02:38:00',null,'2014-08-12','2024-06-30 12:01:02','2024-08-03 13:08:30','9999-12-31 23:59:59','2024-06-30 12:01:02'),(182,'2000-07-24 00:55:34','2024-07-01','9999-12-31 23:59:59','2024-06-30 12:01:02','2000-02-24 12:10:19','2019-11-08 13:36:58','2018-10-06','2005-10-17'),(183,'2023-01-15 08:32:59','2024-07-01','2024-06-30 12:01:02','2009-08-24 20:06:05',null,'2016-06-17 20:59:37','2014-08-12','2018-10-03'),(184,'9999-12-31','2006-11-03 00:45:22','2024-07-01','2017-05-26 21:53:18','9999-12-31','2024-06-30 12:01:02','9999-12-31','2014-08-12'),(185,'2023-01-15 08:32:59','9999-12-31','2010-03-27 02:37:58','2017-07-28 07:13:39','2014-08-12','2024-06-30 12:01:02','2024-07-01','2014-01-13 15:52:02'),(186,'2024-06-30 12:01:02','2011-01-10 22:55:18','2023-01-15 08:32:59','2000-11-05 19:08:23','2018-06-22','2009-03-16 18:48:41','2015-06-11','2003-12-04 17:06:38'),(187,'2016-08-28','2014-08-12','2014-08-12','2018-06-19 21:25:32','2023-01-15 08:32:59','2014-08-12','2014-07-20 08:25:04','2001-05-01'),(188,'2015-12-20','9999-12-31 23:59:59','2011-11-17 06:34:37','2016-11-16','2023-01-15 08:32:59','2014-08-12','2019-07-13 16:48:29','2009-03-01'),(189,'2019-12-13','2017-04-14 19:20:44','2024-08-03 13:08:30','9999-12-31 23:59:59','9999-12-31','2009-03-28 22:32:17','2014-08-12','2024-08-03 13:08:30'),(190,'2024-06-30 12:01:02','2024-08-03 13:08:30','2014-01-11 20:08:32','2024-07-01','2007-11-15','2002-05-11 09:54:41','2024-07-01','2024-08-03 13:08:30'),(191,'2014-08-12','2014-08-12',null,'2012-03-12','2023-01-15 08:32:59','2024-06-30 12:01:02','2024-07-01','2017-06-22'),(192,'2024-08-03 13:08:30','2008-12-14','2014-08-12','2000-11-14','2024-06-30 12:01:02','2023-01-15 08:32:59','9999-12-31','2023-01-15 08:32:59'),(193,'2024-08-03 13:08:30','2024-07-01','9999-12-31 23:59:59','2024-06-30 12:01:02','2018-01-14 23:40:29','2024-07-01','2014-08-12','2009-04-11'),(194,'2024-07-01','2024-08-03 13:08:30','2014-08-12','2000-08-18','2018-05-19','2023-01-15 08:32:59','2024-07-01','2006-04-24'),(195,'2004-02-11','2024-06-30 12:01:02','2023-01-15 08:32:59','2003-03-16 13:03:44','2023-01-15 08:32:59','2012-10-26','2024-08-03 13:08:30','2009-01-17'),(196,'2012-01-24','2023-01-15 08:32:59','2024-07-01','9999-12-31 23:59:59','2006-04-08','2024-06-30 12:01:02','2009-06-23 19:59:18','2018-03-20 10:05:53'),(197,'2005-07-13','2004-11-26','2010-12-11','2024-07-01','2019-02-12 02:10:04','2024-06-30 12:01:02',null,'2010-05-12'),(198,'2024-06-30 12:01:02','2023-01-15 08:32:59','9999-12-31 23:59:59','2017-04-14','2011-09-22 13:35:35','2003-09-20','2023-01-15 08:32:59','9999-12-31'),(199,'2001-10-26','2010-11-08','2005-08-15','9999-12-31 23:59:59','2024-08-03 13:08:30','2024-08-03 13:08:30','9999-12-31 23:59:59','2014-08-12'); + """ + // return type of Timediff is TimeV2 type. + // so we should not create virtual slot ref. + qt_0 """ + SELECT Week(col_date_undef_signed, 2) AS col_alias7142, '2019-05-08' AS col_alias7145, Min_by(Minutes_diff(col_date_undef_signed_not_null, col_datetime_3__undef_signed), pk) AS col_alias7146 FROM expr_with_runtime_return_type WHERE Timediff(col_datetime_undef_signed, col_datetime_6__undef_signed_not_null) BETWEEN "2024-06-30 12:01:02" AND '2009-11-08' GROUP BY Week(col_date_undef_signed, 2) + """ +} \ No newline at end of file