From a7fb09eb723f559cfeb453f6686c0c07207a95f6 Mon Sep 17 00:00:00 2001 From: JiaKe Date: Fri, 26 Jul 2024 08:54:09 +0800 Subject: [PATCH 01/61] [VL] Enable timestamp in parquet write (#6428) --- .../org/apache/gluten/backendsapi/velox/VeloxBackend.scala | 3 +-- .../execution/VeloxParquetDataTypeValidationSuite.scala | 1 - .../org/apache/gluten/utils/velox/VeloxTestSettings.scala | 4 ++++ .../org/apache/gluten/utils/velox/VeloxTestSettings.scala | 4 ++++ 4 files changed, 9 insertions(+), 3 deletions(-) diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala index 9339473015f26..990991c71660f 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala @@ -224,7 +224,6 @@ object VeloxBackendSettings extends BackendSettingsApi { val unsupportedTypes = fields.flatMap { field => field.dataType match { - case _: TimestampType => Some("TimestampType") case _: StructType => Some("StructType") case _: ArrayType => Some("ArrayType") case _: MapType => Some("MapType") @@ -293,7 +292,7 @@ object VeloxBackendSettings extends BackendSettingsApi { fields.map { field => field.dataType match { - case _: TimestampType | _: StructType | _: ArrayType | _: MapType => return false + case _: StructType | _: ArrayType | _: MapType => return false case _ => } } diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxParquetDataTypeValidationSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxParquetDataTypeValidationSuite.scala index 9793df2ab3340..85b3f32a76842 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxParquetDataTypeValidationSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxParquetDataTypeValidationSuite.scala @@ -484,7 +484,6 @@ class VeloxParquetDataTypeValidationSuite extends VeloxWholeStageTransformerSuit val df = spark.read .format("parquet") .load(data_path) - .drop("timestamp") .drop("array") .drop("struct") .drop("map") diff --git a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index 57346f493945b..52efa4b254424 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -731,12 +731,14 @@ class VeloxTestSettings extends BackendTestSettings { enableSuite[GlutenParquetRebaseDatetimeV1Suite] // Velox doesn't write file metadata into parquet file. .excludeByPrefix("SPARK-33163, SPARK-37705: write the metadata keys") + .excludeByPrefix("SPARK-33160, SPARK-37705: write the metadata key") // jar path and ignore PARQUET_REBASE_MODE_IN_READ, rewrite some .excludeByPrefix("SPARK-31159") .excludeByPrefix("SPARK-35427") enableSuite[GlutenParquetRebaseDatetimeV2Suite] // Velox doesn't write file metadata into parquet file. .excludeByPrefix("SPARK-33163, SPARK-37705: write the metadata keys") + .excludeByPrefix("SPARK-33160, SPARK-37705: write the metadata key") // jar path and ignore PARQUET_REBASE_MODE_IN_READ .excludeByPrefix("SPARK-31159") .excludeByPrefix("SPARK-35427") @@ -767,6 +769,8 @@ class VeloxTestSettings extends BackendTestSettings { enableSuite[GlutenFileMetadataStructRowIndexSuite] enableSuite[GlutenParquetV1AggregatePushDownSuite] enableSuite[GlutenParquetV2AggregatePushDownSuite] + // TODO: Timestamp columns stats will lost if using int64 in parquet writer. + .exclude("aggregate push down - different data types") enableSuite[GlutenOrcV1AggregatePushDownSuite] .exclude("nested column: Count(nested sub-field) not push down") enableSuite[GlutenOrcV2AggregatePushDownSuite] diff --git a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index 9716a7c143742..71348c823f125 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -738,12 +738,14 @@ class VeloxTestSettings extends BackendTestSettings { enableSuite[GlutenParquetRebaseDatetimeV1Suite] // Velox doesn't write file metadata into parquet file. .excludeByPrefix("SPARK-33163, SPARK-37705: write the metadata keys") + .excludeByPrefix("SPARK-33160, SPARK-37705: write the metadata key") // jar path and ignore PARQUET_REBASE_MODE_IN_READ, rewrite some .excludeByPrefix("SPARK-31159") .excludeByPrefix("SPARK-35427") enableSuite[GlutenParquetRebaseDatetimeV2Suite] // Velox doesn't write file metadata into parquet file. .excludeByPrefix("SPARK-33163, SPARK-37705: write the metadata keys") + .excludeByPrefix("SPARK-33160, SPARK-37705: write the metadata key") // jar path and ignore PARQUET_REBASE_MODE_IN_READ .excludeByPrefix("SPARK-31159") .excludeByPrefix("SPARK-35427") @@ -773,6 +775,8 @@ class VeloxTestSettings extends BackendTestSettings { enableSuite[GlutenFileMetadataStructSuite] enableSuite[GlutenParquetV1AggregatePushDownSuite] enableSuite[GlutenParquetV2AggregatePushDownSuite] + // TODO: Timestamp columns stats will lost if using int64 in parquet writer. + .exclude("aggregate push down - different data types") enableSuite[GlutenOrcV1AggregatePushDownSuite] .exclude("nested column: Count(nested sub-field) not push down") enableSuite[GlutenOrcV2AggregatePushDownSuite] From d90a7f488559a70528b860feb8b74fc0b9576ab4 Mon Sep 17 00:00:00 2001 From: Chang chen Date: Fri, 26 Jul 2024 09:29:42 +0800 Subject: [PATCH 02/61] [GLUTEN-6067][CH] [Part 3-2] Basic support for Native Write in Spark 3.5 (#6586) * 1. Reduce duplicated codes: BatchIterator::nextInternal now call CHNativeBlock::toColumnarBatch() to return ColumnarBatch 2. Extract a mew function SerializedPlanParser::buildPipeline, which used in the follow up PRs 3. Refactor File Wrapper, extract create_output_format_file for later use 4. Add GLUTEN_SOURCE_DIR, so that gtest can read java resource 5. Add SubstraitParserUtils.h, so that we can remove parseJson 6. Many litter refactor * Make ExpressionEvaluatorJniWrapper static * Refactor GlutenClickHouseNativeWriteTableSuite and NativeWriteChecker * Add BlockTypeUtils.h * Support Native Write --- .../gluten/metrics/OperatorMetrics.java | 3 + .../gluten/vectorized/BatchIterator.java | 18 +- .../gluten/vectorized/CHColumnVector.java | 2 +- .../gluten/vectorized/CHNativeBlock.java | 10 +- .../CHNativeExpressionEvaluator.java | 43 +- .../ExpressionEvaluatorJniWrapper.java | 22 +- .../backendsapi/clickhouse/CHBackend.scala | 74 +- .../clickhouse/CHIteratorApi.scala | 2 +- .../clickhouse/CHListenerApi.scala | 8 +- .../backendsapi/clickhouse/CHMetricsApi.scala | 12 +- .../clickhouse/CHSparkPlanExecApi.scala | 4 +- .../clickhouse/CHValidatorApi.scala | 3 +- .../execution/NativeFileScanColumnarRDD.scala | 3 +- .../metrics/WriteFilesMetricsUpdater.scala | 30 + .../execution/ClickhouseBackendWrite.scala | 84 + .../src/test/resources/csv-data/supplier.csv | 10000 ++++++++++++++++ ...lutenClickHouseNativeWriteTableSuite.scala | 879 +- .../GlutenClickHouseMetricsUTUtils.scala | 3 +- .../spark/gluten/NativeWriteChecker.scala | 50 +- cpp-ch/local-engine/Common/BlockTypeUtils.cpp | 56 + cpp-ch/local-engine/Common/BlockTypeUtils.h | 89 + cpp-ch/local-engine/Common/CHUtil.cpp | 15 +- cpp-ch/local-engine/Common/CHUtil.h | 6 +- cpp-ch/local-engine/Parser/FunctionParser.cpp | 1 + cpp-ch/local-engine/Parser/JoinRelParser.cpp | 1 - cpp-ch/local-engine/Parser/ProjectRelParser.h | 4 +- .../Parser/SerializedPlanParser.cpp | 304 +- .../Parser/SerializedPlanParser.h | 31 +- .../Parser/SubstraitParserUtils.h | 69 + cpp-ch/local-engine/Parser/TypeParser.cpp | 12 +- cpp-ch/local-engine/Parser/TypeParser.h | 3 +- cpp-ch/local-engine/Parser/WriteRelParser.cpp | 221 + cpp-ch/local-engine/Parser/WriteRelParser.h | 46 + .../scalar_function_parser/arithmetic.cpp | 1 + .../scalar_function_parser/arrayPosition.cpp | 5 +- .../Parser/scalar_function_parser/slice.cpp | 5 +- .../Storages/Output/FileWriterWrappers.cpp | 24 +- .../Storages/Output/FileWriterWrappers.h | 239 +- .../Output/ParquetOutputFormatFile.cpp | 2 +- .../Storages/Output/ParquetOutputFormatFile.h | 2 +- .../Storages/Output/WriteBufferBuilder.cpp | 11 +- .../Storages/Output/WriteBufferBuilder.h | 6 +- .../Storages/SubstraitSource/FormatFile.h | 7 + .../SubstraitSource/JSONFormatFile.cpp | 1 + cpp-ch/local-engine/jni/jni_common.h | 4 +- cpp-ch/local-engine/local_engine_jni.cpp | 124 +- cpp-ch/local-engine/tests/CMakeLists.txt | 5 + .../tests/benchmark_local_engine.cpp | 5 +- .../local-engine/tests/gluten_test_util.cpp | 1 + cpp-ch/local-engine/tests/gluten_test_util.h | 78 +- ...881.cpp => gtest_clickhouse_pr_verify.cpp} | 20 +- .../local-engine/tests/gtest_local_engine.cpp | 3 +- .../tests/gtest_parquet_columnindex.cpp | 7 +- .../local-engine/tests/gtest_parquet_read.cpp | 7 +- cpp-ch/local-engine/tests/gtest_parser.cpp | 183 +- .../tests/gtest_write_pipeline.cpp | 253 + .../json/native_write_one_partition.json | 283 + .../tests/json/native_write_plan.json | 203 + .../json/native_write_plan_1_spark33.json | 116 + .../json/native_write_plan_1_spark35.json | 246 + cpp-ch/local-engine/tests/testConfig.h.in | 2 + 61 files changed, 12925 insertions(+), 1026 deletions(-) create mode 100644 backends-clickhouse/src/main/scala/org/apache/gluten/metrics/WriteFilesMetricsUpdater.scala create mode 100644 backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/ClickhouseBackendWrite.scala create mode 100644 backends-clickhouse/src/test/resources/csv-data/supplier.csv create mode 100644 cpp-ch/local-engine/Common/BlockTypeUtils.cpp create mode 100644 cpp-ch/local-engine/Common/BlockTypeUtils.h create mode 100644 cpp-ch/local-engine/Parser/SubstraitParserUtils.h create mode 100644 cpp-ch/local-engine/Parser/WriteRelParser.cpp create mode 100644 cpp-ch/local-engine/Parser/WriteRelParser.h rename cpp-ch/local-engine/tests/{gtest_clickhouse_54881.cpp => gtest_clickhouse_pr_verify.cpp} (74%) create mode 100644 cpp-ch/local-engine/tests/gtest_write_pipeline.cpp create mode 100644 cpp-ch/local-engine/tests/json/native_write_one_partition.json create mode 100644 cpp-ch/local-engine/tests/json/native_write_plan.json create mode 100644 cpp-ch/local-engine/tests/json/native_write_plan_1_spark33.json create mode 100644 cpp-ch/local-engine/tests/json/native_write_plan_1_spark35.json diff --git a/backends-clickhouse/src/main/java/org/apache/gluten/metrics/OperatorMetrics.java b/backends-clickhouse/src/main/java/org/apache/gluten/metrics/OperatorMetrics.java index 8dcb0ef74b13c..53c822b416ce7 100644 --- a/backends-clickhouse/src/main/java/org/apache/gluten/metrics/OperatorMetrics.java +++ b/backends-clickhouse/src/main/java/org/apache/gluten/metrics/OperatorMetrics.java @@ -27,6 +27,9 @@ public class OperatorMetrics implements IOperatorMetrics { public JoinParams joinParams; public AggregationParams aggParams; + public long physicalWrittenBytes; + public long numWrittenFiles; + /** Create an instance for operator metrics. */ public OperatorMetrics( List metricsList, JoinParams joinParams, AggregationParams aggParams) { diff --git a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/BatchIterator.java b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/BatchIterator.java index 1fbb6053a2afc..1809ed26e6929 100644 --- a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/BatchIterator.java +++ b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/BatchIterator.java @@ -19,11 +19,8 @@ import org.apache.gluten.metrics.IMetrics; import org.apache.gluten.metrics.NativeMetrics; -import org.apache.spark.sql.execution.utils.CHExecUtil; -import org.apache.spark.sql.vectorized.ColumnVector; import org.apache.spark.sql.vectorized.ColumnarBatch; -import java.io.IOException; import java.util.concurrent.atomic.AtomicBoolean; public class BatchIterator extends GeneralOutIterator { @@ -43,8 +40,6 @@ public String getId() { private native boolean nativeHasNext(long nativeHandle); - private native byte[] nativeNext(long nativeHandle); - private native long nativeCHNext(long nativeHandle); private native void nativeClose(long nativeHandle); @@ -54,22 +49,15 @@ public String getId() { private native String nativeFetchMetrics(long nativeHandle); @Override - public boolean hasNextInternal() throws IOException { + public boolean hasNextInternal() { return nativeHasNext(handle); } @Override - public ColumnarBatch nextInternal() throws IOException { + public ColumnarBatch nextInternal() { long block = nativeCHNext(handle); CHNativeBlock nativeBlock = new CHNativeBlock(block); - int cols = nativeBlock.numColumns(); - ColumnVector[] columnVectors = new ColumnVector[cols]; - for (int i = 0; i < cols; i++) { - columnVectors[i] = - new CHColumnVector( - CHExecUtil.inferSparkDataType(nativeBlock.getTypeByPosition(i)), block, i); - } - return new ColumnarBatch(columnVectors, nativeBlock.numRows()); + return nativeBlock.toColumnarBatch(); } @Override diff --git a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHColumnVector.java b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHColumnVector.java index 9cbb242824a9c..88d6471c3cc13 100644 --- a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHColumnVector.java +++ b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHColumnVector.java @@ -25,7 +25,7 @@ public class CHColumnVector extends ColumnVector { private final int columnPosition; - private long blockAddress; + private final long blockAddress; public CHColumnVector(DataType type, long blockAddress, int columnPosition) { super(type); diff --git a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHNativeBlock.java b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHNativeBlock.java index f84327e7d9eb5..e3c51ae28583a 100644 --- a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHNativeBlock.java +++ b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHNativeBlock.java @@ -90,15 +90,13 @@ public static void closeFromColumnarBatch(ColumnarBatch cb) { } public ColumnarBatch toColumnarBatch() { - ColumnVector[] vectors = new ColumnVector[numColumns()]; - for (int i = 0; i < numColumns(); i++) { + int numRows = numRows(); + int cols = numColumns(); + ColumnVector[] vectors = new ColumnVector[cols]; + for (int i = 0; i < cols; i++) { vectors[i] = new CHColumnVector(CHExecUtil.inferSparkDataType(getTypeByPosition(i)), blockAddress, i); } - int numRows = 0; - if (numColumns() != 0) { - numRows = numRows(); - } return new ColumnarBatch(vectors, numRows); } } diff --git a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHNativeExpressionEvaluator.java b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHNativeExpressionEvaluator.java index 45a9f01f7337f..01f38cb3b90be 100644 --- a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHNativeExpressionEvaluator.java +++ b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHNativeExpressionEvaluator.java @@ -28,6 +28,7 @@ import org.apache.spark.SparkConf; import org.apache.spark.sql.internal.SQLConf; +import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -36,15 +37,12 @@ import scala.Tuple2; import scala.collection.JavaConverters; -public class CHNativeExpressionEvaluator { - private final ExpressionEvaluatorJniWrapper jniWrapper; +public class CHNativeExpressionEvaluator extends ExpressionEvaluatorJniWrapper { - public CHNativeExpressionEvaluator() { - jniWrapper = new ExpressionEvaluatorJniWrapper(); - } + private CHNativeExpressionEvaluator() {} // Used to initialize the native computing. - public void initNative(SparkConf conf) { + public static void initNative(SparkConf conf) { Tuple2[] all = conf.getAll(); Map confMap = Arrays.stream(all).collect(Collectors.toMap(Tuple2::_1, Tuple2::_2)); @@ -55,19 +53,19 @@ public void initNative(SparkConf conf) { // Get the customer config from SparkConf for each backend BackendsApiManager.getTransformerApiInstance().postProcessNativeConfig(nativeConfMap, prefix); - jniWrapper.nativeInitNative(buildNativeConf(nativeConfMap)); + nativeInitNative(buildNativeConf(nativeConfMap)); } - public void finalizeNative() { - jniWrapper.nativeFinalizeNative(); + public static void finalizeNative() { + nativeFinalizeNative(); } // Used to validate the Substrait plan in native compute engine. - public boolean doValidate(byte[] subPlan) { - return jniWrapper.nativeDoValidate(subPlan); + public static boolean doValidate(byte[] subPlan) { + throw new UnsupportedOperationException("doValidate is not supported in Clickhouse Backend"); } - private byte[] buildNativeConf(Map confs) { + private static byte[] buildNativeConf(Map confs) { StringMapNode stringMapNode = ExpressionBuilder.makeStringMap(confs); AdvancedExtensionNode extensionNode = ExtensionBuilder.makeAdvancedExtension( @@ -76,27 +74,28 @@ private byte[] buildNativeConf(Map confs) { return PlanBuilder.makePlan(extensionNode).toProtobuf().toByteArray(); } - private Map getNativeBackendConf() { + private static Map getNativeBackendConf() { return GlutenConfig.getNativeBackendConf( BackendsApiManager.getSettings().getBackendConfigPrefix(), SQLConf.get().getAllConfs()); } public static void injectWriteFilesTempPath(String path, String fileName) { - throw new UnsupportedOperationException( - "injectWriteFilesTempPath Not supported in CHNativeExpressionEvaluator"); + ExpressionEvaluatorJniWrapper.injectWriteFilesTempPath( + CHNativeMemoryAllocators.contextInstance().getNativeInstanceId(), + path.getBytes(StandardCharsets.UTF_8), + fileName.getBytes(StandardCharsets.UTF_8)); } // Used by WholeStageTransform to create the native computing pipeline and // return a columnar result iterator. - public BatchIterator createKernelWithBatchIterator( + public static BatchIterator createKernelWithBatchIterator( byte[] wsPlan, byte[][] splitInfo, List iterList, boolean materializeInput) { - long allocId = CHNativeMemoryAllocators.contextInstance().getNativeInstanceId(); long handle = - jniWrapper.nativeCreateKernelWithIterator( - allocId, + nativeCreateKernelWithIterator( + CHNativeMemoryAllocators.contextInstance().getNativeInstanceId(), wsPlan, splitInfo, iterList.toArray(new GeneralInIterator[0]), @@ -106,10 +105,10 @@ public BatchIterator createKernelWithBatchIterator( } // Only for UT. - public BatchIterator createKernelWithBatchIterator( + public static BatchIterator createKernelWithBatchIterator( long allocId, byte[] wsPlan, byte[][] splitInfo, List iterList) { long handle = - jniWrapper.nativeCreateKernelWithIterator( + nativeCreateKernelWithIterator( allocId, wsPlan, splitInfo, @@ -119,7 +118,7 @@ public BatchIterator createKernelWithBatchIterator( return createBatchIterator(handle); } - private BatchIterator createBatchIterator(long nativeHandle) { + private static BatchIterator createBatchIterator(long nativeHandle) { return new BatchIterator(nativeHandle); } } diff --git a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/ExpressionEvaluatorJniWrapper.java b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/ExpressionEvaluatorJniWrapper.java index 0a008a04b5596..a5a474d2a2521 100644 --- a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/ExpressionEvaluatorJniWrapper.java +++ b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/ExpressionEvaluatorJniWrapper.java @@ -24,18 +24,10 @@ public class ExpressionEvaluatorJniWrapper { /** Call initNative to initialize native computing. */ - native void nativeInitNative(byte[] confAsPlan); + static native void nativeInitNative(byte[] confAsPlan); /** Call finalizeNative to finalize native computing. */ - native void nativeFinalizeNative(); - - /** - * Validate the Substrait plan in native compute engine. - * - * @param subPlan the Substrait plan in binary format. - * @return whether the computing of this plan is supported in native. - */ - native boolean nativeDoValidate(byte[] subPlan); + static native void nativeFinalizeNative(); /** * Create a native compute kernel and return a columnar result iterator. @@ -43,7 +35,7 @@ public class ExpressionEvaluatorJniWrapper { * @param allocatorId allocator id * @return iterator instance id */ - public native long nativeCreateKernelWithIterator( + public static native long nativeCreateKernelWithIterator( long allocatorId, byte[] wsPlan, byte[][] splitInfo, @@ -52,9 +44,11 @@ public native long nativeCreateKernelWithIterator( boolean materializeInput); /** - * Closes the projector referenced by nativeHandler. + * Set the temp path for writing files. * - * @param nativeHandler nativeHandler that needs to be closed + * @param allocatorId allocator id for current task attempt(or thread) + * @param path the temp path for writing files */ - native void nativeClose(long nativeHandler); + public static native void injectWriteFilesTempPath( + long allocatorId, byte[] path, byte[] filename); } diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala index 320483beb3b62..06fe8c34ca4a5 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala @@ -18,6 +18,7 @@ package org.apache.gluten.backendsapi.clickhouse import org.apache.gluten.{CH_BRANCH, CH_COMMIT, GlutenConfig} import org.apache.gluten.backendsapi._ +import org.apache.gluten.execution.WriteFilesExecTransformer import org.apache.gluten.expression.WindowFunctionsBuilder import org.apache.gluten.extension.ValidationResult import org.apache.gluten.substrait.rel.LocalFilesNode.ReadFileFormat @@ -25,13 +26,17 @@ import org.apache.gluten.substrait.rel.LocalFilesNode.ReadFileFormat._ import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning} import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.aggregate.HashAggregateExec +import org.apache.spark.sql.execution.datasources.FileFormat +import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{ArrayType, MapType, StructField, StructType} +import org.apache.spark.sql.types.{ArrayType, MapType, Metadata, StructField, StructType} import java.util.Locale @@ -187,6 +192,73 @@ object CHBackendSettings extends BackendSettingsApi with Logging { } } + override def supportWriteFilesExec( + format: FileFormat, + fields: Array[StructField], + bucketSpec: Option[BucketSpec], + options: Map[String, String]): ValidationResult = { + + def validateCompressionCodec(): Option[String] = { + // FIXME: verify Support compression codec + val compressionCodec = WriteFilesExecTransformer.getCompressionCodec(options) + None + } + + def validateFileFormat(): Option[String] = { + format match { + case _: ParquetFileFormat => None + case _: OrcFileFormat => None + case f: FileFormat => Some(s"Not support FileFormat: ${f.getClass.getSimpleName}") + } + } + + // Validate if all types are supported. + def validateDateTypes(): Option[String] = { + None + } + + def validateFieldMetadata(): Option[String] = { + // copy CharVarcharUtils.CHAR_VARCHAR_TYPE_STRING_METADATA_KEY + val CHAR_VARCHAR_TYPE_STRING_METADATA_KEY = "__CHAR_VARCHAR_TYPE_STRING" + fields + .find(_.metadata != Metadata.empty) + .filterNot(_.metadata.contains(CHAR_VARCHAR_TYPE_STRING_METADATA_KEY)) + .map { + filed => + s"StructField contain the metadata information: $filed, metadata: ${filed.metadata}" + } + } + def validateWriteFilesOptions(): Option[String] = { + val maxRecordsPerFile = options + .get("maxRecordsPerFile") + .map(_.toLong) + .getOrElse(SQLConf.get.maxRecordsPerFile) + if (maxRecordsPerFile > 0) { + Some("Unsupported native write: maxRecordsPerFile not supported.") + } else { + None + } + } + + def validateBucketSpec(): Option[String] = { + if (bucketSpec.nonEmpty) { + Some("Unsupported native write: bucket write is not supported.") + } else { + None + } + } + + validateCompressionCodec() + .orElse(validateFileFormat()) + .orElse(validateFieldMetadata()) + .orElse(validateDateTypes()) + .orElse(validateWriteFilesOptions()) + .orElse(validateBucketSpec()) match { + case Some(reason) => ValidationResult.failed(reason) + case _ => ValidationResult.succeeded + } + } + override def supportShuffleWithProject( outputPartitioning: Partitioning, child: SparkPlan): Boolean = { diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala index cd829d04f2cfe..d54eb59036d86 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala @@ -76,7 +76,7 @@ class CHIteratorApi extends IteratorApi with Logging with LogLevelUtil { } .map(it => new ColumnarNativeIterator(it.asJava).asInstanceOf[GeneralInIterator]) .asJava - new CHNativeExpressionEvaluator().createKernelWithBatchIterator( + CHNativeExpressionEvaluator.createKernelWithBatchIterator( wsPlan, splitInfoByteArray, listIterator, diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHListenerApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHListenerApi.scala index 0110d085b98c8..69797feb65fb7 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHListenerApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHListenerApi.scala @@ -83,7 +83,7 @@ class CHListenerApi extends ListenerApi with Logging { val externalSortKey = s"${CHBackendSettings.getBackendConfigPrefix}.runtime_settings" + s".max_bytes_before_external_sort" if (conf.getLong(externalSortKey, -1) < 0) { - if (conf.getBoolean("spark.memory.offHeap.enabled", false)) { + if (conf.getBoolean("spark.memory.offHeap.enabled", defaultValue = false)) { val memSize = JavaUtils.byteStringAsBytes(conf.get("spark.memory.offHeap.size")) if (memSize > 0L) { val cores = conf.getInt("spark.executor.cores", 1).toLong @@ -97,8 +97,7 @@ class CHListenerApi extends ListenerApi with Logging { // Load supported hive/python/scala udfs UDFMappings.loadFromSparkConf(conf) - val initKernel = new CHNativeExpressionEvaluator() - initKernel.initNative(conf) + CHNativeExpressionEvaluator.initNative(conf) // inject backend-specific implementations to override spark classes // FIXME: The following set instances twice in local mode? @@ -110,7 +109,6 @@ class CHListenerApi extends ListenerApi with Logging { private def shutdown(): Unit = { CHBroadcastBuildSideCache.cleanAll() - val kernel = new CHNativeExpressionEvaluator() - kernel.finalizeNative() + CHNativeExpressionEvaluator.finalizeNative() } } diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHMetricsApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHMetricsApi.scala index 5465e9b60b672..85b298fa48354 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHMetricsApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHMetricsApi.scala @@ -383,13 +383,13 @@ class CHMetricsApi extends MetricsApi with Logging with LogLevelUtil { s"SampleTransformer metrics update is not supported in CH backend") } - def genWriteFilesTransformerMetrics(sparkContext: SparkContext): Map[String, SQLMetric] = { - throw new UnsupportedOperationException( - s"WriteFilesTransformer metrics update is not supported in CH backend") - } + def genWriteFilesTransformerMetrics(sparkContext: SparkContext): Map[String, SQLMetric] = + Map( + "physicalWrittenBytes" -> SQLMetrics.createMetric(sparkContext, "number of written bytes"), + "numWrittenFiles" -> SQLMetrics.createMetric(sparkContext, "number of written files") + ) def genWriteFilesTransformerMetricsUpdater(metrics: Map[String, SQLMetric]): MetricsUpdater = { - throw new UnsupportedOperationException( - s"WriteFilesTransformer metrics update is not supported in CH backend") + new WriteFilesMetricsUpdater(metrics) } } diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala index 32f372956f0ae..b8a76b4210c3f 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala @@ -674,8 +674,8 @@ class CHSparkPlanExecApi extends SparkPlanExecApi { CHRegExpReplaceTransformer(substraitExprName, children, expr) } - def createBackendWrite(description: WriteJobDescription): BackendWrite = - throw new UnsupportedOperationException("createBackendWrite is not supported in ch backend.") + def createBackendWrite(description: WriteJobDescription): BackendWrite = ClickhouseBackendWrite( + description) override def createColumnarArrowEvalPythonExec( udfs: Seq[PythonUDF], diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHValidatorApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHValidatorApi.scala index 9e4cbf760974b..09659e42ca838 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHValidatorApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHValidatorApi.scala @@ -38,8 +38,7 @@ import org.apache.spark.sql.execution.datasources.v2.V2CommandExec class CHValidatorApi extends ValidatorApi with AdaptiveSparkPlanHelper with Logging { override def doNativeValidateWithFailureReason(plan: PlanNode): NativePlanValidationInfo = { - val validator = new CHNativeExpressionEvaluator() - if (validator.doValidate(plan.toProtobuf.toByteArray)) { + if (CHNativeExpressionEvaluator.doValidate(plan.toProtobuf.toByteArray)) { new NativePlanValidationInfo(1, "") } else { new NativePlanValidationInfo(0, "CH native check failed.") diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/NativeFileScanColumnarRDD.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/NativeFileScanColumnarRDD.scala index af512934bc96b..2f11613a275c4 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/NativeFileScanColumnarRDD.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/NativeFileScanColumnarRDD.scala @@ -49,9 +49,8 @@ class NativeFileScanColumnarRDD( val resIter = GlutenTimeMetric.millis(scanTime) { _ => - val transKernel = new CHNativeExpressionEvaluator() val inBatchIters = new util.ArrayList[GeneralInIterator]() - transKernel.createKernelWithBatchIterator( + CHNativeExpressionEvaluator.createKernelWithBatchIterator( inputPartition.plan, splitInfoByteArray, inBatchIters, diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/WriteFilesMetricsUpdater.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/WriteFilesMetricsUpdater.scala new file mode 100644 index 0000000000000..5a04b404334ff --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/WriteFilesMetricsUpdater.scala @@ -0,0 +1,30 @@ +/* + * 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.gluten.metrics + +import org.apache.spark.sql.execution.metric.SQLMetric + +class WriteFilesMetricsUpdater(val metrics: Map[String, SQLMetric]) extends MetricsUpdater { + + override def updateNativeMetrics(opMetrics: IOperatorMetrics): Unit = { + if (opMetrics != null) { + val operatorMetrics = opMetrics.asInstanceOf[OperatorMetrics] + metrics("physicalWrittenBytes") += operatorMetrics.physicalWrittenBytes + metrics("numWrittenFiles") += operatorMetrics.numWrittenFiles + } + } +} diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/ClickhouseBackendWrite.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/ClickhouseBackendWrite.scala new file mode 100644 index 0000000000000..225d9688c7df1 --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/ClickhouseBackendWrite.scala @@ -0,0 +1,84 @@ +/* + * 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.spark.sql.execution + +import org.apache.spark.internal.Logging +import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.vectorized.ColumnarBatch + +import scala.collection.mutable + +case class ClickhouseBackendWrite(description: WriteJobDescription) + extends BackendWrite + with Logging { + + override def collectNativeWriteFilesMetrics(cb: ColumnarBatch): Option[WriteTaskResult] = { + val numFiles = cb.numRows() + // Write an empty iterator + if (numFiles == 0) { + None + } else { + val file_col = cb.column(0) + val partition_col = cb.column(1) + val count_col = cb.column(2) + + val outputPath = description.path + var updatedPartitions = Set.empty[String] + val addedAbsPathFiles: mutable.Map[String, String] = mutable.Map[String, String]() + + val write_stats = Range(0, cb.numRows()).map { + i => + val targetFileName = file_col.getUTF8String(i).toString + val partition = partition_col.getUTF8String(i).toString + if (partition != "__NO_PARTITION_ID__") { + updatedPartitions += partition + val tmpOutputPath = outputPath + "/" + partition + "/" + targetFileName + val customOutputPath = + description.customPartitionLocations.get( + PartitioningUtils.parsePathFragment(partition)) + if (customOutputPath.isDefined) { + addedAbsPathFiles(tmpOutputPath) = customOutputPath.get + "/" + targetFileName + } + } + count_col.getLong(i) + } + + val partitionsInternalRows = updatedPartitions.map { + part => + val parts = new Array[Any](1) + parts(0) = part + new GenericInternalRow(parts) + }.toSeq + + val numWrittenRows = write_stats.sum + val stats = BasicWriteTaskStats( + partitions = partitionsInternalRows, + numFiles = numFiles, + numBytes = 101, + numRows = numWrittenRows) + val summary = + ExecutedWriteSummary(updatedPartitions = updatedPartitions, stats = Seq(stats)) + + Some( + WriteTaskResult( + new TaskCommitMessage(addedAbsPathFiles.toMap -> updatedPartitions), + summary)) + } + } +} diff --git a/backends-clickhouse/src/test/resources/csv-data/supplier.csv b/backends-clickhouse/src/test/resources/csv-data/supplier.csv new file mode 100644 index 0000000000000..4a9de17182921 --- /dev/null +++ b/backends-clickhouse/src/test/resources/csv-data/supplier.csv @@ -0,0 +1,10000 @@ +1|Supplier#000000001| N kD4on9OM Ipw3,gf0JBoQDd7tgrzrddZ|17|27-918-335-1736|5755.94|each slyly above the careful +2|Supplier#000000002|89eJ5ksX3ImxJQBvxObC,|5|15-679-861-2259|4032.68| slyly bold instructions. idle dependen +3|Supplier#000000003|q1,G3Pj6OjIuUYfUoH18BFTKP5aU9bEV3|1|11-383-516-1199|4192.40|blithely silent requests after the express dependencies are sl +4|Supplier#000000004|Bk7ah4CK8SYQTepEmvMkkgMwg|15|25-843-787-7479|4641.08|riously even requests above the exp +5|Supplier#000000005|Gcdm2rJRzl5qlTVzc|11|21-151-690-3663|-283.84|. slyly regular pinto bea +6|Supplier#000000006|tQxuVm7s7CnK|14|24-696-997-4969|1365.79|final accounts. regular dolphins use against the furiously ironic decoys. +7|Supplier#000000007|s,4TicNGB4uO6PaSqNBUq|23|33-990-965-2201|6820.35|s unwind silently furiously regular courts. final requests are deposits. requests wake quietly blit +8|Supplier#000000008|9Sq4bBH2FQEmaFOocY45sRTxo6yuoG|17|27-498-742-3860|7627.85|al pinto beans. asymptotes haggl +9|Supplier#000000009|1KhUgZegwM3ua7dsYmekYBsK|10|20-403-398-8662|5302.37|s. unusual, even requests along the furiously regular pac +10|Supplier#000000010|Saygah3gYWMp72i PY|24|34-852-489-8585|3891.91|ing waters. regular requests ar +11|Supplier#000000011|JfwTs,LZrV, M,9C|18|28-613-996-1505|3393.08|y ironic packages. slyly ironic accounts affix furiously; ironically unusual excuses across the flu +12|Supplier#000000012|aLIW q0HYd|8|18-179-925-7181|1432.69|al packages nag alongside of the bold instructions. express, daring accounts +13|Supplier#000000013|HK71HQyWoqRWOX8GI FpgAifW,2PoH|3|13-727-620-7813|9107.22|requests engage regularly instructions. furiously special requests ar +14|Supplier#000000014|EXsnO5pTNj4iZRm|15|25-656-247-5058|9189.82|l accounts boost. fluffily bold warhorses wake +15|Supplier#000000015|olXVbNBfVzRqgokr1T,Ie|8|18-453-357-6394|308.56| across the furiously regular platelets wake even deposits. quickly express she +16|Supplier#000000016|YjP5C55zHDXL7LalK27zfQnwejdpin4AMpvh|22|32-822-502-4215|2972.26|ously express ideas haggle quickly dugouts? fu +17|Supplier#000000017|c2d,ESHRSkK3WYnxpgw6aOqN0q|19|29-601-884-9219|1687.81|eep against the furiously bold ideas. fluffily bold packa +18|Supplier#000000018|PGGVE5PWAMwKDZw |16|26-729-551-1115|7040.82|accounts snooze slyly furiously bold +19|Supplier#000000019|edZT3es,nBFD8lBXTGeTl|24|34-278-310-2731|6150.38|refully final foxes across the dogged theodolites sleep slyly abou +20|Supplier#000000020|iybAE,RmTymrZVYaFZva2SH,j|3|13-715-945-6730|530.82|n, ironic ideas would nag blithely about the slyly regular accounts. silent, expr +21|Supplier#000000021|81CavellcrJ0PQ3CPBID0Z0JwyJm0ka5igEs|2|12-253-590-5816|9365.80|d. instructions integrate sometimes slyly pending instructions. accounts nag among the +22|Supplier#000000022|okiiQFk 8lm6EVX6Q0,bEcO|4|14-144-830-2814|-966.20| ironically among the deposits. closely expre +23|Supplier#000000023|ssetugTcXc096qlD7 2TL5crEEeS3zk|9|19-559-422-5776|5926.41|ges could have to are ironic deposits. regular, even request +24|Supplier#000000024|C4nPvLrVmKPPabFCj|0|10-620-939-2254|9170.71|usly pending deposits. slyly final accounts run +25|Supplier#000000025|RCQKONXMFnrodzz6w7fObFVV6CUm2q|22|32-431-945-3541|9198.31|ely regular deposits. carefully regular sauternes engage furiously above the regular accounts. idly +26|Supplier#000000026|iV,MHzAx6Z939uzFNkq09M0a1 MBfH7|21|31-758-894-4436|21.18| ideas poach carefully after the blithely bold asymptotes. furiously pending theodoli +27|Supplier#000000027|lC4CjKwNHUr6L4xIpzOBK4NlHkFTg|18|28-708-999-2028|1887.62|s according to the quickly regular hockey playe +28|Supplier#000000028|GBhvoRh,7YIN V|0|10-538-384-8460|-891.99|ld requests across the pinto beans are carefully against the quickly final courts. accounts sleep +29|Supplier#000000029|658tEqXLPvRd6xpFdqC2|1|11-555-705-5922|-811.62|y express ideas play furiously. even accounts sleep fluffily across the accounts. careful +30|Supplier#000000030|84NmC1rmQfO0fj3zkobLT|16|26-940-594-4852|8080.14|ias. carefully silent accounts cajole blithely. pending, special accounts cajole quickly above the f +31|Supplier#000000031|fRJimA7zchyApqRLHcQeocVpP|16|26-515-530-4159|5916.91|into beans wake after the special packages. slyly fluffy requests cajole furio +32|Supplier#000000032|yvoD3TtZSx1skQNCK8agk5bZlZLug|23|33-484-637-7873|3556.47|usly even depths. quickly ironic theodolites s +33|Supplier#000000033|gfeKpYw3400L0SDywXA6Ya1Qmq1w6YB9f3R|7|17-138-897-9374|8564.12|n sauternes along the regular asymptotes are regularly along the +34|Supplier#000000034|mYRe3KvA2O4lL4HhxDKkkrPUDPMKRCSp,Xpa|10|20-519-982-2343|237.31|eposits. slyly final deposits toward the slyly regular dependencies sleep among the excu +35|Supplier#000000035|QymmGXxjVVQ5OuABCXVVsu,4eF gU0Qc6|21|31-720-790-5245|4381.41| ironic deposits! final, bold platelets haggle quickly quickly pendin +36|Supplier#000000036|mzSpBBJvbjdx3UKTW3bLFewRD78D91lAC879|13|23-273-493-3679|2371.51|ular theodolites must haggle regular, bold accounts. slyly final pinto beans bo +37|Supplier#000000037|cqjyB5h1nV|0|10-470-144-1330|3017.47|iously final instructions. quickly special accounts hang fluffily above the accounts. deposits +38|Supplier#000000038|xEcx45vD0FXHT7c9mvWFY|4|14-361-296-6426|2512.41|ins. fluffily special accounts haggle slyly af +39|Supplier#000000039|ZM, nSYpEPWr1yAFHaC91qjFcijjeU5eH|8|18-851-856-5633|6115.65|le slyly requests. special packages shall are blithely. slyly unusual packages sleep +40|Supplier#000000040|zyIeWzbbpkTV37vm1nmSGBxSgd2Kp|22|32-231-247-6991|-290.06| final patterns. accounts haggle idly pas +41|Supplier#000000041|G 1FKHR435 wMKFmyt|18|28-739-447-2525|6942.67|odolites boost across the furiously regular fo +42|Supplier#000000042|1Y5lwEgpe3j2vbUBYj3SwLhK62JlwEMtDC|22|32-698-298-6317|6565.11| fluffily even requests cajole blithely fu +43|Supplier#000000043|Z5mLuAoTUEeKY5v22VnnA4D87Ao6jF2LvMYnlX8h|12|22-421-568-4862|7773.41|unts. unusual, final asymptotes +44|Supplier#000000044|kERxlLDnlIZJdN66zAPHklyL|7|17-713-930-5667|9759.38|x. carefully quiet account +45|Supplier#000000045|LcKnsa8XGtIO0WYSB7hkOrH rnzRg1|9|19-189-635-8862|2944.23|iously according to the ironic, silent accounts. +46|Supplier#000000046|e0URUXfDOYMdKe16Z5h5StMRbzGmTs,D2cjap|24|34-748-308-3215|3580.35|gular, regular ideas across th +47|Supplier#000000047|3XM1x,Pcxqw,HK4XNlgbnZMbLhBHLA|14|24-810-354-4471|2958.09|sly ironic deposits sleep carefully along t +48|Supplier#000000048|jg0U FNPMQDuyuKvTnLXXaLf3Wl6OtONA6mQlWJ|14|24-722-551-9498|5630.62|xpress instructions affix. fluffily even requests boos +49|Supplier#000000049|Nvq 6macF4GtJvz|24|34-211-567-6800|9915.24|the finally bold dependencies. dependencies after the fluffily final foxes boost fluffi +50|Supplier#000000050|rGobqSMMYz0ErrPhCGS|9|19-561-560-7437|4515.87|warhorses. ironic, regular accounts detect slyly after the quickly f +51|Supplier#000000051|rDkBXb01POIKjOwrij62uM8O4|9|19-475-537-1368|7241.40|ges nag at the blithely busy instructions. fluffy packages wake quickly. even, ironic ideas boost b +52|Supplier#000000052|WCk XCHYzBA1dvJDSol4ZJQQcQN,|19|29-974-934-4713|287.16|dolites are slyly against the furiously regular packages. ironic, final deposits cajole quickly +53|Supplier#000000053|i9v3 EsYCfLKFU6PIt8iihBOHBB37yR7b3GD7Rt|7|17-886-101-6083|6177.35|onic, special deposits wake furio +54|Supplier#000000054|J1s,Wxb5pg|12|22-966-435-7200|2733.69|blithely pending dolphins. quickly regular theodolites haggle slyly +55|Supplier#000000055|OqdYSiOQeG4eGi636Tj|24|34-876-912-6007|7162.15|kages. blithely even foxes cajole special, final accounts. blithely even dependencies r +56|Supplier#000000056|fUVtlUVal GiHBOuYoUQ XQ9NfNLQR3Gl|16|26-471-195-5486|-632.16| sleep special deposits. unusual requests wake blithely slyly regular ideas. +57|Supplier#000000057|bEWqUVRR f0mb2o18Y|17|27-681-514-6892|-831.07|detect according to the furiously br +58|Supplier#000000058|01dEADIZoCULZXg|16|26-659-969-5586|92.44|refully final foxes are. even, express courts according to the b +59|Supplier#000000059|N8lKbYjMnVlEHmTPRmBgtLiX8rrJx|17|27-249-395-9123|586.16|ffily along the even decoys. final instructions abov +60|Supplier#000000060|cb08ntDTARo47WmnBcYXu|8|18-550-360-2464|4515.80|thely express ideas use blithely +61|Supplier#000000061|Oz0M1qBR9I|8|18-396-489-9719|6096.58|s the slyly regular ideas shall +62|Supplier#000000062|bSmlFYUKBeRsqJxwC9 zS6xpFdEf5jNTb|19|29-603-653-2494|9202.57|ts. furiously ironic pinto beans are permanently after the bold ideas. regular, express f +63|Supplier#000000063|NlV0OQyIoPvPkw5AYuWGomX,hgqm1|5|15-781-401-3047|5742.03|ar deposits. blithely bold accounts against the slyly final pinto beans sleep about the exp +64|Supplier#000000064|w80JjnIP lGoLdUjRutbv81gGlqqpW4PQBeOtSYU|24|34-278-790-7004|1309.70|uickly regular requests use. carefully i +65|Supplier#000000065|BsAnHUmSFArppKrM|22|32-444-835-2434|-963.79|l ideas wake carefully around the regular packages. furiously ruthless pinto bea +66|Supplier#000000066|qYdruFJQJYYiKvnNVmYfCVydVB8bcW,AW,U6SOV3|23|33-300-836-9529|2455.98|ar requests. express orbits de +67|Supplier#000000067|7YrEKJncHFk5D W7ZaqfAXV|4|14-563-538-1657|3576.55|ray slyly final foxes. furio +68|Supplier#000000068|Ue6N50wH2CwE4PPgTGLmat,ibGYYlDoOb3xQwtgb|21|31-267-327-4328|5119.38|inal requests. ruthlessly ironic packages cajole +69|Supplier#000000069|T2Dl9,f97e333eRuMi2z |4|14-491-707-8310|8466.50|! carefully ironic instructions nag quickly pending requests. fluffily even deposits sleep a +70|Supplier#000000070|INWNH2w,OOWgNDq0BRCcBwOMQc6PdFDc4|6|16-821-608-1166|9508.37|ests sleep quickly express ideas. ironic ideas haggle about the final T +71|Supplier#000000071|YFo8an7P6wi Q|1|11-743-919-7272|8179.68| final accounts. bold, final escapades must have to cajole about the special platelets. fu +72|Supplier#000000072|mKpAJojtawk2alqV4 ZEbJ3PH3wfYqy AM8rGq1|18|28-113-898-6643|7014.50| theodolites sublate furiously about the regularly e +73|Supplier#000000073|HBZA1NHvrswQCxTTjg 5XrfSOGgMRKNCe2ovE|16|26-758-310-7496|3793.13|. never pending asymptotes a +74|Supplier#000000074|uM3yV5NOc6b5wNdpxF69CW 8QvDxqvKubRJtA|20|30-166-486-1559|4170.51|carefully along the quickly regular sentiments. ironic accounts sleep. regular deposits are blith +75|Supplier#000000075|7f3gN4rP1livII|18|28-716-704-8686|-224.84|eans. even, silent packages c +76|Supplier#000000076|JBhSBa3cLYvNgHUYtUHmtECCD|14|24-228-763-7840|2971.10|, even instructions. furiously unusual deposits wake slyly about the ev +77|Supplier#000000077|wVtcr0uH3CyrSiWMLsqnB09Syo,UuZxPMeBghlY|7|17-281-345-4863|4186.95|the slyly final asymptotes. blithely pending theodoli +78|Supplier#000000078|9y3OZ2CV hGrsrQxzB7V3zTtygHVHlG3SD6yrz|5|15-670-998-6860|1044.10|, regular packages wake quickly bold requests. carefully unusual requests about the unusual request +79|Supplier#000000079|p0u3tztSXUD2J8vFfLNFNKsrRRv7qyUtTBTA|14|24-402-227-9600|1191.94|nto beans integrate slyly across the fluffily pending multipliers. carefully ste +80|Supplier#000000080|cJ2MHSEJ13rIL2Wj3D5i6hRo30,ZiNUXhqn|21|31-646-289-1906|-40.45|ackages. blithely bold requests wake quickly. carefully regular foxes are slyly instructions. caref +81|Supplier#000000081|SLlacbhgpKmVa,gF3saYv12e0|12|22-535-310-6971|166.32|oost carefully quickly regular packages. carefully final excuses sleep blithely slyly pendi +82|Supplier#000000082|WyTKA7ZpF15t1aCNlT3|18|28-177-572-9691|-724.31|xpress multipliers wake furiously even foxes. furiously iro +83|Supplier#000000083|WRJUkzCn050seVz57oAfrbCuw|14|24-529-559-2461|1467.77|ly pending courts would cajole enticingly even deposits. slyly express +84|Supplier#000000084|DcYjWMiZGQqEKOJi4wAmIV08ikx|24|34-869-118-7803|4780.93|even depths. regular foxes use slyly. theod +85|Supplier#000000085|Ckls9RtlzKSF|7|17-167-806-8199|7174.74|egular packages. bold pinto beans wake fur +86|Supplier#000000086|J1fgg5QaqnN|19|29-903-665-7065|1883.37|cajole furiously special, final requests: furiously spec +87|Supplier#000000087|WCw7URDj8zoZ7tqC3cpm7|24|34-860-229-1674|4746.66|all are quickly after the ironic platelets. pending dolphins are. final the +88|Supplier#000000088|yOshY8wwzMgS|11|21-191-938-9469|3086.13|furiously special excuses aff +89|Supplier#000000089|fhtzZcSorhud1|9|19-259-876-1014|1638.02|en instructions across the slyly ironic requests engage +90|Supplier#000000090|bPE6Uhz1f2m3gwSGMrnRt,g,3gq37r5kxgphqss1|6|16-380-123-9217|6201.77|hely fluffily regular theodoli +91|Supplier#000000091|YV45D7TkfdQanOOZ7q9QxkyGUapU1oOWU6q3|3|13-604-986-9056|6255.87|nstructions use carefully according to the special packages: quickly silent th +92|Supplier#000000092|n48Wy4QI3lml8T217rk|2|12-701-432-8346|2470.84| even theodolites wake against the blithely fluffy packages +93|Supplier#000000093|wd1djjKXT,4zBm|16|26-528-528-1157|368.76|yly final accounts could are carefully. fluffily ironic instruct +94|Supplier#000000094|lK,pLPjAMVGJOXN80zPZuNQjpChliE|4|14-728-888-8882|2766.80|usly. furiously slow theodolites could haggle carefully fina +95|Supplier#000000095|p2 2hBfH5TD|19|29-142-539-1403|2588.83|sual instructions cajole slyly with the final ac +96|Supplier#000000096|Lbxp3WIipye o2wZme1i9iJx,xTt1Mp|12|22-888-654-7193|3437.24|efully unusual excuses wake fluffily regular theodo +97|Supplier#000000097|MrCQha2G6ndX1fp6CA|4|14-618-678-1789|6642.22|s, regular accounts. furiously bold ideas u +98|Supplier#000000098|ogHn8dpXB5Q|21|31-914-775-1978|5873.07|esias use slyly under the ironic foxes. re +99|Supplier#000000099|4SxkTHG28nZrtT0,MnVF9H|18|28-272-909-1617|3671.34|nic dependencies dazzle flu +100|Supplier#000000100|rIlN li8zvW22l2slbcx ECP4fL|21|31-351-324-5062|3191.70|es. regular instructions sleep carefully. slyly ironic packages across the foxes boost +101|Supplier#000000101|8KUTzVw32Pw3PD7 h,YJ0ysOoBtz9JfZI|11|21-970-795-5691|4901.77|ar foxes. carefully final ideas are. blithely regular deposits about the carefully regular package +102|Supplier#000000102|pDmsZ3V8W7NR1wW|8|18-884-693-5785|2980.07|ly even notornis nod furiously evenly regular platelets. thinly pending pat +103|Supplier#000000103|hBpQ4GYblzpHKTgATLPAS6ph3|2|12-442-523-5043|1050.66| accounts detect slyly bo +104|Supplier#000000104|Dcl4yGrzqv3OPeRO49bKh78XmQEDR7PBXIs0m|6|16-434-972-6922|1381.97|gular ideas. bravely bold deposits haggle through the carefully final deposits. slyly unusual idea +105|Supplier#000000105|cB YSy5Bla|4|14-951-800-2742|3754.75|sual requests haggle slyly. theodolites +106|Supplier#000000106|50EV3vyfAsWJAjTbT4qwU|21|31-810-990-4600|8091.65|eas affix carefully fluffily silent packages. regular deposits bo +107|Supplier#000000107|fqniA2vC1VZU5DZG2TBiN|16|26-958-723-2164|1378.93|thely special foxes nag carefully. requests cajole along the quickly ironic pinto b +108|Supplier#000000108|eL47lfhfMP7zRw|24|34-341-415-7878|7683.71|, regular packages! bold re +109|Supplier#000000109|4 63llZBLxtWRa A|21|31-405-311-8656|9202.82|al courts. unusual escapades cajol +110|Supplier#000000110|ehv9ObpyN0|15|25-824-874-9077|9804.10|eposits nag thinly furiously even accounts; permanent ideas nag fluf +111|Supplier#000000111|uv56H9j8cNa4qnflVSYbLNN|4|14-256-573-2660|6578.65|ing theodolites are about the slowly pending +112|Supplier#000000112|vdWe5lfgvisRCxdd85DTOZHqAzcuq7f7KKThA|12|22-617-876-1402|4332.95|es eat fluffily bold deposits. furiously unusual i +113|Supplier#000000113|5 YOpqbaHs7dR gG4EmXrI7XtA7DcnRMsWPU1z2D|11|21-211-117-1937|1882.05|blithely regular courts wake quickly even pl +114|Supplier#000000114|Uvz iykaBYOC|20|30-998-334-7841|4559.18|jole slyly blithely regular requests-- even requests haggle alongsid +115|Supplier#000000115|nJ 2t0f7Ve,wL1,6WzGBJLNBUCKlsV|23|33-597-248-1220|9192.10|es across the carefully express accounts boost caref +116|Supplier#000000116|gABGPfNYwB 9g1rMYWSAhpo 4|13|23-188-305-1829|5751.39|xcuses wake quickly above the regular packages; s +117|Supplier#000000117|ZnlLNC,8YN10T4mjI8eq5bJ|18|28-470-879-3141|4589.18|ymptotes. blithely regular theodolites are slyly according to the +118|Supplier#000000118|BYtvNtFpQAHHoBFWF|0|10-475-868-5521|7317.43|y ironic theodolites. furiously bold ideas use along t +119|Supplier#000000119|4CxBrM0o4yt6LYFxZIyZ89Xnf8LZNn6KcYc|18|28-558-264-1202|2060.13|ctions: quickly final courts wake quietly foxes. packages hag +120|Supplier#000000120|TNxoTw0SiMmQzGfZY9fTSkL2kmtPwvtR2L|7|17-645-761-5674|5575.40| beans cajole of the sly, dogged courts. +121|Supplier#000000121|CWGri,tKI 7gDcDsI|6|16-275-849-2485|5733.61|against the ironic, permanent pinto beans. doggedly pending deposits sleep agai +122|Supplier#000000122|2RUSHHspScCVTWC6z vw2XVR|16|26-432-258-4986|2732.95|ackages. carefully special accounts use slyly. slyly silent i +123|Supplier#000000123|IqRn20xsj5ibqAQjb6YNQf0xah|19|29-602-688-1506|5726.19|nts x-ray quickly according to t +124|Supplier#000000124|vmVadCZ xHPbQQA2fLxr68T1YhmjVSuRUJKCrcq6|8|18-313-889-3713|-941.38|ular excuses after the fluffily pending pinto +125|Supplier#000000125|XG eO4Xb4TSF7rj4R6WRQ1v2seTlyga3tvFZaC|2|12-419-430-3983|5157.25|ven accounts. fluffily ironic deposits are carefully. s +126|Supplier#000000126|CaO4YuZ oSkzemn|14|24-728-670-3468|6829.86|unts. carefully regular dolphins s +127|Supplier#000000127|VEqo3HZJ,0ggcaxvqZnW7Rq7l,cPqfKgX2bIRavM|1|11-265-565-1616|2502.95|yly above the fluffily ironic accounts. +128|Supplier#000000128|u,I0ayLPI2HKL|23|33-957-200-2556|1091.04|ng requests. slyly final ideas affix slyly even requests. fluffily regular theo +129|Supplier#000000129|9kWUk5K 7TAR40dW5nVeg i60Fy|15|25-314-164-7014|9001.17|even excuses wake even accounts. slyly bold excuses +130|Supplier#000000130|Rnt93MAs0EpPZjPYQIlGOWNUjk|23|33-322-101-6260|8708.17| blithely regular packages sublate. closely idle accounts are fluffily final, brav +131|Supplier#000000131|u3mTHMgBC0yJTLufr01TuHImgflQUXv|14|24-293-181-3975|1301.20|to the regular, pending instructions detect against the p +132|Supplier#000000132|sU92Jd3aQSbV|5|15-776-909-1326|-812.17|al, final accounts cajole requests; fluffily pending instruction +133|Supplier#000000133|QT,semYwdx|4|14-731-952-5158|9754.60|he quickly express instructions +134|Supplier#000000134|Nv7dxj4FGWrdcP56RmNci|6|16-184-952-7778|-329.16|c deposits haggle. dinos a +135|Supplier#000000135|F4Uy ZQNU6ESTmO3mrL,mI|7|17-290-812-8855|9767.99|courts wake slyly instructions. furiously silent requests cajol +136|Supplier#000000136|SyWVom9ZFrTA6BMAS|8|18-175-739-8397|4623.48|requests. boldly regular deposits are. packages are sometimes! c +137|Supplier#000000137|j81M1VQCvKcIVnpMnIUnBNVUOm0XYJuJY|18|28-108-440-4890|837.27|he blithely bold theodolites cajole blithely among the +138|Supplier#000000138|utbplAm g7RmxVfYoNdhcrQGWuzRqPe0qHSwbKw|19|29-533-434-6776|906.07|ickly unusual requests cajole. accounts above the furiously special excuses +139|Supplier#000000139| 2mQLQsVJ8WLBSnl0R bXrcyTgqXKrplgxb|22|32-788-265-2743|2217.93|arefully ironic ideas: slyly regular deposits about the furiously ironic requests +140|Supplier#000000140|mdfYYe8U sSb|16|26-379-377-5829|6727.48|sly final pinto beans affix furiously about the packages. even, bold accounts affix permanently fi +141|Supplier#000000141|5IN1dvjqFhyfKxoslkY3UL7CrNVPCZmq|10|20-557-547-7976|8842.14|quests detect blithely even ideas. unusual, regular accounts sleep blithely carefully regu +142|Supplier#000000142|HvfHnP57pz0w6l|16|26-302-532-7958|8991.41|dly ironic packages cajole amon +143|Supplier#000000143|a3v,6RZMN9p FzRTdV,fm7ehoVgEhifejGnrNY4H|12|22-916-300-5765|9658.99|s lose slyly about the blithely unusual asymptotes. blithely silent instructions cou +144|Supplier#000000144|f8tddEKps816HHqNwsKdn3|20|30-726-423-7363|9806.29| carefully even pinto beans n +145|Supplier#000000145|pSsVRBOlNKVuvXcjAEKxxy0hD8kA aZf|6|16-136-582-9756|-685.94|he carefully silent requests. quickly careful deposits are quickly about the bold, r +146|Supplier#000000146|rBDNgCr04x0sfdzD5,gFOutCiG2|22|32-792-619-3155|8271.39|s cajole quickly special requests. quickly enticing theodolites h +147|Supplier#000000147|oLdl SQpf,ZXebBw5F3g9|23|33-828-583-6731|8387.76|its. carefully pending packages after the ironically special frays wake above the special deposits. +148|Supplier#000000148|bkCBZzewuerw8xHv|7|17-648-666-9156|6089.75|ckly regular grouches. carefully ironic deposits cajole acc +149|Supplier#000000149|pVyWsjOidpHKp4NfKU4yLeym|6|16-660-553-2456|4518.31|ts detect along the foxes. final Tiresias are. idly pending deposits haggle; even, blithe pin +150|Supplier#000000150|kZajmmtJB4g,nNlaHxUvXLfZW0hVtnR35LKA|5|15-169-420-7828|4161.22|l instructions sleep quickly regular requests. final orbits use never furiously ironic reque +151|Supplier#000000151|2hd,3OAKPb39IY7 XuptY|22|32-960-568-5148|8561.72|hely final packages. ironic pinto beans haggle qu +152|Supplier#000000152|jTZy8PjLJqfLxgHm7,fDoJu7ZMs luO2YmN63|24|34-659-493-1274|3846.60|lyly even platelets cajole. fluffil +153|Supplier#000000153|qkCHAU1v9CtEBOGXhdFXJsx5L5gViVm5k,|9|19-346-843-5260|850.55|ress quickly. even accounts are around the carefully bold packages. slyly regular pa +154|Supplier#000000154|gB51OPlY yleFDn,pnDRbsDqy1gYM,SNRbo|13|23-471-808-2661|4155.67| special ideas haggle carefully ironic pack +155|Supplier#000000155|NFoZlgq90N33cC4HbLcfIGb1uqIZy85l42qa6|22|32-417-987-3690|3401.43|es! slyly ironic requests cajole above the daring account +156|Supplier#000000156|,KXbCcjqZNjmyHzeqaL4|5|15-106-692-4998|9780.74|sits cajole carefully instead of the final, ironic requests. furiously ironic deposits abo +157|Supplier#000000157|,mEGorBfVIm|3|13-776-259-5994|-963.19|ove the silent deposits. carefully pending packages cajole furiously final packa +158|Supplier#000000158| fkjbx7,DYi|7|17-873-902-6175|1596.44|cuses sleep after the pending, final +159|Supplier#000000159|xftQu5vkiD6BF|4|14-606-224-3002|326.36| blithely quickly ironic platel +160|Supplier#000000160|LG6VM3F8MhGnmoZUpyqHgsV0IP6gOagh|4|14-471-505-8811|5067.64| after the furiously express deposi +161|Supplier#000000161|fsteD4OulIaUNa IPXVesILV|0|10-604-123-7428|7810.78|doze among the slyly even platelets. packages believe blithely furiously final ac +162|Supplier#000000162|hf2fnryOkI9VjFN8R8i5rRWguXzdlMu3o|9|19-905-888-6020|-170.22|sits. even frets believe fluffily. fluffily regular accounts +163|Supplier#000000163|1MhydNirC ,cuLIZezbkUxJhAC53ii,B|12|22-980-516-1217|7999.27|es sleep about the fluffily express platelets. even multipliers must have to sublate. bli +164|Supplier#000000164|7H,0hAZkls5qVS 6sjbnVnQtpjMnGEv3gKXVS|2|12-414-446-6598|-264.20| the stealthy accounts. quickly stealthy warthogs detect. final deposits +165|Supplier#000000165|iPso5qCxSnxaNsRe9AU05Vl9hWm5oHIS|14|24-906-333-8640|1550.62|ions sleep silently. furiously unusual requests are slyly. express requests hind +166|Supplier#000000166|zCr2Z0hHyLjSz|13|23-117-751-3240|258.33|s. regular, unusual requests about the bold packages grow careful +167|Supplier#000000167|FEDldnEfBHVGOmKdFevemmG2|20|30-725-194-1727|5731.87|rding to the slyly bold deposits haggle furiously furiously regular accounts. blithely bold pac +168|Supplier#000000168|NNvNhn6dWzr80Igr|9|19-985-776-2090|6559.35|y final packages! express, ironic foxes snooze carefully above the furiously daring theodolites +169|Supplier#000000169|ycymrfB5JV1vU,swPXggAt|13|23-698-509-1073|-927.50|ss, even accounts. ironic packages sleep blithely after the slyly regular d +170|Supplier#000000170|RtsXQ,SunkA XHy9|23|33-803-340-5398|7392.78|ake carefully across the quickly +171|Supplier#000000171|eRpB8T GdtMBvJOo|10|20-229-200-1299|3549.54|es are. blithely final packages are unusual requests. fluffily expre +172|Supplier#000000172|NckigAXBRUXbJI|22|32-481-329-1585|2077.39|efully ironic packages x-ray thinly. slyly pending hockey players haggle slyly. sly +173|Supplier#000000173|OqQzF6rfxDvkjpMXVCwGfQzj4oTHBHyW5kC5Gjxd|12|22-640-545-4690|9583.11|ly regular escapades use among the express deposits. unusual, silent deposits wake. ins +174|Supplier#000000174|e1NMjwAq6RdyIKeA|22|32-840-184-9487|-561.50|s. blithely special dugouts integrate furiously. furiously bold accounts haggle th +175|Supplier#000000175|Lgv gYbBdu S9|12|22-392-226-3266|9845.98|leep. quickly blithe theodolites wake slyly. furiously pending ideas haggle after the slyly +176|Supplier#000000176|OLVnGuOx8m6NfApzODj4 JP01JJIm,qI53BChmgQ|10|20-970-245-2712|6387.89|s according to the carefully expr +177|Supplier#000000177|IUZ 7G x212nzZY5aQS|12|22-246-174-4465|-741.79|fully after the always ironic theodolites +178|Supplier#000000178|VJ9DInoVjbDg|16|26-471-122-2582|4693.27|hely final foxes instead of the express, expres +179|Supplier#000000179|d3le3XaTUC|19|29-560-587-5604|7727.83|kages solve carefully alongside of the furiously regular patterns. blithe +180|Supplier#000000180|JJzFp5wZcS0KpMLM95tYmq5Pv526UBfT8vrfwBk|7|17-600-237-1665|2753.77|ic deposits wake furiously even, express accounts. slyly express packages detect doggedly +181|Supplier#000000181|7g8adZQXXuHAYdoULLDVKYRtBhdddGqYj7pMzva|14|24-682-737-4806|3665.07|usual ideas. silent requests boost across the quickly regular instructi +182|Supplier#000000182|KuSXGdsAdYdkhzVLmgo4Xs|7|17-825-333-7344|7872.30|gular instructions. blithely regular sentiments around the slyly silent dependencies inte +183|Supplier#000000183|zAAIv68BEXvllrfgsW,i8e|0|10-842-403-7954|-192.51| slyly final dependencies alongside of the fluffily iro +184|Supplier#000000184|VTbbcJp9vdqn,tJA5pG5V,596mud6ZdspXgpUWX |1|11-802-685-1889|7448.53|special deposits cajole quickly even asymptotes. quickly ironic d +185|Supplier#000000185|F4C0AWRPk3|17|27-586-282-7422|5435.91| pinto beans. quickly express packages +186|Supplier#000000186|g huGEW5nrQ0Lmd6|1|11-398-253-5445|4397.48|g along the pending deposits. slyly final foxes sleep fluffily above th +187|Supplier#000000187|oMtyTl6hTyLQhiBwrUaa42zKOjfNobEq|18|28-127-320-9230|8517.04|riously along the carefully silent instructions. +188|Supplier#000000188|boP,dP6PjNCADoMv3FaXKRREikgs4J7cYng|20|30-573-811-4001|1020.57|ual requests cajole. final deposits +189|Supplier#000000189|SlPFDFJnn4gtrol|9|19-482-239-6669|1636.09|y regular courts. furiously express deposits haggle slyl +190|Supplier#000000190|qpXGL8oM1 wzQd|10|20-995-359-7606|6705.44|regular deposits haggle across the final, si +191|Supplier#000000191|X dZg6Dtv17X7|18|28-437-699-1621|1445.00|en packages haggle blithely regular requests. silently ironic packages n +192|Supplier#000000192|Tub1t4UlJwZ5U|15|25-585-189-5975|7031.84|ccounts use blithely. unusual, regular ideas use qu +193|Supplier#000000193|0HvoBt,qEF EaKzYNCl|5|15-872-804-8448|9095.64| packages. regular ideas sleep about the fluffily even deposits! special dolphins nag sly +194|Supplier#000000194|MDIkT8cHs7|9|19-741-822-3512|1487.08|nusual platelets cajole according to the +195|Supplier#000000195|xWy21YlUy3R6L01|2|12-395-261-9720|-767.31|e unusual foxes. express, final ideas sleep! carefully regular ideas around the quickly expres +196|Supplier#000000196|TQX4fMwB5tXoz4Fi4CEhALhi6|18|28-430-406-1127|4710.62|ke fluffily regular requests. furiously final +197|Supplier#000000197|YC2Acon6kjY3zj3Fbxs2k4Vdf7X0cd2F|3|13-999-250-8664|2398.30|ithely final ideas within the furiously ironic requests cajole +198|Supplier#000000198|ncWe9nTBqJETno|6|16-355-298-7120|2187.91|ts are blithely stealthily unusual asymptotes: blithely final excuses wake. ca +199|Supplier#000000199|k,8F8FGDuN 3udblO|22|32-268-147-8879|8151.22|requests. slyly express foxes across the blithe +200|Supplier#000000200|MNqafnV52UKvwuk3jFCn,AJkL|11|21-962-509-4762|1589.13| beans x-ray enticingly express accounts. blithely final deposits run; regular packages are +201|Supplier#000000201|e3lZO QY9QpCVdc0HXrqXB5uxXnOi r|18|28-782-585-7326|9537.73|kly above the special ideas. even requests nag carefully. quickly pending t +202|Supplier#000000202|NALZjSfea6SY zB1,I09OJYGrA8bwR4pU|23|33-549-918-5721|6739.52|courts cajole bold, special accounts. bold packages haggle re +203|Supplier#000000203|wvdhblw9JkIe52z0gnGSDrDBsMsI3Aidqk|1|11-257-649-3327|2150.78|ar warhorses detect. carefully final requests since the unusual, ironic deposits b +204|Supplier#000000204| 7bts1RIiMsy35F6V7 lC|15|25-718-760-9193|172.87|cording to the furiously even pinto bean +205|Supplier#000000205|rF uV8d0JNEk|3|13-272-651-7242|-670.30| beans cajole regular, pending packages +206|Supplier#000000206|hva5xXEn5j7H27CdKcjFqi,QnunUZyt8,a|19|29-156-330-8311|7136.21|quickly pending packages. regular, quiet packag +207|Supplier#000000207|0QmBvfdQPh7Xy09txlnJcv|2|12-245-833-1389|3285.77|xes. pending, regular deposits x-ray against the unusual deposits. final, bold platel +208|Supplier#000000208|kr uyD ,K95lEvq77tuHVjOJM57|4|14-932-450-9228|1938.69| instructions boost blithely along the +209|Supplier#000000209|fpTboatC6ogrozMPApz2DXmlukC9YVzCLCgQOtgU|18|28-134-789-6780|-395.27|ly ironic ideas sleep carefully even courts. slyly speci +210|Supplier#000000210|eC5 e7DXYBWi8XlnFtBxF,tSWPyQAzqb8gAZ|2|12-385-448-5157|9657.88|ccounts haggle after the carefully pending acc +211|Supplier#000000211|acmexYmhAZhFyM|2|12-150-553-5979|2080.07|quests sleep final pinto beans. blithely bold theodoli +212|Supplier#000000212|ZutN4kNIsv7sPWIV9,7|7|17-617-724-5874|-335.47|ular requests cajole furiously against the spe +213|Supplier#000000213|1s7 4odatD2BWz1QjXR2 2SnFhc,Rvr2Icxh1m7f|18|28-317-938-6979|2616.54|, regular packages. request +214|Supplier#000000214|B3uLKyb, xkfHbTSUBe6HwwaBPdCvhiOqO4y|14|24-114-624-8961|7089.25|omise blithely regular packages +215|Supplier#000000215|YmZQvVCVanip2E|12|22-333-479-3656|6125.89|fully upon the slyly bold excuses. blithely regular deposits promise across the slyl +216|Supplier#000000216|K83M7iWDJx N Y|14|24-182-902-2539|6902.54|luffily final theodolites haggle slyly pending platelets. speci +217|Supplier#000000217|UVrZj7M2mMd7JLjbc tvFmQad14d1oid|15|25-113-702-9259|3084.60|arefully final accounts. slyly even ideas haggle along +218|Supplier#000000218|c7CBVglgEM0XU,8bOU76GjEr5L5EXu9Uxa7U|10|20-180-767-8426|8843.88| fluffily. furiously regular instructions sleep slyly furiously regular packa +219|Supplier#000000219|f0siVrn1T,dOIJgYU|9|19-708-346-1944|5222.69|odolites use carefully about the reg +220|Supplier#000000220|uvWVvuY3tHcE2W34jcO538wy6|19|29-197-645-8102|6746.19|. furiously regular foxes solve blithely. ironic requests mold carefully. blithely unu +221|Supplier#000000221|aU oCBZs0CUUTU|11|21-402-925-9045|4283.34|onic dolphins nag furiously across the silent deposits. carefully even ideas snoo +222|Supplier#000000222|2JQCRHT8coRlrMria2|15|25-899-779-7536|4173.23|ctions x-ray against the s +223|Supplier#000000223|MCS3 zWF3Py8UZK PHdI6LsQD98phHJ7|2|12-790-796-4808|1518.42|y fluffy packages solve amon +224|Supplier#000000224|iD7eF,uwQxxPCAFTmdXYV7N|5|15-446-447-2944|9036.79|atelets. regular, ironic gifts eat furiously across the permanently silent instructions. reg +225|Supplier#000000225|6Ez0sxpUfmtjqFDO|9|19-787-123-9863|2461.11|d packages. unusual requests after the furiously bold packages wa +226|Supplier#000000226|83qOdU2EYRdPQAQhEtn GRZEd|3|13-768-844-6969|1906.46| deposits run fluffily: excuses breach closely against the slyly regular dolp +227|Supplier#000000227|Qo959Dll Bd7xvfq3ELtCq|4|14-215-994-7949|7896.01|ng accounts solve furiously at the furiously express packages. carefully bold packages sl +228|Supplier#000000228|pyTY uocaSasIUlrHUbBwM,r,|14|24-920-216-6514|2291.35|s above the final deposits detect along the furiously idle packages. b +229|Supplier#000000229|ycjgLrk,w8DcakfwTS1SO5kVch|14|24-392-655-8913|8751.47|ly according to the carefully regular +230|Supplier#000000230|TgT146Clu9ODbYYBx4Wd8oe|1|11-819-222-2128|2975.98| regular dependencies. packages after the accounts are blithely blithely bold packages. furiously +231|Supplier#000000231|uZfFnyAs9oFJ0No97rtQ OIV1AOYi|5|15-551-432-9958|9287.33|mptotes sleep above the regular dep +232|Supplier#000000232|90YJjotHlfwyieaTfuBJ8kohU5Oc83bESout,p|7|17-478-427-3811|300.80| asymptotes lose slyly evenly ironic asymptotes-- blithely bol +233|Supplier#000000233|Pu9mqssUjJNoqiGG4vfLTn3ENJnNPVrBNnXJM5Eb|6|16-283-737-6972|4544.18|kly pending platelets sleep blithely along the ironic, +234|Supplier#000000234|iMrk7HUD87at3IIh4rBi|24|34-187-193-2209|9957.00|nag furiously pending dependencies. unusual deposits integrate fur +235|Supplier#000000235|W89jjgy458|8|18-835-352-6531|2424.93|ackages run blithely above the fluffily even dolphins. dep +236|Supplier#000000236|dZExtd1dlMyrPdSqDC3|15|25-250-525-2436|9844.00|lar platelets. blithely ironic packages cajole blithely special ins +237|Supplier#000000237|NzEXFiPN5tnrZzs1E,uGeoEqolOlDYE7oQHnQWg|23|33-976-749-2413|6170.98|regular ideas. carefully express ins +238|Supplier#000000238|xjSZNq AIqmrV UfxyGcS04RYOzW|8|18-950-354-3488|5763.17|s. carefully final courts impress furiously among the slyly regular deposits. ideas thrash furi +239|Supplier#000000239|XO101kgHrJagK2FL1U6QCaTE ncCsMbeuTgK6o8|22|32-396-654-6826|4672.25|arls wake furiously deposits. even, regular depen +240|Supplier#000000240|yMzL86zw28z6sMa|10|20-843-630-4161|6537.07|sly. final, regular pinto beans unwind slyl +241|Supplier#000000241|K3sXvTEnHT9yQSbzZmbPNY0,iPQsmySI|8|18-213-514-9357|9594.49|arefully final foxes. regular, pending requests haggle busily special asymptote +242|Supplier#000000242|cpZMlI77TRq|11|21-489-286-5908|3736.64|nic accounts sleep furiously according to the quickly pending requests; slyly bold deposits haggle. +243|Supplier#000000243|8aQ3HGeOXxgYeMAXZQe B5y2RKEF5jdmN3Qb|6|16-554-376-5494|747.88|kly silent requests among the blithely regular foxes use fu +244|Supplier#000000244| c6fBN9a 6EOcB1ZjbImMBAQMwI BKScDNVRP8|1|11-556-986-9638|5489.08|d blithely. pinto beans boost quickly. blithely bold accounts affix. +245|Supplier#000000245|8qUceq1TSsNaTol3Ul6TDNSu9yRddanHEL|8|18-739-793-2096|6943.28|ual deposits x-ray slyly. carefully regular epitaphs about the +246|Supplier#000000246|BatWQwH8DgkBs|4|14-278-988-3903|5740.87|ealms are. even theodolites use regular, unusual ideas. ironic, final ideas sublate +247|Supplier#000000247|0bkES oiL2joJGmxdGwPfVCOL,pIQ4JNZBPnOR|23|33-695-935-2388|3758.64|final requests. final accounts affix. express accounts about the furio +248|Supplier#000000248|vDPBiu4L7hPZxwaEoNW u,zHgXVHvBXFBdfuek|6|16-646-663-7202|7487.64|ss packages are idly about +249|Supplier#000000249|ktpns6Bvu4fP8Wzzlc8|10|20-241-221-7700|2800.60| regular excuses about the blithely pen +250|Supplier#000000250|JkzOvHRi1rEdkZ|9|19-347-403-1571|9681.99| packages cajole blithely pending theodolites. ironic, silent requests +251|Supplier#000000251|Uqi3s, iqzLxI4duoRfgkciiN4XuCvITGIUf|21|31-869-288-1674|283.61|eep blithely even, ironic requests. carefully pending courts above th +252|Supplier#000000252|xcaKgadrug|11|21-860-636-2585|1523.14|tain slyly fluffily bold ideas. furiously ironic ideas nag furiously slyly even requests. +253|Supplier#000000253|DbryhLi6gCv8A P9FkjNp56mLP4KnV9Do|21|31-811-659-6792|352.61|ng courts wake quickly against the slyly careful packages. even ideas nag caref +254|Supplier#000000254|c6h4mizJAVT0Oz|7|17-100-212-8737|6230.48|nos. bold ideas wake carefully among the furiously +255|Supplier#000000255|qx16XyCEUh9OawVeQWOlGlhAU32iHFPNkO|18|28-629-327-4139|4663.08|s boost. ironic pinto beans along the slyly unusual foxes haggle regular, final asymptotes. reque +256|Supplier#000000256|ma15idU1PzO7itP67W1Rd|11|21-370-961-2170|6469.78|sly pending deposits haggle +257|Supplier#000000257|BjFRji2XvSyOVe MHMP9r5vC2eP4kfF GXwBe |16|26-806-563-6761|630.74|odolites use slyly finally silent requests. package +258|Supplier#000000258|z8ImnYbuBbohh1 WT|7|17-820-568-1141|5736.09|press packages. final dolphins sleep sly +259|Supplier#000000259|2zzYBzG12K|10|20-901-530-6532|439.97|ts wake slyly along the express dep +260|Supplier#000000260|TQGxRpJe08nPRqPYDIMTnst87mC0HZJ,rlnCQTF|23|33-226-804-7400|597.64|silent, stealthy instructions. furiously final tithes hinder regular requests. expres +261|Supplier#000000261|vUT2UDI,GAqIA|0|10-547-567-3297|-466.40|kages. furiously express requests are. carefully silent deposits at +262|Supplier#000000262| 3kCm2eN3Jra2UzvWWBm,P04b|20|30-952-865-8647|2138.97| ironic escapades. furiously even ideas affix blithely. silent accounts thrash beneath the +263|Supplier#000000263|7Vdou,WjHE|14|24-203-272-3139|-316.81|s quickly regular foxes. stealthily ironic p +264|Supplier#000000264|Oeq2ei9wRqt6yOuAAb0KoMRyoH4v|19|29-870-178-3535|8327.34|e busily ironic theodolites. quick deposits after the unusual dependencies print sly +265|Supplier#000000265|eHF4Edu,B8,NgBSSEV4xNC37i1q08WCNKyOe6jP|19|29-734-865-6334|2638.54|le evenly besides the fluffily fina +266|Supplier#000000266|LGWx4Xcq0GwQVwTR|23|33-256-678-2321|2874.22|y express foxes. special, pending deposits are slyly-- packages detect blithely ironic se +267|Supplier#000000267|,E9NoXeK5qOwVRoutS,X8Gr|10|20-501-962-2896|4068.14|riously close asymptotes are. carefully bold p +268|Supplier#000000268|ApLM6aebtP79LIp|23|33-863-692-9039|6261.99|nic frays use furiously among the final, ironic the +269|Supplier#000000269|LxOgEClzbXDEYHYG2ZMFoWHrD,C|2|12-988-623-6829|6331.84|lar foxes wake quickly about the carefully ironic asymptotes. final, regular do +270|Supplier#000000270|PM P65mhBoeFWkFMi9,|23|33-682-295-4727|3339.18|ss slyly after the fluffily regular packages-- quickly express pinto beans nag blithely +271|Supplier#000000271|U0BSsnILvbk7mL1cim,ZX70QIiTz|15|25-270-292-6296|8576.95|its nag above the silently final escapades. final, even inst +272|Supplier#000000272|ywrDqLLTfKUF93|7|17-209-164-6836|6858.73|of the furiously final accounts. bold dependencies after the fluffily ironic asymptotes hag +273|Supplier#000000273|O0TyoQXlzJx|15|25-247-243-3871|3708.18| pending ideas. carefully silen +274|Supplier#000000274|usxbl9KSW41DTE6FAglxHU|21|31-571-345-4549|6296.15|ecial courts. express asymptotes must have to sleep theodo +275|Supplier#000000275|Mh9eZVjmBPZc5I2KHcMcfZOYbpFqrlVOLR|21|31-446-410-5215|8477.48|packages. ironic packages are. pending deposits are according to th +276|Supplier#000000276|KdVDs6EGfWVsPdjuCh9iep|6|16-752-344-8255|7144.78|cial, ironic theodolites against the decoys cajole slyly ironic foxes. carefull +277|Supplier#000000277|REebuerbQmMxlbCXnl2ihK,VyItkTNxU36 c|12|22-758-939-2357|4300.15|final deposits. unusual pinto beans after the even accounts affix idle, final +278|Supplier#000000278|gBQmkVmclUGbjwHVtrdmCk9Dwqd2Dez1|9|19-563-887-6379|665.89|deposits against the final foxes use carefully express ins +279|Supplier#000000279|aVGVO0XNwTEkFwH0OhaQMJC7un0EizPpq|13|23-617-962-7353|2613.77|accounts. quickly special packages nag at the unusual requests-- special d +280|Supplier#000000280|MZAJPsr3TS62ocxtRc|11|21-536-349-7369|4540.53|nticing platelets. ironic theodolites about the regularly final accounts sleep accoun +281|Supplier#000000281|A2sesSQAAj6wvPPKL X4caRp,O|0|10-553-675-3578|3717.19|ooze fluffily express, pending pinto beans. regular instructions haggle ironic +282|Supplier#000000282|Gh,k4vQQPzOyIA9ac9re5IjaV5LyXDtZX9O1kNG|8|18-420-459-8366|3751.04|into beans. carefully special excuses cajole furiously. regular d +283|Supplier#000000283|MWLpGjjs1WoofKSoN7Lm0DXcSY5H4E9CeneF67ZQ|17|27-409-203-4267|785.26|ess instructions. slyly unusual deposits according to the accounts run past the slowly even package +284|Supplier#000000284|7NF0OkJazPIJ7l,LBSw3abv|15|25-955-684-2997|6866.81|quests nag carefully according to the furi +285|Supplier#000000285|Br7e1nnt1yxrw6ImgpJ7YdhFDjuBf|3|13-907-873-9658|7487.27|hely? slyly bold patterns doze carefully according to the fluffily even packages. bol +286|Supplier#000000286|o80iAPvmwqM3WOA93pqBHT4Dsgy1rwG|10|20-459-893-8984|7517.31|iously regular pinto beans sleep carefully slyly even accounts. packages cajol +287|Supplier#000000287|7a9SP7qW5Yku5PvSg|3|13-484-626-4797|6541.85|nic epitaphs are slyly above the foxes. ideas after the quickly bold pin +288|Supplier#000000288|8fUJhFrKTMmi5L|10|20-850-734-7789|1884.76|ual deposits wake furiously i +289|Supplier#000000289|6Tua0IHqN0sod1Tpdax5hD0|16|26-396-901-4544|6144.47|equests can wake: furiously final accounts lose regular, final requests. special somas affix. fi +290|Supplier#000000290|6Bk06GVtwZaKqg01|6|16-675-286-5102|167.56| the theodolites. ironic, ironic deposits above +291|Supplier#000000291|0qDDQst1b1bznHQh5jsmOq8nxf8Pz1Kn|0|10-404-519-2270|9132.12|wake silent, pending accounts. blithely final as +292|Supplier#000000292|POEdp8d ug5bKQa,w4SAMaM2xna1ZtVk4z9loa|21|31-933-112-8506|9598.62|ular excuses. furiously even requests sleep carefully around the fur +293|Supplier#000000293|H2JnUWv1X3s0yI7i2tY5Vd0kd4f0|9|19-143-962-9484|7114.81|t the instructions. daringly bol +294|Supplier#000000294|20vnacv,dtZ0mr5kukNf|18|28-299-788-1836|6700.75|en accounts. silent, regular instructio +295|Supplier#000000295|gpm7fahY9j6YyTr Dozul|23|33-998-989-3147|-435.02|en requests according to the +296|Supplier#000000296|g,WJbekrbjAcpNtn2QRsWtYx2RNVk 9aY|14|24-250-680-5383|1549.59|through the blithely regular depo +297|Supplier#000000297|gfVOI9aT2roNGEgc|1|11-562-493-7660|4954.57|al, express epitaphs? furiously ironic ideas haggle regular, ironic instructions. carefully special +298|Supplier#000000298|oIB13yXjsqGSW|5|15-790-635-6692|-530.05|egularly unusual accounts +299|Supplier#000000299|kvWv3MYn,Q2SqJlckBfhpWYmnSeRwg6|10|20-180-187-8847|-561.38|lar asymptotes integrate darin +300|Supplier#000000300|YU QZvXHJC7,ZspUPGwaIOa|2|12-468-732-4623|811.42|furiously even theodolites haggle along the final, ironic foxes. bold deposits are. +301|Supplier#000000301|YPFTsQOPRAGIlBw|4|14-434-699-9741|6472.62|express foxes sleep carefully even packages. carefully special ideas cajole slyly. carefully r +302|Supplier#000000302|ow8Io1JHkPnIAGCQYCWC|18|28-734-845-8630|4422.77|tructions after the foxes cajole regularly ironic tithes. ruthless forges across the slyly express +303|Supplier#000000303|MCFk0WJH4O9Fj,m3as3bdIgnP4pe 8qPj,wfo|8|18-932-912-3102|4500.90|ously carefully final foxes. fluffily silent requests across the fi +304|Supplier#000000304|b4rSMq4y hBDMJgcNp|11|21-261-644-8980|8564.16|endencies across the bold deposits nag +305|Supplier#000000305|pKYTaixBtQ3AGJLwndRNxUlLUkXN667JT|17|27-607-328-4914|974.24|uriously across the requests; ironic requests serve bl +306|Supplier#000000306|T5EUviO4mqrGO6ruLcIoo29M5S|16|26-954-209-5690|3979.54| wake fluffily. furiously final +307|Supplier#000000307|3wL9YHFIvddxzh3mwy6SSrpfmzKvwAGmXK|14|24-499-938-5607|2168.65|s above the unusual theodolites was quickly quickly perma +308|Supplier#000000308|mVY7rtoxj9uSqzAJIp TOWPsv080hbPbOStGZx|8|18-360-691-8400|3513.89|tructions hang carefully according to the furiously close requests. fur +309|Supplier#000000309|gT84r,AFSrZQpiSCnE02B3QK|2|12-950-744-5100|7875.48|riously final deposits boost blithely about the even foxes +310|Supplier#000000310|I5Mw,rGgWQOFVotMHUmVjH|0|10-119-372-1062|9312.63|ccording to the carefully express dolphins! furiously pending dependencies integrate s +311|Supplier#000000311|yjGDnCKi4Wmtim H3n9p|22|32-445-679-8585|7431.00|uriously final requests integrate. sheaves against the furiously final accounts are evenly abo +312|Supplier#000000312|8XYqBq S,OWbSp9Y2qGBWEV4hH l7ywxk|13|23-829-479-5202|7844.41| furiously requests. always bold accounts solve +313|Supplier#000000313|9xrpKrwr9,Jgvm5q rGkZePkpsH5xEXZnuLJug|9|19-648-945-5128|4114.68| beans. special deposits use carefully after the blithely even +314|Supplier#000000314|CW6uXfPdJ6lmcJ|22|32-389-335-6770|7456.49|s dazzle regular, final dolphins. furiously unusu +315|Supplier#000000315|HRLhxvqBP,f,NjeUiT|2|12-574-691-2832|7315.00|eas. blithely pending packages cajole blithely ironic packa +316|Supplier#000000316|uXJ13cI7fXByGLBoQ8VU9AT|0|10-382-551-5105|5189.82|ding to the slyly unusual frets. accounts use carefully slyly silent theodolites. slyly ironic depen +317|Supplier#000000317|EwuMPFkb3IrSqitX8ddgxy QWhXxeM4RoOu2d|17|27-131-276-7910|4277.34| packages according to the deposits sleep carefully regular hockey players. quietly bold +318|Supplier#000000318|Cm5fnQ4M6VeJF17mBbV6iB0XoQL31STrb3Su|24|34-466-917-6708|2268.02|ly express accounts cajole blithely brave instructions. furiously even +319|Supplier#000000319|tcES8Ay3zcHQER9K6gAKFWlDvcg nrbPl|18|28-288-466-7847|1578.18|ts are furiously even, pending pinto beans. fluffily idle courts cajole bl +320|Supplier#000000320|v2FlHgKKiGiDwR7uoJNWF|21|31-984-744-5308|1485.45|ns nag carefully carefully pending platelets. pending deposits dazzle. requests above the i +321|Supplier#000000321|pLngFl5yeMcHyov|19|29-573-279-1406|4941.88|y final requests impress s +322|Supplier#000000322|lB2qcFCrwazl7Qa|6|16-803-605-4129|6102.62| to wake. slyly even ideas breach furiously. ironic foxes haggle carefully. ironic asy +323|Supplier#000000323|0LEOmcTTomY1F0y|22|32-563-275-6438|704.83|accounts. unusual requests haggle slyly special packages. always silent instructions e +324|Supplier#000000324|QDsgOSozg jniYR2HzxofLZyk0qGdJe|18|28-637-452-5085|8042.43|ithely slyly special waters. +325|Supplier#000000325|bQgx8ABSanNFNfCHY,2 uPvGfafaMC5|8|18-671-443-5744|-786.95|unwind carefully among the ironic, final requests! quietly unusual foxes hagg +326|Supplier#000000326|9kFiCwhcBldg4xwm|7|17-390-604-7483|4680.75|quests could use furiously across the ironic, even f +327|Supplier#000000327|MoC7Jc7oThpZ34HmJPKuUbOZwOyPOb1ksGlvT8o|0|10-519-344-7532|6408.52|final accounts poach quickly slyly regular requests-- furiously careful requests cajole car +328|Supplier#000000328|SMm24d WG62|7|17-231-513-5721|5069.27|he unusual ideas. slyly final packages a +329|Supplier#000000329|zPg8 aqmUQ4wHGC 8nO39C50AOMB1VxTsJWmpYyz|17|27-571-511-5321|2166.85|endencies. boldly silent pinto beans cajole. carefully unusua +330|Supplier#000000330|MUxmICc8xS41A|22|32-845-516-3236|6199.85|ix bold requests! final deposits against the ironic ideas boost across the bl +331|Supplier#000000331|VkdMTxXHy22069CWmK658|22|32-337-220-5366|6929.91|special theodolites use furious +332|Supplier#000000332|OpvOrxiHWJ6QM|9|19-777-809-6033|1106.07|ve the furiously pending warthogs. slyly special ideas haggle carefully. express +333|Supplier#000000333|MaVf XgwPdkiX4nfJGOis8Uu2zKiIZH|22|32-508-202-6136|8702.02|oss the deposits cajole carefully even pinto beans. regular foxes detect alo +334|Supplier#000000334|NHMGXoP 8cWsNXvV46AkZZJuptUm4mFo8|13|23-767-548-6051|7848.40|efully regular requests: final requests solve quickly. pending deposits across the blithely express +335|Supplier#000000335|JSwzRFY7dDNKfBe8ebMtm|16|26-741-643-2945|5119.09|ong the regular ideas haggle along the close ideas. furiously ironic ideas alongside of the fluff +336|Supplier#000000336|orYe2VXtABdK sUvMCOZ9ZMhBJTPp7W3pffWmjZi|1|11-866-373-2323|3603.62|refully even packages above the unusual accounts detect +337|Supplier#000000337|IRrbCdIS,GB4YYhr|12|22-951-643-8793|9029.85|en theodolites-- special, final deposits should have to boost ca +338|Supplier#000000338|2b6Gj,rSxqeIiAOMquj6c03p GmAzIog BrF05|16|26-565-914-7207|6835.16|fully silent requests cajole idly. even dugouts wake f +339|Supplier#000000339| ESOHA53rEW5G7Z75w5hJ|5|15-967-506-6094|3589.64|ts snooze carefully express accounts. foxes nag carefully +340|Supplier#000000340|Pk5F9dPqsa3k16I3UR282gY|22|32-849-350-3343|5001.23|along the packages. blithely final foxes cajole. unusua +341|Supplier#000000341|sdb9yPRPun Z awDuDPUId2NH0Yrz7dnJ1GBDc|14|24-320-626-2836|2755.92| express deposits against the dependencies use blithel +342|Supplier#000000342|YgcA0MYOSB1ou|2|12-829-736-2942|6791.04|s wake above the quickly pending attainments. furiously special re +343|Supplier#000000343|LTRj tcLNq34je60R7LkZtNrEwmry00DIXguSxMS|4|14-617-526-7362|-905.27|ckages sleep among the slyly express excuses. special, regular pinto beans are carefully: express d +344|Supplier#000000344|F mMVT6PuRj5S|11|21-808-924-2361|7082.37|inal courts nod fluffily for the care +345|Supplier#000000345|ZYuibETx2zArwg|18|28-503-710-4886|351.98|lar dolphins. carefully unusual packages according to the furio +346|Supplier#000000346|9vl7c3azrqt9wITrcglQhIGfwea|8|18-148-870-6674|6710.10|ecial accounts. quickly pending packages print. slyly bold pinto beans detect slyly unus +347|Supplier#000000347|zg0m5HrQtl D|8|18-932-818-1269|2674.43|special escapades promise pending, regular deposits. blithel +348|Supplier#000000348|ZNEHF5c7kP5tPGdQ ZrJZNRdQnu0M9LVyO urMm|8|18-338-824-3860|1172.37|eas. frays nag furiously final accounts. fluffily unusual theodolites use furiously above the slyly +349|Supplier#000000349|21Sp51XIZ9vTChQBWv0CA1o1P,26clhjR|13|23-532-708-7267|1033.10|lithe packages. carefully final accounts on the carefully final accounts sleep slyly +350|Supplier#000000350|KIFxV73eovmwhh|7|17-113-181-4017|3294.68|e slyly special foxes. furiously unusual deposits detect carefully carefully ruthless foxes. quick +351|Supplier#000000351|ZLWTvVCSmwsKfElT7K 2O1Ui|12|22-508-407-2628|8684.60|ithely ironic theodolites play. decoys sleep slyly against the deposits. s +352|Supplier#000000352|LsVcltEi9NYu10ByH 5grND|16|26-911-452-7918|1189.55|ickly. platelets sleep slyly blithely slow escapades. special requests boost furiously. slyly enti +353|Supplier#000000353|Bo,HYtujUMsMvE,|15|25-153-823-7261|3646.16|ular packages. deposits nag slyly abo +354|Supplier#000000354|w8fOo5W,aS|3|13-796-527-4255|8965.72|blithely regular accounts around the special foxes kindle blithely across the even dependencies? +355|Supplier#000000355|DuCQn,7qi1KL a99mTWSY4Z9eC53dslWPABGj7|16|26-389-592-6792|6929.22|ackages cajole according to the slyly ironic theo +356|Supplier#000000356|Lefi6RDDtvaVXqvhlbMuUrVm45oJbtkZM,Mup|11|21-198-523-7929|397.74|carefully blithely ironic excuses. enticingly blithe packages along the attainments haggle carefu +357|Supplier#000000357|Vf7Hi5DuzZ6RJ,mfaXBVNqx0|20|30-147-268-1519|5724.04|e slyly among the furious +358|Supplier#000000358|V3yxhHhHSXqOoc5UPv5TwkVDGKQsG|3|13-831-247-2584|6974.74|ans. ironicCustomer requests cajole carefullyComplaintsy regular reque +359|Supplier#000000359|J5HqPZnfLcFMtlgwCnZPUI|24|34-121-923-9858|6476.58|ons. furiously unusual accounts above the blithe +360|Supplier#000000360|3avxIDL4YPrZVHie4rOiPrK8Z IJwEPqZaioHK|5|15-671-472-7761|4956.01|uickly regular asymptotes. packages ar +361|Supplier#000000361|f8IUYRmdVXhQC9qJQjWknCXmzhe38vCbk6|3|13-192-383-9438|1678.56|deposits. carefully final deposits cajole carefully after the furiously regular ideas. +362|Supplier#000000362|XdtN0U5Qm2Z|23|33-445-749-9918|5571.81|e furiously. slowly regular accounts sleep furiously. carefully bo +363|Supplier#000000363|sYpqZxYin8GKkPtNWKOnJMTLm9f5e0lZ61N8wp|1|11-220-343-2951|505.69|express requests cajole furiously blithely final deposits. quickly special foxes breach +364|Supplier#000000364|OhfGUPn8U9oBx5|9|19-653-706-8164|5105.84|regular dolphins promise after the special deposits. blithely final pinto be +365|Supplier#000000365|SZaykm40Np0vOKp|23|33-253-791-9564|901.98|s. deposits use slyly except the slyly final instr +366|Supplier#000000366|AtIdvjsMt9peVyEbpoDerNTteRF|12|22-648-291-8851|-535.40|ts. slyly special asymptotes c +367|Supplier#000000367|E Sv9brQVf43Mzz|22|32-458-198-9557|8488.53|ages. carefully final excuses nag finally. carefully ironic deposits abov +368|Supplier#000000368|3o5w6T5HzjFmSf1|0|10-694-873-8364|5783.96| nag fluffily alongside of the silent depo +369|Supplier#000000369|XKLa3tQT7,TgtuLi2Vme8vGyx|7|17-381-930-4614|2094.34|cording to the special, regular pinto +370|Supplier#000000370|yyNSJAG9UXcWit4SeMkEIrNcdVq5|0|10-602-768-3758|8515.99|ound the unusual foxes sleep finally within the furiously unusual requests. sl +371|Supplier#000000371|7kc0KqnPxrJuGZdrrec7Cn,wrCPdxPemNPZQ|19|29-501-449-3837|5358.50| among the ironic accounts-- regular accounts nod slyly +372|Supplier#000000372|Bdhu5NV4VfPYBxsCmK,YnkoHIaW|5|15-246-325-3001|1311.15|ltipliers. blithely regular deposits was above the furiously even accounts. q +373|Supplier#000000373|oQAQ3UNvyJW|16|26-656-301-9303|1461.85| asymptotes wake quickly around the slyly regular dependencies. regular attainments haggle along th +374|Supplier#000000374|svrrNWiqg1f3tEXZdVbFK CAtLDsW1CidtyS|2|12-699-158-6062|4263.58|ithely. ironic pinto beans use furiously abou +375|Supplier#000000375|3CIBgjwAjB A1uxkiJNjc 7pI9AKhvnr1BHV9|11|21-250-668-2735|1962.02|ully regular pinto beans acros +376|Supplier#000000376|L8OWL3jXMCR3Gh|16|26-752-731-5943|6579.41|usual dependencies haggle above the carefully regular platelets. never regular foxes detec +377|Supplier#000000377|L4SF6EzZ xhyZCQ59onlADR4|16|26-634-598-9185|6113.96|ly express accounts wake enticingly special, express frays. furiously +378|Supplier#000000378|FfbhyCxWvcPrO8ltp9|3|13-930-567-5190|4429.27| among the furiously pending excuses. fluffily express deposits except the slyly final packages +379|Supplier#000000379|jyGWzIJTAdI0Iko2o|20|30-202-917-6929|3698.31|sleep? express packages are quietly around the slyly even ideas. express accoun +380|Supplier#000000380|LiXmikqsO6R40FKovSUZpl|20|30-245-487-4913|5421.70|re. grouches against the blithely regular asymptotes sleep slyly daringly p +381|Supplier#000000381|NfyIKLJrXusWgmgkkCS6Gn3RRra|6|16-117-297-1825|7476.55| packages haggle blithely express tithes. blithely final deposits wake bli +382|Supplier#000000382|wNasxwKglHa|15|25-531-651-1411|3486.56|furiously final deposits-- even foxes haggle carefully about the fur +383|Supplier#000000383|pGEPbc70IKUZuTiTaigKuizjt7Y5oUoDL3Bq1|22|32-792-648-3774|-484.12|nic excuses. carefully even requests alongside of the regular p +384|Supplier#000000384|zMr51gtJ0Vu83Dk|7|17-554-428-8511|1342.17|taphs cajole furiously blithely final +385|Supplier#000000385|4RDwKCNc6 yBY|18|28-730-866-8837|5836.17|frays. requests sleep ironic theodolites. carefully even requests doubt furious +386|Supplier#000000386|zxFf8YWdD9ltSzw0NOTFpKcBH8zJrzif9|13|23-535-472-3290|-404.12| foxes wake carefully dependencies. slyly fluffy depen +387|Supplier#000000387|EEmqrW2gNAbuJjKuTPgA8kmKA0bZcQSmV|7|17-797-328-7624|3854.14|ld, unusual packages alongside of the carefully final deposit +388|Supplier#000000388|n27XQohXrXlJRLdsyXNoljPS|18|28-306-827-7902|6540.34|rate around the regular accounts. furiously special pinto beans use bli +389|Supplier#000000389|FW96liSdq3noHJpwM|24|34-885-883-5717|9467.35|nag ironic packages. ironic pinto beans would sleep furiously. regular realms wake across the +390|Supplier#000000390|8Qv7RjLQSFfyt5JpH8fsTf0|7|17-821-610-9726|868.36| to doze along the foxes. final requests are furiously. furiously express accounts use bl +391|Supplier#000000391|HBkwkigT2P9bU2wXBrPnQ|15|25-736-211-2793|6475.66|ckly furious dolphins about the furiously even d +392|Supplier#000000392|5YSB73Q4LMC9648IF1GGJAP|17|27-961-813-1153|527.38|ckages outside the furiously silent deposits sleep within the fin +393|Supplier#000000393|hxGMP2tFry WHEeI5cmEFVF027E|5|15-120-912-1594|6686.84|t the furiously regular deposits. excuses about the ruthless, regular +394|Supplier#000000394|5mGD3d,LeKycAyeYbVlrGMlzmT|9|19-517-731-4139|7685.82| accounts play quickly carefully unusual requests. blithely pe +395|Supplier#000000395|vYD9APwEz6R1BFWJ,GDJ7yCUZJm|18|28-723-165-7801|688.37|xcuses. regular deposits across the blithely final packages haggle slyly u +396|Supplier#000000396|WMtmXuQwBKESTTFH4ru1f|10|20-236-453-5458|6420.86|ts. quickly unusual accounts are fluffily above the express requests. daring, stealthy pi +397|Supplier#000000397|kFrhzYG1SR8aWrHsftcptN,im88pImSkxxFu|24|34-880-360-3462|3425.62|ependencies about the regular pinto beans haggle quickly about the s +398|Supplier#000000398|WTV,rO0S1KZFKlhLsUpAH|10|20-536-426-8920|9354.75|mong the courts nag slyly special foxes. furiously regular theodolites w +399|Supplier#000000399|UCzZPQfZXnRhGZcXfnnvkfnqLVEOc|20|30-814-148-6803|345.97| haggle furiously about the close theodolites. foxes along the bra +400|Supplier#000000400|QACx8vfYzPsZHCet2Yjst4e2XzjOXF|21|31-514-285-7013|4624.87|eas sleep furiously among the regular ideas; slyly bold excuses alon +401|Supplier#000000401|9 zStaJ sD|10|20-586-179-6994|2233.53|ages. dolphins integrate blithely final waters. carefully unusual accounts are fluf +402|Supplier#000000402|i9Sw4DoyMhzhKXCH9By,AYSgmD|3|13-109-731-3195|4943.01|around the carefully pending dolp +403|Supplier#000000403|TTVlcRcFrglnhCffA11iw l6bZyyaI9xcRF|9|19-513-268-3908|3102.18|he regularly ironic packages. idly final excuses integrate according to the +404|Supplier#000000404|RQwxP4tpScystYCqOJ,XpCWr4Kox4|7|17-943-161-3434|7311.61|. carefully silent instructions affix. blithely even requests unwind. final dolphins de +405|Supplier#000000405|dJIijVTX n7M0NDEQvCA |24|34-728-545-7374|3213.18|er the foxes. special warhorses nag fluffily +406|Supplier#000000406|zMhU58CDF4aHTeodxg9IgRZgq|21|31-926-216-4352|2867.41|o the even accounts. fluffily ironic asympto +407|Supplier#000000407|WliGC47Vto2nh7mj|11|21-112-803-6707|-459.62|ar asymptotes. carefully regular req +408|Supplier#000000408|qcor1u,vJXAokjnL5,dilyYNmh|22|32-858-724-2950|6173.87|blithely pending packages cajole furiously slyly pending notornis. slyly final +409|Supplier#000000409|LyXUYFz7aXrvy65kKAbTatGzGS,NDBcdtD|7|17-719-517-9836|-820.89|y final, slow theodolites. furiously regular req +410|Supplier#000000410|6V,FO4xJPwvxGzReYzVj6dwTSIechnSSCyz9iY|19|29-822-375-4854|6234.92| beans sleep outside the thin instructions. thinly even soma +411|Supplier#000000411|G9H53XVrdbhRgvQwho1AS|18|28-518-787-9625|2250.74|ial foxes. furiously permanent packa +412|Supplier#000000412|S2onAA,jGtQ3qfpN|12|22-351-499-2131|8183.66| the packages! quickly even warhorses haggle slyly along the final, expre +413|Supplier#000000413|GAufsRQQE P,dVCZWIMEUAsm,7|11|21-875-836-5585|7788.45| the fluffily even pinto beans. closely regular asympt +414|Supplier#000000414|FkmlHgU9pqCboQ32Lcx|21|31-118-322-1371|9547.00|. blithely unusual packages might cajole blithely regular requests. +415|Supplier#000000415|ibzrtLp NIBzzQVh2mc6M7GJj3V2Z5uKlIDw,z|20|30-297-706-6489|8575.44|inal deposits against the ironic Tiresias wake according to th +416|Supplier#000000416|F,9zQapGlzjmqRhVTj1DR|11|21-245-879-3004|3550.06|ic orbits according to the furiously f +417|Supplier#000000417|b3CbQxCMWWu,YyeQU 51fccuv7Mt|6|16-563-597-5520|-113.45|equests hinder quiet courts. carefully +418|Supplier#000000418|l07dIg BFdcW|24|34-689-611-9130|5229.01|re of the carefully final courts. ironic pearls haggle slyly along the bold, regular d +419|Supplier#000000419|FpWtqjkbqEXn|6|16-510-433-1061|2899.03|of the carefully express accounts. even tithe +420|Supplier#000000420|kLWtAMtbSn|12|22-971-269-4753|2968.22|eas cajole around the regular accounts. evenly even escapades +421|Supplier#000000421|tXZPR dOYjjbGjarXxKPn,1|8|18-360-757-8604|-128.86|c dependencies. quick, express deposits cajole quickly. fo +422|Supplier#000000422|iu4c7rkFFNOvmfx,aSs62I|10|20-645-417-6790|-755.64|ly even theodolites. blithely c +423|Supplier#000000423|VCgMjClu4IDaVVMwMW0ARf1ho|24|34-577-174-3894|2937.16|quests; bold deposits lose pending deposits-- slyly pending packages play slyly. regular, ironic mul +424|Supplier#000000424|uOdFKME6fSAI,rvLcpTL|22|32-406-948-7901|5881.52|es. furiously pending ideas against the fluffily si +425|Supplier#000000425|a KnEGf,bqEnGd2Wd9Tl|0|10-262-132-6639|2805.77|ular pinto beans are among the fluffily bold acco +426|Supplier#000000426|zjIHPRMAI8vF|23|33-576-289-4702|8621.42| requests nag. slyly regular ideas +427|Supplier#000000427| Hnd2bzQ95, Adq bg7BQAbFVmRV9rQ,jY98|20|30-315-782-5162|6223.17|he regular requests haggle blithely about the forges +428|Supplier#000000428|v,lOR2jikAbT0hNyPuYhhJODDs2VBPp|17|27-542-634-4350|2890.48|ly final packages. silent depo +429|Supplier#000000429|6ITML8w7yXMd5wzp4xUYXX7rb|23|33-283-246-6716|2267.98|l ideas sleep. furiously ironic +430|Supplier#000000430|SMdrDaNv,2XyFOL6oVEfvH|1|11-541-204-3496|939.22|? silent pinto beans are abo +431|Supplier#000000431|WoeV,NINojE6hJjauAdrl5fGcdxX5JUPLnrim|3|13-269-548-1399|9477.34| according to the bravely quick dolphins. deposit +432|Supplier#000000432|ZkBaoMg9n7nXd1fyn |10|20-401-350-6978|3812.16|ven deposits sleep slyly after the blithely busy decoys. slyly ironic deposits +433|Supplier#000000433|At103qyX,VicINJGCOU51mQyfdYBB44Cg0S|14|24-141-780-8375|4988.55|r carefully according to the furiously regu +434|Supplier#000000434|endL6N 85uU0NemLv4L3mSEH4LT2BF|5|15-763-277-6054|9522.03|n, final packages. furiously pending c +435|Supplier#000000435|xKvU,V2SZj3OqEwdlgXs01K jSbJRjYYF|8|18-874-271-6733|6776.54|nic theodolites. blithely dogged accounts haggle furiously express pinto beans +436|Supplier#000000436|MV8Xu3m,93IINpPlE|20|30-253-200-6170|8396.49|. carefully regular request +437|Supplier#000000437|HuqIk0sK4yC6x5be,cTlPrFqqBCHYf|7|17-652-134-3031|9807.53|usly bold deposits about the pending +438|Supplier#000000438|cePboEvTZ6IfUAG 8asHxVbEmZnLSph9z01|24|34-502-705-5611|9291.35|hin deposits. blithely pending deposits sleep slyly. slyl +439|Supplier#000000439|dTnCcwPBKS J WRmt,Emi KnILcwcR9YOrdLuWD2|9|19-415-856-7881|1871.86|ithely ironic packages use special foxes. carefully even packages snooze quickly +440|Supplier#000000440|s4UNoE4WDs9vXLZFJjsCVxZ0W8XU YCgqGBxVX|15|25-475-341-4841|9290.23|uses. requests cajole among the quickly regular fo +441|Supplier#000000441|fvmSClCxNTIEspspva|14|24-252-393-5381|5008.40| the even, unusual ideas. slyly even plat +442|Supplier#000000442|PJSCHXMAsqyRr5aPD9lp4tUl1B1WytbosVY8EdNZ|11|21-463-951-7051|2257.13|es wake. accounts sleep slyly. slyly ironic platelets haggle. slyly +443|Supplier#000000443|nbs9cmnC63bi|10|20-238-345-8589|5569.82|sleep even dolphins. enticing deposits wake. furiously regular deposits acc +444|Supplier#000000444|mHr2VcUpRkvyQ9rjKMaPkeWbVZmEIhxhb8F|21|31-256-493-5644|-12.78|riously final requests sublate slyly. furiously ironic packages +445|Supplier#000000445|WqzJKhnLnF05It4 5TDkGkUwVYszIko|20|30-686-270-2150|3065.22|r the stealthy theodolites. bold, unusual attainmen +446|Supplier#000000446|na LX4kqDQbh|8|18-902-583-3116|2141.08|usual warhorses-- carefully unusual dep +447|Supplier#000000447|UHeJiRfImFw1r4MTrBk0vcwUx9|5|15-104-804-3139|2108.30|deas thrash blithely. unusual packages in +448|Supplier#000000448|cYzWCXDovaNR ,S3PICo3KYKAG3bYm0YKyqaZVu|24|34-817-553-5356|1209.30|e thinly slyly even warthogs; final asymptotes boost unusual pinto b +449|Supplier#000000449|kOYLYC4JQ5tBVlul15gdo6smU,VdIObtXyC|10|20-757-629-3940|5701.21|gular deposits are carefully about the furiously ir +450|Supplier#000000450|t3hxOMnv5AFdpM4|23|33-257-936-2585|8199.71|oss the gifts. final accounts cajole. sometimes special asymptotes are carefully along the package +451|Supplier#000000451|cqMKQiLjokvIFG|6|16-328-146-7253|2503.45|cial packages. pinto beans +452|Supplier#000000452|6bT4efJCWyxEtXmA1ZdwmqfrPGK|17|27-445-799-5245|9524.84|ions wake slyly alongside of the carefully ironic theo +453|Supplier#000000453|bpt98PxU5HSQt61bVB695JPjBmJKUv hNzQeHvC|21|31-882-179-6015|-905.25| quickly until the ironic accounts. pending requests doubt bl +454|Supplier#000000454|K8p1uXD3L,L|0|10-453-843-1585|7337.45|ronic requests haggle furiously furiously regular accounts. stealthy asymptotes sh +455|Supplier#000000455|de1QmawQjYipd|9|19-822-816-5632|8563.10|le. slyly even requests thrash blithely across the flu +456|Supplier#000000456|iSqve6KC7t69,jX6,HF B8Hbat11O|5|15-951-880-7133|5154.37|e along the slyly bold theodolites. packages detect. regular requests sleep furiously b +457|Supplier#000000457|Jld2rUj,Xw3u,lLq2EevCRQVYwSnkCT1K7nY1if|8|18-721-125-2213|2458.18|lyly across the slyly even courts. quickly silent +458|Supplier#000000458|IFNkUK1H53HwUHabiONkMFAUDb|21|31-318-754-9316|7654.94|blithely enticing deposits are. furiously final accounts about the regular requests h +459|Supplier#000000459|w12ixcTkh6AtG1LvWxX8El8Nf4vEGQiZrrpy|18|28-295-883-6516|9569.85|arefully even deposits. furiously final requests sleep. packages sleep. q +460|Supplier#000000460|lGEBJPLJaDwOhZpc7DQMY,PTUEv6BVBUsOGK0oF|20|30-851-458-4942|7619.85|ar theodolites use carefully about +461|Supplier#000000461|VLYMztlQim7tjPGSK0xPZXnb91a8,9wqVRwk62BP|22|32-897-799-8437|2922.33|foxes. ironic, ironic packages lose furiously regular accounts. carefully slow excu +462|Supplier#000000462|Bg,C2gIsljPAG|5|15-387-146-3147|9497.29|uriously unusual courts. blithely express deposits could run +463|Supplier#000000463|XOb4DatMUyqMuFM92ZRaapwsEQ|0|10-178-678-7353|9091.71|lay. deposits breach instructions. sl +464|Supplier#000000464|XwZyuXCVeO5wb5izvhfeX|5|15-653-204-7643|1448.94|ly. ideas are slyly after the ironic, regular accounts. platelets among t +465|Supplier#000000465|YOOuLmTfTFNFiipLtt iL7HQ fj lf0xOKDjnu|17|27-586-454-8554|7267.03|ly regular accounts nag. slyly regular deposits run furiously slyly final requests. accoun +466|Supplier#000000466|HEW3DIL,Aw0Ud|22|32-408-942-6024|2051.26|requests. closely final pinto beans according to the quickly ironic instructions breach b +467|Supplier#000000467|aibBbBkbtmDJ988LnMNkCAi|12|22-666-307-4101|-467.16|ackages. even, final dependencies sleep quickly. carefully regular deposits cajole furi +468|Supplier#000000468|T,pwE,skbYjr5DCAD2EfmEHNychqFKw1loF|6|16-494-568-3545|449.82|nusual dependencies. blithely even packages are blit +469|Supplier#000000469|G4Xq2 RFlLP7uDadWjZ96 uyaGNk8 216c6|9|19-527-692-4725|994.99|ely express foxes. carefully unusual packages nod furiously. blithely unusual pinto beans cajole at +470|Supplier#000000470|XckbzsAgBLbUkdfjgJEPjmUMTM8ebSMEvI|19|29-165-289-1523|727.89|gular excuses. furiously regular excuses sleep slyly caref +471|Supplier#000000471|Fc4 FkVkaA8zsUVr,bT3PcTko0n|12|22-925-324-7167|-414.45|hely ironic accounts. ironic dependencies sleep furiously about the bold requests-- q +472|Supplier#000000472|NlJV2a0ovbomfosgHUBx6sgT|20|30-976-134-3524|6238.12|to the quickly even deposits print slyly ironic requests. sp +473|Supplier#000000473|x1skh3uebekXL4BIKGgIGDUfTk CDn5FIJGaq2|15|25-658-329-5141|1094.74|old, unusual grouches. furiou +474|Supplier#000000474|USHBMdX8iFodU|0|10-327-319-7717|5226.21| wake. even pinto beans sleep quickly about the slyly special theodolites. courts +475|Supplier#000000475|xw4V6,4QQW LI5Qg EOKy4JD B4Cq1tjzaOma9Y|3|13-397-755-1516|-115.01|among the slyly regular deposits cajole after the even theodolites. carefully unusua +476|Supplier#000000476|ZvT qI2gMbh|0|10-219-531-3554|980.32|o the silent hockey players hang quickly around the furiously special theodolites. carefully bold d +477|Supplier#000000477|VtaNKN5Mqui5yh7j2ldd5waf|7|17-180-144-7991|7205.20| excuses wake express deposits. furiously careful asymptotes according to the carefull +478|Supplier#000000478|4jV maCw9SEt8jyLUsjej60bmMhP6zBv ajTk|11|21-580-334-3987|7901.42| regular asymptotes: fluffily unusual accounts integrate +479|Supplier#000000479|tdLkV2Ks0wBP1VlwdnPUxZnWADmxlbmRVE0a6h|18|28-851-500-5156|5283.98|s. blithely final asymptotes haggle fluffily. regular ideas +480|Supplier#000000480|q8,LH5UQiP3Tv60slOsFzX,HM0JPcwM0rD7eg d|14|24-645-644-2970|2927.68|ular deposits according to the furiously even asymptotes use among the bold deposits. quickly +481|Supplier#000000481|VqFS2DPW Ci2TpkfD|9|19-951-947-8520|539.24|refully stealthy instructions hang blithely ironic pinto beans. ironi +482|Supplier#000000482|LkVra4orMCs|14|24-516-940-6953|7978.18|ages. final ideas need to wake quickly fina +483|Supplier#000000483|ncWfnroE1n639qMrW8|2|12-443-228-5035|8366.89|refully regular ideas. furiously express theodolites across the sl +484|Supplier#000000484|WwSH FFzB2lViwrWli6Z4QVV AN1KH2G8|24|34-506-254-3252|7350.40|t the pending, even instructions. blithely +485|Supplier#000000485|ULR12B9vkQg4g0nFMaW|1|11-779-798-5195|6282.72|tegrate across the pending, special instructions. furiously regular di +486|Supplier#000000486|nCwPqLXZwjH20OVRy,fCQskTo3it2JHEGn7l|20|30-632-817-3179|2709.83|nusual, pending deposits wake blithely ironic deposits. +487|Supplier#000000487|BvDBcJa,jQPslM|5|15-551-730-1265|2124.86|ly about the requests. accounts could use blithely? furiously pending accounts nag regular, even ide +488|Supplier#000000488|AzfObar4VYwnQvsGbISGCshVM AIWYq9|13|23-609-606-3265|4546.13|ly ironic packages use quickly about the ironic theodolites! blithel +489|Supplier#000000489|y9NMoYGxDUPfrB1GwjYhLtCeV7pOt|10|20-375-500-2226|9836.43| quickly carefully pending accounts. fina +490|Supplier#000000490|JNqlktPWJ4|2|12-619-779-5325|10.55|ng to the packages. carefully final +491|Supplier#000000491|mTbDcJHQ7d|0|10-361-729-1693|-836.47| to the blithely even deposits. fluffily silent waters sleep blithely above th +492|Supplier#000000492|8wEulEYM zGvMXfDNNEw4B|14|24-875-296-5180|8368.06|y. slyly express deposits alongside of the accounts nag fluffily after the evenl +493|Supplier#000000493|7tdI3AtlDll57sj5K48WLX j5RDbc|11|21-252-702-2543|4999.17|gular foxes. slyly pending requests hang along +494|Supplier#000000494|6hAiQHDGTy6,8bjpxI i3f|13|23-200-629-1426|1080.57|pending packages. slyly even deposits wake closely. specia +495|Supplier#000000495|p086j79twIlC25BD6A|12|22-881-968-9019|9097.65|y regular theodolites shall cajole. requests cajole slyly +496|Supplier#000000496|be4auZxyqAgF5ysH3nXWcc7bDsNgdZ|20|30-810-880-3654|2266.32|theodolites. slyly regular +497|Supplier#000000497|iLAqlCKDLUGqHrjuOcId7 uYoTmpA|12|22-895-454-2151|5980.87|es nag about the furious +498|Supplier#000000498|4jvUQrC4acOQ82EFM vLNHG|17|27-772-853-6514|485.25|. slyly ironic ideas cajole slyly quickly ironic deposits. blithely even theodolites boo +499|Supplier#000000499|NbcQeBiDiN2tFiVxHIaWU03BVFIuxt |18|28-243-638-7646|2518.34|al, express deposits hang furiously. regular, unusual pinto beans wake a +500|Supplier#000000500|jLfNCVrj7X5h31yfSR02Z4x7K|20|30-229-226-6452|6738.72|ly. carefully final packages boost +501|Supplier#000000501|PSStC43vWlQQpmTku4s|24|34-141-983-6520|-264.89|s nag quickly. platelets haggle quickly above the furiously silent packages +502|Supplier#000000502|AutSetu5u6moXK6Y3rpoWREh|4|14-678-262-5636|963.33|al excuses haggle furiously iro +503|Supplier#000000503|OtYqMbur3v7nfzYgFYmMrJvq5YTj1MtXgefj|20|30-263-152-1630|7643.78|players are across the slyly silent requests: even, r +504|Supplier#000000504|P8k2mjRiRUFCJfxw7KrEdRpNNQPDxiI|10|20-322-544-5770|9050.12|y final pinto beans. blithely regular instructions wake abo +505|Supplier#000000505|aqcYZYQD5TYlLDgIxhKZyFCzL3Ch5qKOxj|9|19-480-691-1853|6399.78| requests engage slyly regular ideas. fina +506|Supplier#000000506|hnXNVHB1ao5rlGLkrS64kBz5C5rx7 R4dqO5CNv|23|33-582-741-7991|1202.98|riously even accounts sleep furiously ironic foxes. quickly final requests haggle fu +507|Supplier#000000507|aF2w4JF8qV aaqApYqzTFtIXtKV57Na|9|19-981-569-8699|9464.26|p carefully besides the furiou +508|Supplier#000000508|F9,suuHYbe6kCRCPZaeSHSPAFBk9vOcFX8TUx|14|24-179-400-2422|3878.22|sits. blithely furious requests boost slyly about the quickly even packages. closely +509|Supplier#000000509|SF7dR8V5pK|6|16-298-154-3365|4315.15|ronic orbits are furiously across the requests. quickly express ideas across the special, bold +510|Supplier#000000510|VmXQl ,vY8JiEseo8Mv4zscvNCfsY|19|29-207-852-3454|-314.06| bold deposits. carefully even d +511|Supplier#000000511|RWNdTEe,VJFarN4Pu7Xl|23|33-476-213-8045|-455.12|t the quickly even deposits. carefully careful ideas sleep slyly bold deposits. unusual, even accoun +512|Supplier#000000512|MoNOuMC4QMOnBgD5uLcvtHCcfzf9cW|15|25-617-226-9364|4762.66|special accounts. daring foxes nag quickly silent, special packages. silent, unusual a +513|Supplier#000000513|YMhcTaVkhw0nO9B,|4|14-481-495-8505|8075.30| final requests. slyly final reque +514|Supplier#000000514|Q4ErX,NN,Z2UDP|14|24-677-367-2786|1669.85|arefully regular ideas: pinto beans detect across the slyly pending pinto b +515|Supplier#000000515|5TOuZXAb9df7m3BCW2 TeHMY1Zdf46DqpT2,0t|17|27-470-220-5233|2127.89| beans sleep after the final frays. special ideas across the carefully regular instructi +516|Supplier#000000516|z5Mm65PAP4m|5|15-165-647-2301|371.38|s. idly final theodolites ha +517|Supplier#000000517|NkRQYLe9d8vEXNO KKk8rxK502OI2 |8|18-577-641-3805|2781.03|ing asymptotes. carefully fin +518|Supplier#000000518|g2buDv7WzbkNDU63IN5af0i6SAdUxihirS2X|1|11-853-939-1266|-199.77|egular dependencies integrate carefully among the enticingly f +519|Supplier#000000519|0zgIDeAmk5976RzKiXZi,kobff8IxQn|19|29-133-225-9811|6361.20|onic requests cajole blithely furiously pending ideas. quickly quick accounts sle +520|Supplier#000000520|RHuqyeshPnOa6gwEiV3zDhP0o2aYvZ9glQURu1w|12|22-654-808-2429|1682.84|aggle among the final, pending realms. carefully regular escapades woul +521|Supplier#000000521|jQ648xqiuJaHLQjwrq5b|1|11-887-652-3799|507.99|carefully final asymptotes: carefully regular epitaphs about the blithely u +522|Supplier#000000522|joLGRuiXIsVWk|8|18-355-956-2843|-336.14|tect slyly final instructions. fluffily ironic ideas after the final, fina +523|Supplier#000000523|zzfDhdtZcvmVzA8rNFU,Yctj1zBN|18|28-458-231-8410|2492.06|e, regular deposits eat. fluffily express foxes haggle a +524|Supplier#000000524|rTYNcqhFosNU,4|13|23-993-423-3788|5093.35|c theodolites integrate quickly ironic deposits. furiously even de +525|Supplier#000000525|GtHZ9ooyeKe|11|21-247-956-8546|-79.52|y bold theodolites; express requests across the packages haggle slyly carefully final pinto b +526|Supplier#000000526|x8r7,jgpBRjq6Ns|5|15-539-980-8303|3477.18|as kindle ideas. blithely bold requests +527|Supplier#000000527|udKczd6U1Bm79UVDkA8P2Xa1VY qv9mvsXo|18|28-894-961-1780|6938.43| sublate blithely pending instructions. blithely final packages nag blithe +528|Supplier#000000528|QsnGjo7irxCIbN3|2|12-961-772-2408|1285.81|the furiously final tithes are across the pending ideas. car +529|Supplier#000000529|Fj4vm8y2 Tlgd|10|20-521-276-3787|4581.80|wake across the furiously regular excuses. express dolphins are q +530|Supplier#000000530|0qwCMwobKY OcmLyfRXlagA8ukENJv,|3|13-747-781-9694|4327.86|fily pending dependencies wake slyly final deposits. platelets against the slyly ironic requests na +531|Supplier#000000531|fN0ix827c112YajETqxxuofrfl7v VndWB38n|22|32-950-377-4573|4258.42| the carefully ironic gifts. carefully unusual multipliers sleep slyly amo +532|Supplier#000000532|ep92hT7VLaVlDKM7lgbj02kIL|3|13-595-401-8233|3278.71| fluffily fluffily express ideas; blithely special instructions wake quickly among th +533|Supplier#000000533|WF9wtTxzbBa4kv FAeer9I1pQJ0Qe,uJf3f w|7|17-404-617-2581|4213.95|e ironic foxes believe fluffily across the u +534|Supplier#000000534|JBhF3gZcQiNWGxh8DuoAhmVi|9|19-198-519-8383|3930.79|after the furiously even requests haggle thinly a +535|Supplier#000000535|aiq9Honllr6hFt, YJ6|10|20-121-889-4500|7839.46| foxes. carefully thin dependencies sublate furiously. regular instruction +536|Supplier#000000536|dzf PbgzKpWBDim5S1BSPLBNzxFpxZNUE|17|27-749-678-1361|8797.40|cajole blithely slyly even dependencies. carefu +537|Supplier#000000537|KsYYPIw2kWP|13|23-671-207-6720|5046.81| forges breach upon the bold ideas. final foxes nag frets. final instructions eat fluffily +538|Supplier#000000538|KBZ0RSDGTVJQPbWaU6x|17|27-799-369-5739|9669.24|e fluffily regular theodolites. special packages are into the careful +539|Supplier#000000539|GG5N3GIdNmmvhKs52Y|1|11-844-496-3836|5976.60|otes; carefully ironic deposits sleep idly along +540|Supplier#000000540|Tmyis ,xX7XjU2E|16|26-891-481-8993|5704.81|. blithely final instructions shall cajol +541|Supplier#000000541|Nxggufcm ,hR|17|27-914-557-6989|2830.62|blithely ironic accounts poach blithely alongside of the carefu +542|Supplier#000000542|3yWRklEDbAvfVuidQPgOOe,x7f,4 J5lSp4v|5|15-378-700-5884|8142.81|g requests believe carefully dolphins. quickly pending +543|Supplier#000000543|P10rl2 o A0jtJQDcB|15|25-433-303-6328|3222.71| against the carefully ironic excuses boost bli +544|Supplier#000000544|pUS3drDXbPeNqvI kUNlnz5GDSU5,aLI|1|11-471-707-9878|9042.70| sleep ironic accounts. fluffily even dependen +545|Supplier#000000545|D8SE9UgTdgq3oNH8RzaxDKpbsA BlRKsf|2|12-876-408-4086|2723.99|onic packages use slyly about the theodolites. final platelets are. finall +546|Supplier#000000546|YznQGwWvZbNA0O9ZV|22|32-679-789-3863|9036.47|es haggle. blithely final theodolites wake blithely. carefully regular packages +547|Supplier#000000547|MaV373lvwj|8|18-600-279-8954|7937.31| the quickly ironic asymptotes nag carefully +548|Supplier#000000548|btRiQsq qEK0qg0T|12|22-790-987-6339|3456.36|ly even tithes sleep alongside of the asymptotes. blithely ironic requests are +549|Supplier#000000549|oy89mLRUwTVCoU|0|10-377-880-8280|8948.84|iously final ideas. carefully unusual ideas wake fluffily special platelets. furiously unusual pin +550|Supplier#000000550|QQavssDXnYHbvOrg|12|22-648-743-9295|9238.79|en, bold ideas. ironic, unusual deposits boost carefully quick accounts. slyly e +551|Supplier#000000551|ZNiqP1w6Z SGZsLllIhaicTnLCCuAepdNbkm6pJ|8|18-297-775-8421|9364.67|lithely even instructions poach quickly. furiously bold accounts sleep final, final accoun +552|Supplier#000000552|JP3ebtH5epPwpU2lVQqKNZC3NTvnTlrhLy5eeGN|12|22-861-905-2401|2114.99|uickly about the deposits. furiously pending accounts wake. packages are slyly bold de +553|Supplier#000000553|a,liVofXbCJ|6|16-599-552-3755|3526.53|lar dinos nag slyly brave +554|Supplier#000000554|FAEEZyNtDGEhZ|7|17-365-531-4630|335.69|enticing accounts against the deposits use q +555|Supplier#000000555|TfB,a5bfl3Ah 3Z 74GqnNs6zKVGM|3|13-607-927-7514|7896.01|uriously regular theodolites according to the carefully f +556|Supplier#000000556|g3QRUaiDAI1nQQPJLJfAa9W|6|16-951-842-4482|2253.90| silent deposits haggle quickly ironic, final theodolites. boldly bold accou +557|Supplier#000000557|jj0wUYh9K3fG5Jhdhrkuy ,4|3|13-704-788-7706|6343.15|ckages. unusual, regular attainments +558|Supplier#000000558|T9hNjrRRRQmkbZomdaeLKDOqmmUcJpAJzXOxq|9|19-494-404-8664|6404.51|s against the carefully pending packages cajole al +559|Supplier#000000559|SkKThClbkbH8mIv|11|21-205-567-6566|3646.46| ideas cajole alongside of the carefully ironic packages. regular pint +560|Supplier#000000560|rYTPbri8qJ49rRfFmChtnDIQ|17|27-261-214-5284|3009.57|slow platelets. quickly pending ideas are requests. even theodolites may nag about the regular, +561|Supplier#000000561|1V3DMQWQpfjPJybZYAP|22|32-547-343-1231|1824.47|d packages. carefully bold ideas are quickly across the platelets. final, express pinto b +562|Supplier#000000562|8TXCtnRQzByqjie|23|33-782-496-5965|4848.52|he furiously special accounts hag +563|Supplier#000000563|Rc7U1cRUhYs03JD|7|17-108-537-2691|-942.73|slyly furiously final decoys; silent, special realms poach f +564|Supplier#000000564|IufyqhG4fmo VkgQT w BF4|2|12-702-995-1506|4410.70|epitaphs. even attainments cajole slyly regular packages. final deposits cajole. furiously final +565|Supplier#000000565|,oYB9wlD3mtL lj3PJC67a RGXaqh69sHK5G4e|20|30-487-989-9411|-334.52|s haggle never furiously special deposits. final attainments breach special pains. fl +566|Supplier#000000566|hBRvnhCUVIiXQK6dyAZYN,TNZItOlBvsVYCisb,O|24|34-396-766-5348|7705.73|rs! special packages cajole-- furiously final packages maintain slyly around the blithely spe +567|Supplier#000000567|fvuRpAap0MvoBguGKBfp|1|11-390-878-2811|5264.91|ke fluffily furiously ironic ideas. qu +568|Supplier#000000568|z70Hj8qVi8jQu|10|20-162-593-6831|1367.90|furiously fluffy instructions about the regular +569|Supplier#000000569|jjFjVCjK91yy2B dj|15|25-803-734-8127|9166.95|al frays until the slyly ironic requests cajole furiously about the quietly final foxes. furiously p +570|Supplier#000000570|8VkNpvXFgKgbKY2ypMKyIOBlK|17|27-658-225-4655|922.72|! regular platelets sleep furiously idly silent foxes. even courts sleep slyly. regular, reg +571|Supplier#000000571|GMifmfVJba|9|19-223-236-6710|7132.44|nto beans haggle carefully after the furiously regular reques +572|Supplier#000000572|J,RTwd9mNOTralFovrCrH99 f9rbvsDf3|19|29-444-247-7800|6039.27|its. carefully even requests along the quickly unusual pinto beans inte +573|Supplier#000000573|83GRV1s,yGFwl1NClLSXnJVRlh0xS8YW8|12|22-306-652-6853|3107.46|icing accounts. carefully regular sauternes according to the accounts wake after the fina +574|Supplier#000000574|2O8 sy9g2mlBOuEjzj0pA2pevk,|22|32-866-246-8752|8096.98|ully after the regular requests. slyly final dependencies wake slyly along the busy deposit +575|Supplier#000000575|J24LOV AQiHuYuR|19|29-201-935-5958|2918.54| special packages along the carefully e +576|Supplier#000000576|Plc2DypORn4qNOTpZ|12|22-316-723-5789|817.26|e of the final deposits. regular, unusual requests wake slyly. furio +577|Supplier#000000577|kn5oGAnFD1CQjet8awWorC,UMf37MP71yNcVD|13|23-973-363-7797|5593.17|olites along the quick accounts cajole throughout the regular asymptotes. accounts maintain +578|Supplier#000000578|bn5J0A4426DpcW7m rQ9,qxqJ1KN|20|30-105-334-1726|7428.76|carefully about the slyly regular warthogs. special packages above the regular pa +579|Supplier#000000579|U6sejT6kSPi5p1FUcUxjdJ|19|29-854-341-3857|3144.98|ly regular pinto beans. furiously regular ideas against the accounts nag blithely final pinto b +580|Supplier#000000580|MuRScZH74veaM2|6|16-732-277-6239|614.57|packages. furiously final theodolites integrate according to the carefully silent braids. caref +581|Supplier#000000581|X pNyEcNqxYwiP0gJ7FzkJ,haGkn|18|28-320-345-9799|8998.40|cross the quick packages wake carefully except the accounts? +582|Supplier#000000582|o6h0Bjjwags0FDRSAoKJPCWyt|4|14-341-851-2277|6178.48|ly unusual packages. regular, pending foxes are blithely. fluffily +583|Supplier#000000583|9st8mjB5G7J|15|25-760-126-2928|-339.15| dolphins across the carefully regular instructions hagg +584|Supplier#000000584|XvDYsHYpmY5AkX60fj0bZo4WW|18|28-223-704-2186|6912.86|e requests haggle carefully even ideas. express, bold requests integrate quickly furiously +585|Supplier#000000585|DQZTWEfNYL9UDlMqcQAEThcPdbyD45PYzL|23|33-357-931-8857|433.74|ar, silent instructions i +586|Supplier#000000586|9tfHwYyFe2t2,6pAVpkURXAxtc2cQw4qfGKYJ|2|12-747-610-3099|5850.91|ccording to the stealthily ironi +587|Supplier#000000587|58,gb EuMperMCg2lv XUQ9vi4GzhO2a|7|17-128-699-9949|5322.35|thin pinto beans boost silently. ruthless deposits haggle quickly above the slyly unusual th +588|Supplier#000000588|e3yF5zmSj y81I|14|24-180-601-5741|9760.06|gular, permanent accounts. +589|Supplier#000000589|3C4,WjUCjL59QhMSxyq1|18|28-878-356-5116|3415.90|apades are final, unusual instructions. bold, unusual +590|Supplier#000000590|KhRtsL4Foycp2hUwg bEHkDusXAf|9|19-277-247-1833|-12.84|de of the express requests. pinto beans are +591|Supplier#000000591|iXlVA9y6oX4|24|34-204-742-6291|2082.42|ven instructions try to are slyly about the quickly b +592|Supplier#000000592|tdYqh7rm0Zc7E0etRqHakcg,m34gQX|9|19-220-707-3861|6151.79|sual, express accounts integrate fluffily. dependencies cajole slyly +593|Supplier#000000593|qvlFqgoEMzzksE2uQlchYQ8V|6|16-262-671-5187|2214.36|arefully even ideas sleep quickly. ironic foxes wak +594|Supplier#000000594|8GY0oRK64AFmY7pys51Uqm7YbMn9luO,Z|17|27-826-454-6643|1760.34|quests use fluffily quickly final packages. carefully pending pinto beans are blithely among the ca +595|Supplier#000000595| CURZCs4l306M2ir8rFkgeYVg|24|34-354-570-3604|4922.60|ecial instructions cajole alongside of the requests. i +596|Supplier#000000596|caTnKVKTsCHNEVi1xVPD|4|14-761-106-2656|1180.93|its sleep. carefully unusual somas use furiously above the +597|Supplier#000000597|CKt5G XZ5DBt|24|34-219-790-3864|1828.73|ecoys sleep slyly according to the furiously regular requests. furiously expres +598|Supplier#000000598|p9AGBjg4DZuChQbY8gAj3LtMrxpOWqMpJR|7|17-985-962-2292|-590.83|uickly unusual ideas sleep blithely after the +599|Supplier#000000599|R u1wkvs4 B0wlbPbT8WrSzqoXZG0CjbsuAbPpUs|4|14-208-385-3654|769.29|affix. carefully final accounts about the care +600|Supplier#000000600|YHyUzea88sXoNmqmCMamiEfGC54xpdX|12|22-508-410-2758|2342.35|s. fluffily ironic deposits hinder furiousl +601|Supplier#000000601|TS2xMrQuUs9VrgWt4,gpdcEyWNw3K6,P|8|18-528-362-8573|7906.22|atelets cajole according to the pending, ironic orbits. carefully regular packa +602|Supplier#000000602|xEtByOs0Pydp9y75MSgoy6T R6PT8e|20|30-106-955-5651|8924.02|tes. furiously careful pains are. quickly even platelets boost sly, +603|Supplier#000000603|mECtpm1pmMnqK4K0DLZ5Gtkj 5bUydzBak6|12|22-807-182-2059|-846.12|dependencies. slyly regular accounts +604|Supplier#000000604|B53WjrwJCSh14Bx,oCEinGgCJ3ZCc8m|24|34-390-848-6584|227.59|regular asymptotes solve accordin +605|Supplier#000000605|wdwiNoNT8pVHOTHQ8jhVzaOTkU|6|16-835-870-9488|6071.58|foxes poach blithely beneath the excuses: ironic multipliers haggle quickly furiously unu +606|Supplier#000000606|n,iOFy5X,4GFeXNrCCKBmHucz1|19|29-856-255-1441|6988.38|es haggle across the carefully even accounts: unusual instructions x-ray carefully. blit +607|Supplier#000000607|vgEaPkxAonSSdAUn,7usQ c4G3Ho2r0|15|25-370-994-3762|4667.27|ests are closely quickly ironic orbits. carefully regular attainments cajole furiousl +608|Supplier#000000608|SQ,f89cn6x6g|17|27-435-165-2250|-210.13| do was furiously above the accounts. unusual, ironic packages hang about the carefully final +609|Supplier#000000609|n9 nkdqilT|12|22-852-519-5068|8287.95|ronic, regular ideas nag furiously across the final ideas. bold, express do +610|Supplier#000000610|cRikc,rgxAM3yz0IR85OD|20|30-402-585-4900|668.12|gainst the ideas. regular instructions are. theodolites cajole furiously final, un +611|Supplier#000000611| 6BVljZ1HeradVcmcDm90NYkla3iHPBsTSoUJr|8|18-554-185-6487|7048.30|al courts sleep carefully about the blithely express accounts. fluffily even request +612|Supplier#000000612|gt9T2nnuWBiy5zcrWG2iSdZt,sAEYnD6|23|33-377-769-8060|-118.86| regular requests after the slyly regular packages belie +613|Supplier#000000613|DYwZjMQj26Es8D8pxn2zx|11|21-796-340-9401|2201.94|e furiously. final foxes haggle carefully quickly express theodolites. regular deposits affix bli +614|Supplier#000000614|DteCEt557XpSo8CejUUbFm RgTeT4FRz7bC,6l|14|24-185-488-4015|1956.55|inal theodolites shall have to boost. unusual theodolites are +615|Supplier#000000615|dIT3WOBBwUuakVwd965N4logoVW1A|23|33-196-233-6474|2741.54|platelets. pending, regular +616|Supplier#000000616|Ktao GA3 5k7oF,wkDyhc0uatR72dD65pD|6|16-738-270-6883|7119.71|al packages are carefully after the regular p +617|Supplier#000000617|Q4haZeO8aVzl2dXziDw3f|7|17-563-347-4748|1002.43| blithely unusual theodolites haggle furiously. even courts use quickly against the +618|Supplier#000000618|mPrv5o5d22wyXUgUw69x8m dtx7I3mMh|4|14-912-871-9422|4710.51| the furiously pending deposits x-ray about the fluffily unusual accounts +619|Supplier#000000619|CzILYciyb3fdioa9LflK,ADrP|17|27-505-962-3048|559.50|ironic asymptotes. express, final pinto beans are furiously inside the furiously regu +620|Supplier#000000620|5pd GQ2NTM3c2uR,gCg9NspSE|24|34-347-881-4300|7516.12|posits promise. quickly express accounts according to the regularly pending accounts lose blithely +621|Supplier#000000621|fjFomMNvcBWHb|6|16-492-530-5790|8436.37|ns. even, even platelets up the carefully pending platelets u +622|Supplier#000000622|gCQimU1jYHoQiglDmW1FkQM9wzi YC1P15pMy1|21|31-421-544-2948|9199.28|ent instructions. furiously silent packages detect regularly quickly even somas. even pearls ha +623|Supplier#000000623|dSSQ3dTYwThbLppbetVUeuPfBIUF|7|17-593-337-7365|5408.07|ial frays use. carefully special foxes wake carefully slyly pending deposits-- final requests a +624|Supplier#000000624|JlCK4aBP3PCO|1|11-487-571-9291|9497.65|the silent, final pinto be +625|Supplier#000000625|0zW5d Hyogg0z,sXxl1PHS0Ya,muKs4N dS7|23|33-946-363-3870|4065.25|unts. silent accounts around the quickly final r +626|Supplier#000000626|uaYN3Mg6sVtD|13|23-672-133-9305|1586.50|ic deposits above the blit +627|Supplier#000000627|k6W51QENyVPe4 6z0ajEMTkEA|11|21-632-574-8701|9318.43|l instructions serve slyly regular deposits. carefully busy excuses cajole quickly pending d +628|Supplier#000000628|Gk75kOa26bzFvztn3rgkiOdL M0jTMU|0|10-599-740-9848|5975.00|ccounts play along the special ideas. blithely final deposi +629|Supplier#000000629|SjPnzOlaF3,4u1QAyH1l57EnL,h1IgnmoG|7|17-533-560-8817|856.17| furiously ironic requests by the furiously regular accounts wake slyly across the +630|Supplier#000000630|6hLBs3Rnd5elLLVv1i p3A2U6G1dkIApKDkiCy|23|33-522-267-9970|-418.50|kly bold notornis; idly even forges wake furiously quickly special pinto bea +631|Supplier#000000631|F1uLHPh2tHB6j1YLrB,vilZ5SIn6P1RFTPoz|15|25-354-834-6526|4127.62|egular, ironic packages. slyly fin +632|Supplier#000000632|pSP0SwLWVUKjWyzZtT|2|12-953-253-6464|4975.09|luffily even warhorses. carefully special requests are furious +633|Supplier#000000633|HhHzqIEl0jP8SQYZ7EybSWCtRFhmytByO7CPNZ|20|30-502-537-4154|3322.37|ent accounts mold. blithely unusual packages wake. furiously f +634|Supplier#000000634|hS62vraooyHWnMKyZV3f1GSPeKJ,7uRK6M5|23|33-105-608-2902|1133.80|equests affix around the blithely special theodolites. unusual accounts wake. pend +635|Supplier#000000635|JNDTs06uwtXvRZUWQVpDgAz|10|20-119-524-2053|1739.90|s. packages wake after the slyly ironic frets; quickly pending reque +636|Supplier#000000636|Kc rcRwa,q,TQx1W 3fu|18|28-357-934-4951|2408.11| wake fluffily above the slyly final ideas. silent instructions wake carefully: blithely silent d +637|Supplier#000000637|V6AMGzXQ7Eqs|5|15-832-253-5581|2002.17|rses haggle blithely about the carefully silent deposits. slyly pending packages th +638|Supplier#000000638|YfXfPM0,m6CdwYYiQjmy9dcN|8|18-147-424-5181|-614.31|quickly unusual instructions would wake carefully. slyly ironic request +639|Supplier#000000639|WGqnQRU1xoC,UV9xDGjc48rC4Cow4|9|19-128-575-2303|2172.39|sual theodolites. slyly even accounts according to the quickly special accounts are f +640|Supplier#000000640|mvvtlQKsTOsJj5Ihk7,cq|3|13-758-222-1059|281.90|nic accounts sleep daringly at t +641|Supplier#000000641| 0L8yoIwSCP4EJyESI 6bVH9k|17|27-393-904-4536|5749.43|l accounts use furiously against the fluffi +642|Supplier#000000642|eldBmBVYRbcB YfMRBlNyuQe8k0zYK,v Obk|11|21-775-952-5836|4622.19|reach carefully against the final, pending instructions-- slyly +643|Supplier#000000643|mJN4aN B Lxz2esIAW0GoxEw1rAU|18|28-782-409-7844|1218.59|gular requests. even, pending notornis thrash fluffily against the enticingly regular i +644|Supplier#000000644|70mM6 QN882bcuY|10|20-367-561-9783|7783.86|regular accounts. quickly final theodolites sl +645|Supplier#000000645|blbALMoy2OihlkvD7FtkLXPvTShY9JVtsWT|7|17-742-832-9508|9459.29|accounts. blithely unusual pinto beans at the blithely +646|Supplier#000000646|IUzsmT,2oBgjhWP2TlXTL6IkJH,4h,1SJRt|6|16-601-220-5489|8430.52|ites among the always final ideas kindle according to the theodolites. notornis in +647|Supplier#000000647|x5U7MBZmwfG9|23|33-258-202-4782|9828.21|s the slyly even ideas poach fluffily +648|Supplier#000000648|0RXVM8t80LLWl|20|30-526-602-8400|4877.49|aggle daring instructions. furiously final deposits detect furio +649|Supplier#000000649|8sfoyPTvZbFMXC93ti9qSI6dYN0QuXh3wO|7|17-341-611-2596|1927.21|equests. ironic dependencies are quickly slyl +650|Supplier#000000650|lqBJUDL9EXwh0|4|14-980-933-9338|4624.13|ons are. unusual, pending foxes affi +651|Supplier#000000651|oWekiBV6s,1g|22|32-181-426-4490|683.07|ly regular requests cajole abou +652|Supplier#000000652|Cwyzz7 uW9mWq|12|22-957-225-7894|5235.95|ly fluffily ironic realms. slyly even accounts sleep slyly. carefully even packag +653|Supplier#000000653|IK8OvngBYI1zh9bbK0vLThzVvk7F69hxytOmq|17|27-391-635-2412|9584.63|ar foxes cajole about the quietly final pinto beans. ex +654|Supplier#000000654|T96kVu5bsOeH6|7|17-361-437-5840|2997.61|ial ideas haggle carefully according to the carefully express accounts. ironic accounts pri +655|Supplier#000000655|j8ga9M1KhzXKSk6g,bXi0zbLWjckDEpwBeqs|4|14-708-916-3581|9745.28|stealthily slyly special deposits. final packages behind the regular requests na +656|Supplier#000000656|mQXqRMgstvOI|19|29-633-362-8481|8069.74|ronic packages integrate. even excuses integrate carefully ruthlessly bold packages. regular ideas a +657|Supplier#000000657|nas2fhRwM97W8EEqYpBN|3|13-546-747-5121|7182.24|es wake above the ironic instructi +658|Supplier#000000658|kw9Ts9s38tEPf,IKi6kXNEv02|22|32-991-641-3699|6888.65|ular notornis integrate. permanently final accounts wake final +659|Supplier#000000659|jjVP5XCtV9jbvUnkoeFUTrgB,ke|20|30-917-437-7814|631.86|r, ironic requests. carefully ruthless theodolites across the bravely bold deposits cajole car +660|Supplier#000000660|AmvX 3nxd9r EOYZErE6PvBtFx |20|30-126-295-9200|2555.98|longside of the requests. fluffily bold +661|Supplier#000000661|lZGFurTW1snIqk0oLWMMaeq3L|13|23-401-253-9405|1333.75|s detect quickly. blithely ironic dugouts maintain furiously ironic +662|Supplier#000000662|geJEMlJvE3HdW96Rz3touARh|18|28-497-129-7855|7337.00|the blithely ironic ideas use qui +663|Supplier#000000663|tPtpVTsSAQNw,4GgXR2Hxx5FyCxxgqmW,jE sRT|18|28-798-122-1574|956.88|osits. quickly quiet requests cajole against the slyly regular accounts. ironi +664|Supplier#000000664|ln6wISAnC8Bpj q4V|4|14-244-772-4913|9261.13|ly special foxes cajole slyly ironic reque +665|Supplier#000000665|n4JVAxZUnvT5dVZBK,3CIIDoa|12|22-273-991-9361|-197.51|nding theodolites. fluffily final packages wake? idle req +666|Supplier#000000666|7emVs,4gxuqP95JNK|19|29-330-510-9985|433.93|ously ironic requests haggle. deposits amo +667|Supplier#000000667|La6cVlSLCZZDhhX9FtKsRlylP,,lI3IYjHT8yJJX|23|33-382-268-5150|9009.30|ular accounts after the fluffily pending accounts are according to the +668|Supplier#000000668|lLpUAYxvq5Gu9eLRdlrj|8|18-876-287-5756|2317.52|x quietly among the braids. blithely final asymptotes would a +669|Supplier#000000669|,Csubly4KD59igxGYacW2q7jUvQ4ZaOKQC|20|30-256-757-7811|8577.07| pinto beans nag after the slyly final packages. final requests among the furiousl +670|Supplier#000000670|z2NRPuuVeu1jmuuyzoMOlpoCI6P|20|30-364-864-1141|8887.18|arefully fluffily even pinto beans. pinto bean +671|Supplier#000000671|VlDv51ScrQCe1eVVnz S4Kq5wFZUKJd PyBI|8|18-730-953-4689|6935.18| busy dependencies sleep blithely after the ironic, iron +672|Supplier#000000672|iu9d66fGNBYX|11|21-299-539-7383|1594.77|uickly carefully express foxes. ironic requests cajole about the requests. unusual acco +673|Supplier#000000673|GCmswucbTQe2Q3OHcnsNI|20|30-592-284-5403|5335.98|y ruthless requests. furiously regular accounts wake after +674|Supplier#000000674|jMxLRDxoP1Pf kzzyMVIfLB|6|16-128-338-8014|7822.90|thely after the furiously even pains. quietly +675|Supplier#000000675|pbDwRMZ6nNUpcFirCvv|15|25-499-280-9384|5579.98| bold deposits. regular, regular pinto +676|Supplier#000000676|USGIdhKusoe8dcvWdBbZWUfxnVxNnsgY mG|9|19-833-604-9178|5783.61|s use deposits. quickly even packages haggle quickl +677|Supplier#000000677|8mhrffG7D2WJBSQbOGstQ|13|23-290-639-3315|7128.81|nder blithely. slyly unusual theod +678|Supplier#000000678|SLpBfeoHSImv|1|11-465-565-3513|-58.41|he blithely even requests. blithely unusual theodolites sleep furiously against the +679|Supplier#000000679|qLzdFRbVDeEH|4|14-771-110-7666|6291.34|ole slyly against the furiously silent instructions; ironic instruc +680|Supplier#000000680|UhvDfdEfJh,Qbe7VZb8uSGO2TU 0jEa6nXZXE|22|32-522-382-1620|4586.49| the regularly regular dependencies. carefully bold excuses under th +681|Supplier#000000681|ArTNWD5g1KfANCMFDfk83TUlX|12|22-208-790-6946|6159.40|. carefully special ideas promise slyly foxes-- pending accounts about the furiously special the +682|Supplier#000000682|4bD4f4zKh88YutGs|9|19-577-707-8772|9127.14|eep carefully above the slyly final requests. carefully express foxes nag +683|Supplier#000000683|W0rFJpyes6atCIuwAmktnK|0|10-108-564-6160|2956.02|uests. platelets breach blithely among the furiously regular requests. quickly fin +684|Supplier#000000684|nqw,GGxCoNZ3UOuIa0edX3SdoYKER|19|29-345-334-1955|3810.81|sts are slyly. doggedly final warhorses wake carefully after the deposits. reg +685|Supplier#000000685|JgoYDMLdJeM|12|22-599-473-1489|4297.36|o the furiously final braids. ironic requests sleep among the even foxes. regula +686|Supplier#000000686|LxjyC4i3RxAqWnUF|9|19-818-456-6713|8724.42| requests haggle carefully. silent, ironic accounts along +687|Supplier#000000687|PN0ZGBcv2F7yzeOMZflOAoEnhAUMPocj6sc|20|30-168-842-6668|-624.22|usly. regular theodolites along the careful +688|Supplier#000000688|D fw5ocppmZpYBBIPI718hCihLDZ5KhKX|3|13-855-777-3804|602.75|after the busy platelets serve across the even packages. final pinto bean +689|Supplier#000000689|v8MJCOfDDFgJbxjwgtdKLtlZRKlSRGl|2|12-934-814-6084|8436.92|y final deposits. blithely unusual accounts along +690|Supplier#000000690|nK6Lv WWUh59jE525|19|29-330-952-4018|7448.46|nic pinto beans doubt blithely b +691|Supplier#000000691|XU1STWHllW5I5Rw9X,jsZi7X7M 4|12|22-930-512-3497|4239.95|to beans nag around the careful accounts. ideas integrate: daringly ironic pack +692|Supplier#000000692|K8M3uIAEsKuFGIc43sALPKCDSyKXtc0w VcdS|0|10-727-704-5789|845.01|ly ironic packages. excuses +693|Supplier#000000693|S,mnHfsroFOVieQGdcaY5eod,8Zmji8|8|18-231-996-9225|9956.55|wake quickly around the foxes. +694|Supplier#000000694|20i8TNU3K6H0SEL20|10|20-902-352-7633|5776.60|ounts. regular requests are ca +695|Supplier#000000695|xhSCyzMl iQ|21|31-274-635-2607|-590.99|cial asymptotes across the slyly unusual foxes use slyly furiousl +696|Supplier#000000696|hWvK 9N1EQX0kjEI|0|10-745-572-7198|9114.26|regular packages wake slyly after the carefully silent dependencies. packages dou +697|Supplier#000000697|CnQUnxL9Jk1ew4 kK,DqzwwV34c1KUiu4xFVsG|12|22-673-286-5547|6463.10| furious frets. furiously even accounts should affix furiously blithely final theo +698|Supplier#000000698|ciim3Adyrh1gqQlOsw0YYeF9gIfUM|11|21-373-751-4459|9356.72|y ruthless pinto beans. slyly final pinto bea +699|Supplier#000000699|CplgysgQzKm7KRFKOJe|4|14-247-404-4838|453.28|sts. express accounts boost. silent platelets boost fl +700|Supplier#000000700|K5l3kvvjnRQJJ,|5|15-648-846-4789|4196.26|es haggle quickly. slyly bold ideas serve at the regular attai +701|Supplier#000000701|ijyXEKJPjoVzpXY9g|0|10-713-854-3832|3513.22|ously ironic accounts nag quickly. car +702|Supplier#000000702|1IfvvCrOk6tDle1AjJisjgmZTSrf6Y 2t,Mdv|9|19-354-412-3179|7655.97|about the unusual, bold foxes. quickl +703|Supplier#000000703|QQ Z27PMXZP|13|23-876-543-9729|4921.83|ar patterns sleep about the instructions. p +704|Supplier#000000704|hQvlBqbqqnA5Dgo1BffRBX78tkkRu|19|29-300-896-5991|-845.44|ctions. carefully sly requ +705|Supplier#000000705|9up,Z78TUVPrp2QdumA8fRjL8PG5H6PG|15|25-673-680-4438|571.85|sual packages. carefully ironic reques +706|Supplier#000000706|oXF4XhSiEJMIJouBmMjLZ|11|21-513-570-8754|6043.19|uickly. special, ironic attainments cajole carefully according +707|Supplier#000000707|gIbGXDVlfL3Zl4dmtnAKrnoO|0|10-971-806-9591|2858.06|counts haggle ruthlessly bold deposits. furiously regular instructions wake near the furi +708|Supplier#000000708|qGdOm1xZczyifQ ,Ba2ptq2L7i2K9oWKXu dO9z|20|30-101-252-7593|7364.29|y ironic instructions. bold packages are after +709|Supplier#000000709|D8Mg5T7enR4HOYbpwPgkdDycdI5FpoTnXWUHB|20|30-921-692-7914|8638.35|nal requests. furiously even requests are +710|Supplier#000000710|f19YPvOyb QoYwjKC,oPycpGfieBAcwKJo|3|13-147-519-9896|4876.86|s hang about the accounts. slyl +711|Supplier#000000711|oG9,,CGt6x5c sDr1tzAdzvq1y|19|29-291-385-3264|2462.97|ts. blithely special dependencies i +712|Supplier#000000712|u0ZzFkqHLbJbJ|10|20-433-125-7032|24.49|y express theodolites are busily. bold theodolites cajole carefully furiously ironic pinto bea +713|Supplier#000000713|DBMIf1HiYY8OyRFcbtHpKIz|21|31-890-482-5331|2587.02| accounts serve furiously ironic deposits. ironically pending +714|Supplier#000000714|q1PPTQ0r9QK4PkYS95S yftFXTypAM 2l|1|11-165-805-5563|6077.06|nic excuses run after the final pinto beans. bl +715|Supplier#000000715|feMb9HkfGM8e,4i|14|24-382-559-6937|149.86|lithely quietly express atta +716|Supplier#000000716|OJtq1HiFQczPdQvmhx0gE2exTEdnJr |16|26-413-887-2014|8617.52|requests boost. carefully ironi +717|Supplier#000000717|hhUrgvyxsdTfzGY4OrQSHeZmMNB2L75xk|14|24-797-880-9149|6741.18|ng to the furiously speci +718|Supplier#000000718|W9byXRtqvNdPivnxG76r6|20|30-551-841-7946|8000.05| around the pending, special excuses are against the final instructions. regular deposits +719|Supplier#000000719|nQoXFQ,ztoTyboWFmO,a|18|28-664-720-1497|1922.82|jole about the requests. quickly ironic +720|Supplier#000000720|82 sGqlCVpJgAKKoW6yTkci95tvt|17|27-183-709-8965|7752.13|ckages sleep stealthily above the blithely special deposits. requests sleep furiously above the +721|Supplier#000000721|yF,pgNxRtqb1uql2l21qj|1|11-788-642-3247|4179.15|cajole slyly. requests hind +722|Supplier#000000722|XWycFRsEF4TGhGrCPnM17JRB|2|12-909-341-4605|1895.71|s are carefully. carefully silent somas over the furiously regular dependencies was ironic dugout +723|Supplier#000000723|ZFI9Pb HTy,8e,mY0mqP6ThbN|5|15-714-811-1747|3117.73|ests around the deposits cajole enticingly among the slyly express deposits. quick +724|Supplier#000000724|P92abZ6rWSfO2cm|18|28-471-255-1476|4696.62|ly final accounts use carefully +725|Supplier#000000725|fFk BGhsIcG6|1|11-382-323-5460|9077.74|pinto beans are carefully. blithely regular depos +726|Supplier#000000726|jASHnVdyT7e7Lxf6a|16|26-825-782-3808|8617.63|sual foxes. silent instructions are always across the quickly dogged pinto beans. de +727|Supplier#000000727|kc0Fnr5won8yJhzYC2j4H98m 59CRJTs|2|12-885-980-2162|8518.50|gular pinto beans under the pinto beans haggle around the carefully special pinto bea +728|Supplier#000000728|upr7iaSWGsmneQlVWifcSeJQ4|14|24-789-980-6663|3574.05|as. furiously even deposits are. fluffily ironic pinto beans about the packages +729|Supplier#000000729|pqck2ppy758TQpZCUAjPvlU55K3QjfL7Bi|3|13-627-404-3241|7113.46|haggle furiously among the express accounts. ironic, express warhorses promise. even, expr +730|Supplier#000000730|GQ8t3HK2XoGWSP,Sonc|1|11-468-792-6804|5903.85|kages. furiously ironic accounts are slyly. bold packa +731|Supplier#000000731|dxArCeCKpYV4yBOCZOZb39Y3s7EhIi3d|21|31-306-298-2911|3954.93|s engage carefully alongside of the idly regula +732|Supplier#000000732|3Q117DJd7vC3cBv,L4DAiVqWZNa,nBBoA|17|27-230-428-3838|6432.12|nto beans. carefully ironic dolphins nag reques +733|Supplier#000000733|mUdSdgmayvR|18|28-634-280-5540|5714.85|blithely express packages. final deposits nag above the regular +734|Supplier#000000734|dsoJfURkkt|10|20-428-370-2403|6479.49|ly pending excuses are. even instructions sleep furiously; reque +735|Supplier#000000735|7FqUrdaC732vBX3J7ruv0W4 Jfpx84|17|27-984-128-6691|9895.02|kages. furiously ironic depths unwind permanently slyly regular requests. carefully bold +736|Supplier#000000736|l6i2nMwVuovfKnuVgaSGK2rDy65DlAFLegiL7|3|13-681-806-8650|5700.83|the carefully pending waters wake about the requests. f +737|Supplier#000000737|5fna7sQRrNfLatMkl0Oy,Sps0IWTez|2|12-503-512-3693|1922.80|carefully ironic platelets use across the blithely speci +738|Supplier#000000738|dklvCmHEmlCFTuKU5YNnYAi96v,FMMq|22|32-910-791-4020|7435.07|usual packages use final, even ideas. carefully pending requests integrate carefully regular +739|Supplier#000000739|d7rDQneZCae9E57u3LSq7K|11|21-304-934-6837|-811.26|efully alongside of the slyly f +740|Supplier#000000740|vi82FyqGelGW0G1mpP17eDCBJRgNLvNi |23|33-714-391-9055|3057.29|und the quickly unusual id +741|Supplier#000000741|BLP6zAc29lDLOvSE3 h2|7|17-292-821-2297|824.94|even, unusual instructions b +742|Supplier#000000742|yaX50kwIVnFeY3|0|10-673-629-5928|2697.53|en theodolites are about the blithely unusual requests. bold deposits wake. furiously even packages +743|Supplier#000000743|ccFQShf qHch yPwbryx12DfnIYAp83,F|2|12-841-918-5889|4384.23|e slyly after the quickly final platelets? special, special foxes nag slyl +744|Supplier#000000744|5NKuw5W4mFDMQBbUjUO4k kfokG8yvAe|3|13-573-977-6527|5699.09|ular accounts. blithely ironic th +745|Supplier#000000745|KBaVOy ,RKCWhLiYxW|21|31-469-792-6546|5672.23|thely unusual ideas. pending, final de +746|Supplier#000000746|DI4uPKbEC5 D4LIwwSgGZ8SwqA1hLGJJN6guc|22|32-484-989-2368|5238.76|sly special sentiments-- carefully unusua +747|Supplier#000000747|0XGddxTld5cBDIN4Z30Je siitSCbFC|12|22-543-907-2770|1900.17|dolites haggle closely: theodolites affix after the express, even deposits. even ideas are blithely +748|Supplier#000000748|x3Wq1rbka5LB1UnGMzk0hd5,mWjrLfG49kR|1|11-549-384-3124|1084.18|furiously regular ideas-- express packages sleep quickly quickly e +749|Supplier#000000749|KxSfaofVZcFWnLsjaNszI5GTrlyAW,|5|15-131-224-8530|3839.44|gular pinto beans. blithely even accounts wake blithely accounts. careful +750|Supplier#000000750|uJB curMQwnzz79rXo9v4jy|18|28-361-120-8058|1533.06|y among the blithely regular accounts. regular, ironic instructions +751|Supplier#000000751|20kEocUg39iam9T EzquK P7grgp,QD|5|15-773-726-6594|2094.94|carefully special instructions cajole slow re +752|Supplier#000000752|l cHMtEnodxj3FV|3|13-854-821-4478|7588.27|ages. even, regular packages c +753|Supplier#000000753|Kbxpp9hdpX6bgG,|0|10-667-838-1746|151.10|tructions. pending deposits wake. pending dependencies haggle. regular accounts boost. unusua +754|Supplier#000000754|GLSmwjGddmyMx2D BlOKJm1Ji|17|27-971-371-9417|7425.83|leep. pinto beans haggle according to the unusual, e +755|Supplier#000000755|IRW3Y6qorkh4GBy4gHSpVTF5L|15|25-750-724-4757|9046.17|ding accounts was. carefully express ac +756|Supplier#000000756|ySXGqbQpYbXLoaFMKqIdH5Pai|7|17-726-757-7711|6116.81|out the final, express id +757|Supplier#000000757|PE9,2xp10mYiiKvHbHIVG1KIPLDtomT|8|18-152-957-5174|8209.16|s accounts. unusual instr +758|Supplier#000000758|Dko8jlTeGYKaDg s0o a9|17|27-130-847-7866|7448.00|oldly among the quickly regular platelets. furiously ironic packages around the furiously regula +759|Supplier#000000759|1wnIv4xlR2,zrcb495qI0gcXrJyVPrQjNU |8|18-430-878-7018|7602.04|its nod about the deposits. brave pinto beans lose quickly about the ironic foxes. even, even a +760|Supplier#000000760|ob94rzX66tJ35aKv2jR,inK1|21|31-367-390-9165|486.80|aphs. regular asymptotes wake quickly slyly ironic ideas. si +761|Supplier#000000761|zlSLelQUj2XrvTTFnv7WAcYZGvvMTx882d4|3|13-725-649-4070|6610.51| packages boost doggedly. fluffily pending accou +762|Supplier#000000762|GRVedLFC19uN9o8bNyNZWwzcbDUo2eT|20|30-393-711-8623|939.86|bove the bold packages. blithely final dolphins wake against the +763|Supplier#000000763|rpZRD,h5XXPIOe6bX1KqgwvBPoN|17|27-579-968-4858|9206.66|bold ideas. blithely express theodolites haggle blithe +764|Supplier#000000764|2qcwW0V7q3Ipei1tPW3|0|10-917-495-8225|8465.14|counts nag slyly along the deposits. quickly regular accounts brea +765|Supplier#000000765|RLsvd,9rVYPSoMUCDBVoB9|24|34-103-698-2282|8681.24|accounts grow. ideas cajole slyly. ironic requests haggle slyly e +766|Supplier#000000766|wfQTdIQSA7p5PFFvXNjhO|4|14-284-910-3726|-297.76|ickly along the final, pend +767|Supplier#000000767|bHEuqKKdmCMEKOV|21|31-880-346-2583|9504.89|e slyly carefully special accounts. furiously bold foxes sleep regularly. furiously unusual +768|Supplier#000000768|P,qwP7pGhJVoeq LJI|22|32-493-564-7451|4705.25|ffily unusual foxes sleep carefully according to the special platelets. a +769|Supplier#000000769|ak2320fUkG|6|16-655-591-2134|165.76|ly ironic ideas. quickly ironic platelets hag +770|Supplier#000000770|KNKouHfOJPphqjJXncoGYvv|9|19-372-844-8190|830.15|dolites nag blithely blithely final accounts. fluffily regular +771|Supplier#000000771|lwZ I15rq9kmZXUNhl|19|29-986-304-9006|2221.25|nal foxes eat slyly about the fluffily permanent id +772|Supplier#000000772|GxfKyTzgm 6bVmhZPQ6nUMCx5NSsl0 ATg1WccX|11|21-367-987-9338|7540.44|uietly quick packages! furiously bold pinto beans haggle carefull +773|Supplier#000000773|U0Sh9u896MJve84VFCmc6TLb8RUmg9BzJJBks44|23|33-321-732-9512|-707.02| furiously final sauternes about the carefully special packages could engage slyly +774|Supplier#000000774|XVYeiG4,BopCyYAQwld4l0scarsoe8J0cQ|0|10-311-896-5917|6030.51|sheaves. packages about the slyly express pinto beans thrash according to the +775|Supplier#000000775|tmhWTbbv9lv|2|12-946-153-9942|9751.14|final foxes around the blithely reg +776|Supplier#000000776|nklfFoSkCwf,ooSuF|21|31-317-593-4029|7550.40|es haggle instructions. bravely furious deposits haggle blithely. quickly regular water +777|Supplier#000000777|oJlJ0xr1b9l3t IHmi9|11|21-502-733-8098|1636.48|ly even pinto beans. slyly expre +778|Supplier#000000778|SHE9zl 2BWMYhH25|20|30-653-194-4012|2200.43|tructions along the furiousl +779|Supplier#000000779|iAtd5nxCjii|17|27-104-960-9666|9550.77|ut the permanently silent sauternes. slyly pending dolphins integr +780|Supplier#000000780|,G6UHU26b8dkvwpYiM,|6|16-367-150-9939|5799.04|express theodolites sleep. final, even instructions across the c +781|Supplier#000000781|kim1Maww3pdircDNv6hnVK21cI|2|12-757-769-5008|2501.73|ar, regular instructions. permanent, pending packages sleep blithely among the +782|Supplier#000000782|z5xIc71Rb5CsrmE0kO11P|11|21-940-365-9213|2493.13|s around the ironic requests engage according to the silent packages. attainments sleep about +783|Supplier#000000783|dMpZwZ95xznZWg4acMQW dK8AQMhB|3|13-785-672-8751|958.07|e the blithely ironic accounts. final, final warhorses along the ironic, expre +784|Supplier#000000784|Or3 KncT1AHPPb|0|10-734-420-5738|7284.90|he slyly even accounts. furiously unusual i +785|Supplier#000000785|W VkHBpQyD3qjQjWGpWicOpmILFehmEdWy67kUGY|22|32-297-653-2203|5364.99| packages boost carefully. express ideas along +786|Supplier#000000786|QiKBtsiRdDZ2xGcwZgOSoMaKSH4HQ360,88L|8|18-280-624-2919|406.37|uests. regular warthogs across the blithely express +787|Supplier#000000787|xaSs5H BquWpG7s38xn Rf5X|24|34-278-568-1942|7830.47|ously silent accounts dazzle carefully furiou +788|Supplier#000000788|jL QLbG475Uszs4 2RxBp4oR|20|30-377-394-1108|827.21|bold requests along the quickly special dependencies use outside the +789|Supplier#000000789|LF,j2pxKPgtbDGaj,l47vutF5Vz|13|23-428-566-8444|7874.25|counts. blithely special accounts haggle furiously carefull +790|Supplier#000000790|HSYD9,gCfAwpsgCLKGhf,Z4SH8GSYOc|5|15-189-744-3618|-415.18|iously ironic packages. final accounts boost blithely aft +791|Supplier#000000791|2dFfV7U1kBADWXl,NpgP0|22|32-320-959-1370|4992.15| even foxes alongside of the pending deposits boost foxes. bold, silent +792|Supplier#000000792|vsmDNbNfu2bAX2lAj4OTNaMKF x3pBq0yHYPzmm|1|11-678-517-4073|4128.40| wake fluffily against the doggedly sly pinto beans. final ideas sleep carefully ironic accounts. +793|Supplier#000000793|Z4N2V ERHL ds3jr9F|18|28-379-479-7140|7044.94|y regular packages sleep. requests slee +794|Supplier#000000794|ZUzPptVa1Vq9Xo9Pq8rPENR,0KGFd1Q214Hq3dJ|4|14-851-370-3696|9271.66|s against the unusual packages use about the +795|Supplier#000000795|1ozaCnTPf4sAV7oY6VE6y7RqC,WLUVXi|8|18-772-967-6194|887.34|efully final pinto beans. +796|Supplier#000000796|xre srJq9Ivai94OkW0yhsfrx|11|21-322-901-8359|6452.60|ts nag furiously fluffily even requests-- regular accounts unwind. regular, express pinto beans in +797|Supplier#000000797|3kcPU9j dU i|14|24-356-955-8704|8908.16|press instructions. ironic, even foxes use carefully +798|Supplier#000000798|Q4atQnxS0XRmpP|18|28-182-490-3136|7588.31| regular courts alongside of the requests believe slyly slyly unusual deposits. regul +799|Supplier#000000799|jwFN7ZB3T9sMF|22|32-579-339-1495|765.69|nusual requests. furiously unusual epitaphs integrate. slyly +800|Supplier#000000800|Z4 hpmBjpjBXREqzixsBCIaF|0|10-497-654-8607|7956.80|he bold foxes boost blithely about the blithely final epitaphs. slyly +801|Supplier#000000801|zohVF4 4GHOJpWy9kdytvYwm27mJEBhk|1|11-673-791-6926|976.53|ckly final accounts wake since the even instructions. regular, permanent accounts are against t +802|Supplier#000000802|,6HYXb4uaHITmtMBj4Ak57Pd|19|29-342-882-6463|9453.01|gular frets. permanently special multipliers believe blithely alongs +803|Supplier#000000803|,IXoixqcMluU5OEK7RhD,mDrBE2E3ygs|15|25-347-400-8216|1887.55|uiet ideas. even packages haggle carefully according to the fluffily slow requests. furiously ironi +804|Supplier#000000804|N3B GhiD6nanhYdssmqlpy2XVz5jZH|21|31-830-383-7329|9093.94|osits. regular theodolites are regularly slyly unusu +805|Supplier#000000805|LXWtvxudFJf56Uep17HO,NYC4A4mlr|8|18-407-342-1685|5968.71|ructions haggle. carefully silent foxes must wa +806|Supplier#000000806|mYSUX4mem2|13|23-702-985-4737|5054.15|foxes. blithely special packages are furiously. regular theodolites solve. ironic pinto b +807|Supplier#000000807|ClHvM1nuPUESGg35Ls|14|24-255-894-5069|1077.97|lly. even, pending requests boost furiously furious +808|Supplier#000000808|B3zlGM54ECUk5MgRzKI9f7F bB8|15|25-297-954-4894|9438.28|y even packages. requests sleep quickly fo +809|Supplier#000000809|dPqPaxh,IbS|22|32-172-990-2830|7241.31| accounts. express dolphin +810|Supplier#000000810|RMJoA1yw 1fM|9|19-328-138-9772|9713.42|ithely furiously final dolphins-- furiously ironic warhorses beyond th +811|Supplier#000000811|pLtLc7cdmb|20|30-734-469-7797|558.91|bold ideas sleep against the ideas. silent deposits are furiously even foxes. blithely +812|Supplier#000000812|8qh4tezyScl5bidLAysvutB,,ZI2dn6xP|6|16-585-724-6633|8615.50|y quickly regular deposits? quickly pending packages after the caref +813|Supplier#000000813|6EfZUjqLY8G28PhtbPGfz9FjWIXLx|22|32-887-679-3561|5793.63| silent somas. furiously unusual packages affix blithely along the s +814|Supplier#000000814|GWytN8Vx0IWzB8BByw6thupbonInepu|15|25-147-738-5484|-468.26|accounts. blithely final accounts haggle carefully special instruc +815|Supplier#000000815|3f8XIvP m9v5fv|7|17-984-775-9865|3855.74|ms. final packages use finall +816|Supplier#000000816|uCvvad6NCkXBUkr28t dtq swXPtu|23|33-830-680-6168|361.01|lve furiously according to the final accounts. even accounts on the +817|Supplier#000000817|0GTKh7JybR8sVahPoJT8kbNtDV0TzA79Q|0|10-282-124-6047|4468.89| blithely even requests. blithely ironic deposits wake slyly. ideas haggle! quickly i +818|Supplier#000000818|78Rr rF8zcBGTSud4,5B|16|26-754-547-4185|9594.51|es. carefully final deposits use +819|Supplier#000000819|n1YA v3IWFGmIP tZr|5|15-492-900-7246|4049.06|s use blithely. blithely regular ideas according to the pending theodolites haggle above t +820|Supplier#000000820|QoduMcALdP|15|25-716-340-3140|4604.84|osits. slyly final packages are furiously according to the ironic, pending deposits. ruthl +821|Supplier#000000821|O HVe5AKfowNLnep8qd2brd9fbZm WkJuc9Lli|20|30-472-667-2703|6497.93|old, regular packages about the platelets haggle slyly pending, unusual +822|Supplier#000000822|0NJZiE1bKnpzqT j,0|6|16-795-236-9887|797.90|ly even packages shall cajole up the carefull +823|Supplier#000000823| gC0DrEG5U,v893fp3nj mmXa6rYhJ0tjpJ|21|31-834-127-5277|-723.78|ccounts haggle blithely packages. carefully express pinto beans against the unusual +824|Supplier#000000824|wJnn6YrLnzsQWLOZNdMSBz1utk9EFS6icrvQyy|13|23-197-596-6598|-609.59|o beans are blithely across the bold, fi +825|Supplier#000000825|,9nqq,JWR0ztG0qp4rlDHgtShbP7AViBj|20|30-486-346-1320|4461.68|es haggle since the carefully regular theodolites: bold deposits according +826|Supplier#000000826|n,hapmxkVq19Yy9UQ8BVF00sQD|15|25-731-100-2823|9401.23|sleep furiously. regular deposits wake furio +827|Supplier#000000827|AlFjWDq6jDtaSUKnxn54OwQzt8CPUItYbCGztPQ,|9|19-131-253-5697|515.22|ggle. carefully silent requests lose slyly. final, final foxes among the fluffily iro +828|Supplier#000000828|0B2aPqJ6KTEr2fqxuC7z |21|31-911-715-8972|289.32|ions are carefully along the regular, pending pinto beans. special +829|Supplier#000000829|w247SZbFQvw1Fp4f0JFIfVXmIBfulBxfgUg|2|12-123-147-1171|9542.56|nding packages use fluffily above the blith +830|Supplier#000000830|5DHC2QScT6P6lXgRtHajXOfztB7ddjYH1LLUC3|10|20-415-380-4083|-65.23|ideas wake blithely quickly even notornis: furiously pending theodolites sleep. +831|Supplier#000000831|NkkeNeVsWdw8U SeVBnUX 2GB|15|25-275-692-5827|1536.13|carefully express accounts wake quickly about the quickly silent p +832|Supplier#000000832|SwUNp9Yyt5pe 6i5EYiV3hHU9RWJnd3VzBjsbtvO|15|25-658-573-4042|9747.16|old ideas wake carefully above the slowly regular pack +833|Supplier#000000833|ig2vYxu,8xwEzl0UfA4t5VJ|17|27-470-524-5760|6604.79|. quickly bold packages sleep among the packages. theodolites bo +834|Supplier#000000834|fwX0Z5,PgFaauaEXlVQX6UmHM0RDKS4EXe,Tn3nJ|9|19-419-490-3356|3732.75| express foxes nag slyly after the regular pinto beans. regul +835|Supplier#000000835|a7ZBr9561n7CHzwtrfoZnpNWf71uKtH|14|24-772-959-9240|1348.35|final asymptotes are furiously bold deposits: unusual, even accounts across the blithely +836|Supplier#000000836|KU2O25D5,FXdv|1|11-892-817-2809|-256.13|ke finally even asymptotes. accounts x-ray al +837|Supplier#000000837|717LGrDM2ChnIS91,PE4 ycp4mu4HPdcX|12|22-626-153-5392|5167.00|gular instructions are furiously a +838|Supplier#000000838|Zsa9XTlYna7SUIpK0RQAGYSVum|11|21-409-796-7661|5808.87|xcuses. furiously express deposits wake among the furiously ironic instructi +839|Supplier#000000839|1fSx9Sv6LraqnVP3u|6|16-845-687-7291|2761.59|ess, regular accounts haggle slyly across the carefully +840|Supplier#000000840|iYzUIypKhC0Y|19|29-781-337-5584|2963.09|eep blithely regular dependencies. blithely regular platelets sublate alongside o +841|Supplier#000000841|dvQXS7Wi29wVuSUWsknpHVQWVrJ6cUvB8V|7|17-359-161-6634|1685.95| final accounts unwind furiously among the furiously pending req +842|Supplier#000000842|3D3tmBm8zD3A BxuTWfoKXD|4|14-977-614-8564|8732.97|dolites impress quickly about +843|Supplier#000000843|iguHIr15YUL9RMmH7U3XsDfdyEg1441|18|28-503-987-8146|10.33|ss packages among the pinto beans cajole slyly outside the packages. regular, final +844|Supplier#000000844|COb5r2WsvJ0zf 58tJJLgYvEZHJb74EBdp|24|34-454-417-4967|6538.01| pinto beans. carefully stealthy theodol +845|Supplier#000000845|tgajQJpBFV6PGa9bzWSkW1eSGE3q5NsCMYE|23|33-350-532-8789|3408.47|sual courts. carefully special dependencies nag furiously. deposi +846|Supplier#000000846|C1Cs3zHlceRMxlaPIBprCC76x4LKVdH3QhZ|20|30-283-261-2020|7122.90|thely special deposits was furious +847|Supplier#000000847|wMieciw3 YGegk|15|25-908-575-3915|-382.49|ully express, regular instructions. daringly sp +848|Supplier#000000848|tx44JAuF,Jnw1|3|13-169-466-8402|4404.29|olphins. pending packages are during the regular packages. furiously regular requests haggle c +849|Supplier#000000849|uy a2rQl1Bag,cFo5GRTZigHtqh Gguuj2xtlz|1|11-608-566-4693|1955.97|ideas will have to sleep pinto beans. deposits around the dependencies ha +850|Supplier#000000850|l6KnHycBhdFcz58UReuEe1Jf2R40ZICoQ5Qpjh6f|4|14-161-130-9091|3891.72|e across the regular requests. silent ideas alongside of the carefully even ideas integrate slyl +851|Supplier#000000851|fhhjsCI1s8uC|15|25-692-383-2877|157.21|ffily express instructions. close deposits after the bold instructions nag regular, even asymptot +852|Supplier#000000852|n3zasd04WljXdo9xMjQRkZKrEB|1|11-574-892-3228|213.41|Customer s. even asympRecommends haggl +853|Supplier#000000853|hQuhL6zQSZmVifUzpnKvRLNEfJhShDomdbGC|8|18-286-624-2526|9797.52|iously. bold accounts cajo +854|Supplier#000000854|omM4Df4DWXSTKmenaUUSr|16|26-346-921-7567|6082.22|n requests believe fluffily. carefully special depen +855|Supplier#000000855|ekQwhb9fh5VGIvMBJ m,yT571ICZpI,LEb1e|9|19-105-166-2849|9964.88|ess patterns sublate blithely-- slyly pending requests use carefully about +856|Supplier#000000856|U,GH2ZjlmA78JRbjWhbf7jCgrU7a0Dx|4|14-216-125-2920|2763.95| special packages. theodolites haggle carefully. special packa +857|Supplier#000000857|srpKWldPZrVWm0dKjc7p 8fMKm1fYF|2|12-715-212-6604|9761.92|y final ideas poach across the special dugouts. furiously pending accounts cajole furiously. fin +858|Supplier#000000858|aA2g9NPEljznwqrZp4Fdw1Z|9|19-525-296-9901|1671.59|ependencies boost regular, regular requests. quickl +859|Supplier#000000859|OJ7rr38hbK1BlZSRXKojpIho8QazivUtIh0|16|26-731-166-4296|9296.31|ggle slyly among the express accounts. slyly unusual accounts above the thi +860|Supplier#000000860|C9FacsR,23JNHd8ioSI5qC7FfnR|17|27-547-426-7236|8210.13|ilent, regular ideas. bold, unus +861|Supplier#000000861|vj9yxpOqUdt3HIxfOWbA9|7|17-581-652-1425|1747.35|against the carefully regular requests sleep blithely fluffily ironic packages. +862|Supplier#000000862|JAH3,OdzNzdiWYK2ODrKfLLR2IQ8YVNJcsds|2|12-159-806-3875|6698.84|ously special pinto beans. deposits +863|Supplier#000000863|TsC9OuodnybJhWXq4PFNdEJf9jx2y181N3ilV|21|31-589-608-3508|487.31|ounts. fluffily special platelets along the even pinto beans boost +864|Supplier#000000864|D95VTylwusz7OYesg,|5|15-293-129-4196|4089.61|ites. quickly even ideas wake always express requests. express theodolites are. carefu +865|Supplier#000000865|zYVm4GPPlvV1MysjqDUItehzU9hD0tIaHeg|17|27-993-155-8321|4111.07|ly ironic packages. slyly pendi +866|Supplier#000000866|CosbyBH1bG81zFspjW|24|34-374-244-3932|1768.34|riously regular excuses. quickly close ideas sleep. final requests haggle along th +867|Supplier#000000867|WTM7RpRoZPk5MKGEtE2dsh|11|21-221-554-8461|7476.72|y final requests. blithely final instructions will haggle fluffily. iron +868|Supplier#000000868|dFPwNYsP 9xC|1|11-255-295-8702|6239.02|ar instructions. blithely ironic foxes doubt against the quickly bo +869|Supplier#000000869|xi6g0llBz3O1ECUHCk7p341ThVdavdqf6PM|13|23-269-725-9523|5303.66| regular theodolites thrash slyly about the sl +870|Supplier#000000870|QIgRinpKvCLPG|21|31-675-338-9417|3689.14|ronic accounts. quickly pending pinto beans after the regular asymptotes sleep furiously +871|Supplier#000000871|gTuPG353pz9sxS6iazJuqQtji6xN,Q9qgG2|8|18-566-173-8686|5418.37|ess, ironic platelets boost furio +872|Supplier#000000872|oJQEy8xyrptE|11|21-271-893-1965|6571.13| the pending, even foxes? blithely ironic depe +873|Supplier#000000873|jBfvdFseU7cz315kGbbocXnCwDKW,3iRlyaj3wn0|1|11-253-186-4222|-951.70|pinto beans. platelets serve slyly bold, pendi +874|Supplier#000000874|xsGhP46dDeavM6wnREIi7Q09jfAUTzucwp|16|26-826-579-8300|7755.50|beans. deposits haggle after the blithely express deposi +875|Supplier#000000875|pBjvc 55kMxfQ3gtYUiuy5TNRyd|13|23-165-986-6088|5757.36|ep. furiously final deposits sleep. regular, regular packages affix slyly +876|Supplier#000000876|PYBAM85Nrb2cuXH8VDrX 3TjZbsIAlIQ,,xA|11|21-432-561-3770|5979.92|ly special packages. carefully ironic deposits dazzle furiously acco +877|Supplier#000000877|4yN9i5CbQ8Tw1X5InsMlOjjhGg8MAciTG|4|14-685-851-5424|6489.62|beans. quick packages at the pending dependencies poa +878|Supplier#000000878|cennOpnejXFuwxsxrfoz6U,WN TC7|8|18-462-213-5795|4140.02|gular theodolites wake. blithely bold deposit +879|Supplier#000000879|6DGZ6o7FNRspPhM B1nZiMOAgq9fZT8UHW|17|27-204-329-1068|4269.56|s the ironic, ironic platelets boost carefully bl +880|Supplier#000000880|s7AgxI7139o6arS2SfUX|10|20-178-395-2605|3550.33|s hang always against the pinto beans. blithely spe +881|Supplier#000000881|02RRXg45CBGhuzwpKKz3fi4ewYv|13|23-877-378-2281|4423.06|ounts are according to the express theodolites. silent instructions above the d +882|Supplier#000000882|5op1w94,JerNmOkyPfAVkZEtb7|14|24-437-170-2579|9450.21|ly final requests haggle furiously final, regula +883|Supplier#000000883|5ppzWDz6xcMOO09LkrAOvHzFEMfP3CIpndr|18|28-614-756-7513|9746.01|hely final excuses sleep quickly. slyly pending instructions hang +884|Supplier#000000884|bmhEShejaS|3|13-498-258-4793|9223.93|requests. furiously pending accounts haggle furiously. blithely regular ideas wake along t +885|Supplier#000000885|aJUXiGC6qSAWr0Dl0VBahtF|7|17-578-639-8695|1736.47| furiously. carefully pending pin +886|Supplier#000000886|R52IgT6b0yBuU r8,dNRZVWRY|1|11-329-720-1904|-158.08|ts during the blithely silent packages c +887|Supplier#000000887|urEaTejH5POADP2ARrf|3|13-738-297-6117|3113.73|s. regular realms haggle. special, unusual accounts wake furiously. bold pearls play c +888|Supplier#000000888|JA,f8nt64wdZ4XkiHCYHC0r2FJ|24|34-606-153-4636|3420.34|deas wake blithely. regular, special id +889|Supplier#000000889|saKBdGnsGS9ccCMGJ5NFoKwZ7HkS7|20|30-566-872-3482|8570.93| the furiously final requests +890|Supplier#000000890|yY7XhfdJe4ewh8R, xHR06QABT|11|21-196-765-4250|4685.09|ly unusual asymptotes cajole along the +891|Supplier#000000891|cv64gZAB3Ax5XblZ19gNA zOHb9q2nGjtvUzLm9P|8|18-541-281-5118|2284.88|deposits. slyly final accounts are fluffily slyly +892|Supplier#000000892|j6prA4M3sX9a9xHem3HOZpYy|8|18-893-665-3629|9993.46|mong the regular instructions. regular, regular dependen +893|Supplier#000000893|WxOTCcoe RFwKWyZUCURPNAumww1nW,EYcrVjrj|22|32-328-447-9531|-823.97|ully pending pinto beans affix quickly after the decoys. sl +894|Supplier#000000894|T,9KqHZzFlFVvesdyzvzs9FHOQZjLk|9|19-875-711-4227|9490.22|ss the furiously special packa +895|Supplier#000000895|Tm5QKYFUhtY|2|12-826-730-8247|246.80|y final foxes cajole blithely. packages over the blithely ironic accounts haggle silent, regular dep +896|Supplier#000000896|yvNZycuQYm9d9A8v1m|7|17-790-100-9143|9880.72| regular deposits. carefully unusual accounts haggle ironic, +897|Supplier#000000897|9HoSSwrIPM8ge69XLD81Br993krGbn9aeUW4U|10|20-272-778-2639|7373.28| boost. bold accounts nag furiously +898|Supplier#000000898|Uud3qAUC91Cy,c|4|14-247-832-2299|5737.18|print carefully. blithely regular deposits after the deposits cajole against the sp +899|Supplier#000000899|oLlkiVghtro IwzcwFuzwMCG94rRpux|11|21-980-994-3905|7741.42|equests wake quickly special, express accounts. courts promi +900|Supplier#000000900|,6RXmcRyA48c0yvZ2I|5|15-926-534-2005|165.26|counts cajole carefully pending foxes. bold packages mold carefully unusual +901|Supplier#000000901|dVN377SgJQURQd8,XtVF9|8|18-664-532-4405|3465.20| sly foxes are. special requests x-ray about the slyly unusual foxes. furio +902|Supplier#000000902|VrxG9VHAp45UMWrL|22|32-382-410-6632|3660.22|are slyly unusual excuses. pending, special +903|Supplier#000000903|fQVbA9,L6tB9iFrCfk4Yt1dwT0kRWg9DV|22|32-277-492-3117|3238.01| sometimes across the furiously express accounts. unusual accou +904|Supplier#000000904|Wev4Rig5BkWdQIsATpN7W5lG5jh4LQ ,pMxD|15|25-960-333-1191|2152.23|thogs snooze blithely fluffily bold pinto b +905|Supplier#000000905|WAGJHr1OUm95U5|17|27-266-689-2728|2415.66|usly regular deposits. foxes boost blithely ironic theodolit +906|Supplier#000000906|KwvAh8P9RcDPjbx9Qv2xZeWPmkCmK hY|2|12-926-664-9785|7888.41|ructions. slyly final req +907|Supplier#000000907|Y79rRfd5UhgXi6Ahj9ooM0vNHts|12|22-595-604-3254|6558.92|t the instructions. bold, unusual pinto beans above the regular, unusual foxes wake blithely regu +908|Supplier#000000908|05YocPlDfIe SFz7r5BeVAgqOx8i|4|14-669-206-5538|2844.11|s the slyly unusual foxes. furiously +909|Supplier#000000909|BXuLybzUeFLI0GJqMG8xewTuKFqk8n|11|21-989-597-5142|4012.42|ss requests. even packages haggle furiously. +910|Supplier#000000910|0X S 2E55,maqch|5|15-960-448-6633|4763.20|arefully unusual deposits. +911|Supplier#000000911|TpZMQSDVVmgKZX9 wB,HY|11|21-748-770-4721|6026.09|he dependencies. furiously special deposits cajole slyly. theodolites use fluffil +912|Supplier#000000912|ppF7DX,JePrdoo9qYFYwTOwszgUjHk|8|18-633-593-4048|1492.02|the ironic Tiresias. requests above the slyly even deposits affix abo +913|Supplier#000000913|c78mMYZkHE7ktVSoB9D|24|34-601-419-1634|5266.72|s sleep bold, regular accounts. ironic packages integrate +914|Supplier#000000914|li7dM9CrPF213,Jkh3MJRSRhjSB,wRMuOvidQg8u|14|24-682-308-9029|9767.75|ajole. bold theodolites above the quickly ironic frets are quickly along the fur +915|Supplier#000000915|hzB2437Op7JLYX73d3,qU2|18|28-191-772-5459|687.45|ffily. slyly pending pinto beans haggle furiously regular accounts. furiously regular asymp +916|Supplier#000000916|tD 9oW5VNUWTBQCpsISJO2TZCwFzKLdqIZoTWV6|11|21-185-427-4872|1852.85|e packages haggle carefully along the furiously ironic dugouts. unusual frets impres +917|Supplier#000000917|tMr5motk0IFyIXJDwCr98Q O5|22|32-754-782-6474|8877.42|r dugouts? final, ironic packages breach furiously f +918|Supplier#000000918|e0sB7xAU3,cWF7pzXrpIbATUNydCUZup|20|30-303-831-1662|7893.58|ependencies wake carefull +919|Supplier#000000919|xg60CQmqGaVavmevvCjOSGXC 3YuMRVb|16|26-379-721-3359|5428.44|, regular requests: furiously even deposits wake blithely ironic packages. furiously even +920|Supplier#000000920|Ix0QnIqftxMwHW5KHeB,xB|20|30-713-464-9920|-111.84|leep carefully among the e +921|Supplier#000000921|2dsK093unFO|2|12-670-146-9689|-686.97|ounts. blithely final requests wake blithely. regular instructions cajole among th +922|Supplier#000000922|V2KIQXPxtYnOkul|13|23-456-977-9276|2015.59|yly even packages affix quickly! quickly pending foxes haggle at the final, bold deposits. blit +923|Supplier#000000923|aUJaK ezwGVA43Mo0XF|10|20-376-561-9214|5057.87|y. furiously express courts sleep. +924|Supplier#000000924|13INVXLNjpU9eTsoc4dLeid|14|24-768-719-6154|8562.82| shall have to integrate blithely alongside of the fluffily even epitaphs. quickly unusual +925|Supplier#000000925|x3n4pg,q28EckYO,613G7seoAmTPSX0jTvDvM2U|19|29-398-723-8226|406.59|regular packages can haggle acro +926|Supplier#000000926|SEEq DJHLi I6|10|20-500-435-2716|505.92| the furiously ironic dinos. closely ironic instructions al +927|Supplier#000000927|Ype0QDb17eJbg7l35PFzJso|9|19-185-526-3201|8997.88|ily final courts sleep alongside of the ruthlessly regular escapa +928|Supplier#000000928|VL,J8Fq0GI0BnVTaTU9Dcp9Z|18|28-382-849-1505|8512.48|equests are. slyly specia +929|Supplier#000000929|XYQyy9rraHvHMCBQcoGnAvfw0iGG4jOCMENI|20|30-190-675-2042|1235.72|ully according to the doggedly even theodolites. f +930|Supplier#000000930|jYRHnCNnk55 CODbFLqby,ewwOQa1M|10|20-435-373-1909|9740.48|ly unusual asymptotes at the fluffily ironic requests hinder slyly regular +931|Supplier#000000931|CqslHrffpOBxqMDxiGH8nz7scX,i3HmUhNlwBBU6|8|18-174-741-5563|4398.36|t carefully. express accounts sleep. ironic, final theodolites haggle regul +932|Supplier#000000932|2Ke5SDwuwZ1y7H2QROIfgQ1d7|10|20-315-146-6687|1278.74|counts wake carefully across the pendi +933|Supplier#000000933|TrYn5zjv6nrT47EwbXi1S,IU|7|17-446-406-8093|6756.59|ully pending platelets. quickly ironic realms cajole slyly across the even ex +934|Supplier#000000934|2o3Fav4osE|9|19-497-408-2402|3103.34|t the fluffily ironic platelets. foxes use. blithely pending p +935|Supplier#000000935|ij98czM 2KzWe7dDTOxB8sq0UfCdvrX|3|13-437-885-9309|4734.47|ly regular pinto beans wake blithely bold pinto beans! warthogs between the +936|Supplier#000000936|CuIXj6RYaHGQ5SOkmhu8ZDRt5IU|17|27-555-580-3995|7304.30|leep. final sheaves affix across the requests. carefully express ideas ar +937|Supplier#000000937|UBfhiW HpJzqh9uTnhXkOqjRvP1vAIWokSeR5|15|25-410-699-7522|1463.21|lyly regular decoys lose packages-- quickly ironic foxes across the express accounts bo +938|Supplier#000000938|3xcs3BuTqx8b|4|14-704-203-8596|5204.43| express platelets integrate slyly above the ca +939|Supplier#000000939|mWBKbdzDn3yJNpOT8p|23|33-487-125-3117|7815.06|efully. final requests after the unusual requests wake fluffily after the furiously r +940|Supplier#000000940|QqKPZBeHgcIKDeOfT6J8sRlg4|2|12-972-343-8810|5789.96|ccounts. quickly final patterns mold +941|Supplier#000000941|gqG2XEnVlzUhjjfQGYGlwk,jcaNsplI8Rleg|21|31-412-752-5573|3846.91|ronic theodolites. final, unusual +942|Supplier#000000942|VkukmyN0Dq3NkC1RMw2ZBk,I5icQfLFFG|24|34-943-753-9952|6074.75| deposits haggle. regular packages unwind fluf +943|Supplier#000000943|AaeNFJbUAF8MOb5VKA7wXB6|16|26-877-341-9002|5339.25|uffily unusual packages wake bl +944|Supplier#000000944|tUVVFs351nHm|13|23-452-536-7876|7776.86|pinto beans. quickly express requests haggle +945|Supplier#000000945|y8LENmYfvNpYOnHG4XdxLQGqjOPJJ4c0CacGxu8|1|11-785-307-6941|1706.26|quests wake slyly quickly ironic deposits. instructions wake +946|Supplier#000000946|RpFjkZmA ScvLe|15|25-522-937-5559|9093.75|shall have to use slyly. carefully ironic ideas alon +947|Supplier#000000947|6xS,tBAgcKGW,nXNhfm2Hv26uYicU3|18|28-277-111-4291|2699.78|ronic ideas. slyly final dolphins wake furiously after the +948|Supplier#000000948|LvcPHBbzYZKySxlda,0McYfaV2bb poGXMF|21|31-562-389-2753|9219.26|grate slyly after the quickly even accounts. bold accounts haggle carefully. quick, b +949|Supplier#000000949|a,UE,6nRVl2fCphkOoetR1ajIzAEJ1Aa1G1HV|23|33-332-697-2768|91.39|pinto beans. carefully express requests hagg +950|Supplier#000000950|MukfcGGpbbRXp5v52Rx43QpPjU8RFB|1|11-888-171-3390|4992.53| the furiously daring dependencies wake blithely blithely regular braids. ironic ac +951|Supplier#000000951|Tuh3kFB,zCuI,jtPmV,IMcXQF|2|12-561-627-9752|-511.17|eans. quickly fluffy accounts are quickly about the ironic +952|Supplier#000000952|n8W6MbJdih Ckh6wDvYJz84ZmabvK4yQDz|22|32-276-558-4960|8621.52|cajole permanently? carefully slow deposits cajole quickly. regular +953|Supplier#000000953|wTTb0ilU6Nba1VLsHj6k0jUt4TFFM6rvtXszzA |21|31-642-490-3022|7916.56|kages are carefully platelets. blithely enticing platelets c +954|Supplier#000000954|P3O5p UFz1QsLmZX|6|16-537-341-8517|6721.70|ect blithely blithely final acco +955|Supplier#000000955|7OFLXDHjSgGrFlHTg8VHFS4glUuN|17|27-839-781-6125|-408.45|ave instructions haggle. regular instructions past the theodolites are slyly depos +956|Supplier#000000956|dmmnYeCuIZB7b2pWTOQ9zrAdi6zxwIrj4aT446L|18|28-741-846-4826|8068.26|, regular accounts use against the furiously express ideas. furiously +957|Supplier#000000957|mSpFa,4jJ5R40k10YOvGEtl4KYjo|23|33-616-674-6155|4324.51|hily after the fluffily regular dependencies. deposits nag regular, silent accounts. i +958|Supplier#000000958|uAjh0zBiJ0d|8|18-553-836-9296|3011.31|ress pinto beans cajole carefully among the quickly special requests. fluffily even dep +959|Supplier#000000959|8grA EHBnwOZhO|7|17-108-642-3106|9032.15|nding dependencies nag furiou +960|Supplier#000000960|yk1dgGaQlbTN1YhnJjyM3ULEDDf|12|22-811-163-5363|1480.82|onic theodolites wake carefully. ironic packages use carefully. ironic +961|Supplier#000000961|jlfkD00x6r0M34ctcSTY3cABv4yWKcjvHV|15|25-911-416-1546|4139.88|luffily bold packages wake ste +962|Supplier#000000962|0udWpU30ecw3XMLiEVhaM7I,BhufyzF4i|18|28-105-675-7555|1898.39|ent deposits are furiously slyly ironic es +963|Supplier#000000963|s3WbU6w31FgnuZVtwGnH6PkyZFg|11|21-395-611-8793|4152.04|ily quiet accounts! blithe +964|Supplier#000000964|JpH9YUQjGXD1GChWcPj9LEGMN8xwZkCzpdG8HlNb|7|17-693-102-9498|8411.78| instructions; fluffily ironic pinto beans across the unus +965|Supplier#000000965|aPiA00HXK1,L2FmArcXVT|7|17-117-423-8603|2843.80|ut the express packages. ironic sentiments are qui +966|Supplier#000000966|FRlJWy32I6TsERrGDq,GS7|4|14-681-361-1636|9657.79|beans-- fluffily unusual deposits x-r +967|Supplier#000000967|uPDH,GMFjz|4|14-583-250-9472|502.38| across the quickly regular pinto beans are furiously brave accounts +968|Supplier#000000968|6idVVWn8RbFoZgPeyVJlQOJ|4|14-431-296-9521|4487.96|final accounts. slyly regular deposits wake fluffily after the +969|Supplier#000000969|thRdVx7vCajVFs7gsK8VKxzydPiHvIKK,DtR|21|31-171-204-7902|162.06|ounts nag slyly across the furiously +970|Supplier#000000970|FD,NVDvbAAT7rI6BdKI2rTo9UqH8AghW1e8DSJ|2|12-826-418-6561|4643.76|cording to the regular, ironic depo +971|Supplier#000000971|23XaUVLYuC3tQIPHCuLLgM5UawL|3|13-380-957-8529|709.58|ly final pinto beans among the furiously regular theodolites caj +972|Supplier#000000972|MkZSNXPZf9g8ZW3ez TU, s6S9aPVY|12|22-291-368-8958|-203.99|ing pinto beans integrate. pending +973|Supplier#000000973|5 nhBZ 03rG6EcOEDkZXvt|21|31-385-469-4031|1548.60|al ideas cajole quickly ironic packages. carefully unusual theodolites detect. unusual packa +974|Supplier#000000974|xIscm3sM7v5hU7NioMfSJ9tLeIBvSm3vGc|16|26-370-235-3418|-778.92|x furiously slyly regular +975|Supplier#000000975|,AC e,tBpNwKb5xMUzeohxlRn, hdZJo73gFQF8y|3|13-892-333-9275|7577.42| slyly express deposits. pending asymptotes could have to use furiously. fluff +976|Supplier#000000976|MVpCgFTl7sGge4cFxVXD|24|34-998-900-4911|4744.07|t the pending dependencies sleep blithely about the blithely pending acco +977|Supplier#000000977|Kuud1x4l,UNEkRAQjCEsu|20|30-938-867-9723|3633.55|lithely ironic requests sleep enticingly ironic foxes. deposits along the slyly pending dolphi +978|Supplier#000000978|XzhDlm7Mr3RyWZL7PV6ush|16|26-135-110-8202|6104.45|ly ironic requests. carefully bold ideas haggle quick +979|Supplier#000000979|cdvHjrKZR7iDlmSWU2a|10|20-151-688-1408|9538.15|ckages cajole quietly carefully regular in +980|Supplier#000000980|jfgiJfywBW88ZEYM 5V|13|23-105-829-3910|2783.33|xcuses. unusual, special accounts integrate furio +981|Supplier#000000981|uf24XV3FD7J0BY5FQ29Nbco8A|19|29-161-734-6046|5343.95|inal packages sleep along the accounts. fluffily special pains wake fluffily. blithely bold requests +982|Supplier#000000982|2GJow4mz8ZkIPUSibA0NZ3OyR5TkfHx0|10|20-884-330-2979|9763.28|deas cajole carefully furiously regula +983|Supplier#000000983|XOYb xohl2j0U7wTTUaT4F6DShKfH4Hv3p,hnP |0|10-384-209-1825|2576.28|onic requests. slyly unusual ideas wake carefully final depo +984|Supplier#000000984|6H6qqye iYbYzCmwWhj|21|31-519-879-5266|1444.79|iously except the blithely unusual packages. c +985|Supplier#000000985|kzI8mk3jN9F67EStJ 8dlpx 6GwZYwzXPFOKJ5R|1|11-131-656-2612|3524.10|ut the furiously final deposits integrate according to th +986|Supplier#000000986|tKoJtnykz0R39BWTgglt0rZxT|22|32-342-471-2481|3516.69|jole enticingly: regular foxes among the regular deposits shou +987|Supplier#000000987|DAWJ1lDhybbSO3mngqD28aX|18|28-375-179-1732|-40.30|uriously unusual courts. slyly unus +988|Supplier#000000988|dFt73JWMYsSxR3 UQN K3FAz|0|10-630-928-4130|2536.81| according to the ironic packages. +989|Supplier#000000989|e5uX8AJF,,zRIIDgJc YMB59ITz4v|1|11-895-219-9405|9098.64|t the silent, final dependencies use busily above the reg +990|Supplier#000000990|DeOjGX,4Ns1|2|12-647-684-5389|7985.78|y ironic packages sleep fluffily despite the grouches-- bold, special accounts nag along the carefu +991|Supplier#000000991|Bh4Danx VvUpMce x42|16|26-793-462-2874|4026.14|foxes are slyly above the furiously express t +992|Supplier#000000992|iZPAlGecV0uUsxMikQG7s|2|12-663-356-1288|4379.45|silent packages. quickly regular requests against the carefully unusual theodolites affix fu +993|Supplier#000000993|z2NwUJ TPfd9MP8K3Blp1prYQ116 |2|12-316-384-2073|2336.52| asymptotes haggle slowly above the +994|Supplier#000000994|0qF9I2cfv48Cu|4|14-183-331-6019|8855.24|sits boost blithely final instructions. ironic m +995|Supplier#000000995|CgVUX8DtNbtug2M,N|18|28-180-818-2912|9025.90|s nag. furiously even theodolites cajole. +996|Supplier#000000996|Wx4dQwOAwWjfSCGupfrM|7|17-447-811-3282|6329.90| ironic forges cajole blithely agai +997|Supplier#000000997|7eUWMrOCKCp2JYas6P4mL93eaWIOtKKWtTX|3|13-221-322-7971|3659.56|y regular excuses boost slyly furiously final deposits. evenly fi +998|Supplier#000000998|lgaoC,43IUbHf3Ar5odS8wQKp|15|25-430-605-1180|3282.62|hs against the unusual accounts haggle r +999|Supplier#000000999|XIA9uPu,fDZTOC,ItOGKYNXnoTvCuULtzmnSk|2|12-991-892-1050|3898.69| ironic requests snooze? unusual depths alongside of the furiously +1000|Supplier#000001000|sep4GQHrXe|17|27-971-649-2792|7307.62|press deposits boost thinly quickly unusual instructions. unusual forges haggle ruthlessly. packa +1001|Supplier#000001001|3 r5bhzl9,C3wVHCfjTR4TC5W4lMF1 dg071|9|19-393-671-5272|2140.50|detect slyly. ironic, unusual platelets haggle blithely accounts. bold packages according to +1002|Supplier#000001002|bzwzvRnMTA2IEIv,AKydTru0vsbETQkhV|22|32-102-374-6308|3316.16|ns are carefully. blithely regular requests cajole furiously. furiously +1003|Supplier#000001003|igHYuUv LqmAM09j02tF5epUICqYdWFFkW|10|20-763-167-9528|6680.29|e quickly daring deposits: even packages boost. slyly express requests cajole furiously among th +1004|Supplier#000001004|mX0LfriA7aW17hdA16d8NA4EB9RqPG6c,QiJ|10|20-614-711-6853|7723.42|ular dependencies. pinto beans across the quickly ironic depos +1005|Supplier#000001005|MJ59SpLU,OkpgoiBQAUvkcs,cfHx4D3B|19|29-916-461-8559|5538.20|s doubt final frets. special acco +1006|Supplier#000001006|d0DSDuBcFCmu9jVJn,sVnyG|18|28-404-261-2246|7520.71|ording to the packages cajole slyly ironic accou +1007|Supplier#000001007|d9CnB3YuD2DtIolYrYacR5RbiW9|20|30-148-837-8918|8702.81|sual pinto beans. slyly blithe foxes alongside of the even accounts sleep furiously along the slyly +1008|Supplier#000001008|Dw3V1HADKPrGZb90fWTDBpvb|19|29-218-505-4622|-858.86|ly after the deposits. regu +1009|Supplier#000001009|,1Dpa8mtvfnGrje4|0|10-676-938-4100|6133.05|ve the furiously ironic deposits. final ideas sleep about the fluffy, final deposits. idly unu +1010|Supplier#000001010|Zem5taPOhC5T2ZQbhiATcdpw7rm|22|32-499-859-6008|6241.13|ggle carefully. requests was carefully about the theodolites. fluffily +1011|Supplier#000001011|QlLyOsPM4HcX0Sq8CXnAYG3JOH39a4Lpzg3l|9|19-284-168-1725|7639.67|ld escapades are. carefully express instructions alongside of the carefully close packages was fox +1012|Supplier#000001012|WSwwbhWWo7eU6xAdlrppKUro HEj7d7R UULrYVZ|2|12-916-437-1781|6013.38|deas breach carefully slyl +1013|Supplier#000001013|riRcntps4KEDtYScjpMIWeYF6mNnR|22|32-194-698-3365|7914.45| busily bold packages are dolphi +1014|Supplier#000001014|8o3SbVMNnBM1foNWeoiVXoPpLhhZcc6tB|2|12-528-127-4486|4164.87| blithely special packages use slyly among t +1015|Supplier#000001015|P4DJu8H4uBwPuxz5|8|18-358-817-6779|3928.61|eans. final instructions sleep f +1016|Supplier#000001016|iRdPyOMBwZJ1tC SuISP|21|31-756-833-2500|4168.02|. escapades nag after the ironic requests. carefully even inst +1017|Supplier#000001017|MohDjydWIPi1so8 |2|12-476-223-5170|1917.30|e quietly express theodolites +1018|Supplier#000001018|8ii8OvolU4lTcuHKFr8K3VNpfQ2B7,|16|26-575-376-6277|3103.39|ut the slyly unusual excuses wake slyly slyly reg +1019|Supplier#000001019|fxQJbCWBDTnHI7L9DpLUo3Vplg|20|30-867-236-5805|5686.20|nic deposits wake finally among the reg +1020|Supplier#000001020|Zu9g5YKvFjex|1|11-878-636-2246|8957.03|ress deposits haggle furiously acro +1021|Supplier#000001021|WCXJ5r,lNGHu,HMckhiuRi91gu0|2|12-140-187-7393|3848.42| platelets above the accounts wake according to the carefully unu +1022|Supplier#000001022| ERUtHzpZ3GXDESNGi0o72 FP1AAslki|14|24-859-889-7512|9353.72|e among the ironic ideas. furiously +1023|Supplier#000001023|33AoyM80E K3Yyb20ug8S|17|27-534-251-9499|4974.97|s are. special, express packages boost furiously. blithely unusual theodolites are. entici +1024|Supplier#000001024|w924TeycgGZvIbF2BApEzAzvZg7pVgRmWomz|10|20-360-741-6895|1321.82|s wake above the furiously unusual accounts. accounts cajole carefully for the furiously brave pac +1025|Supplier#000001025|H76hZYHauWUYycKZtDN7beix1eC3Tb|22|32-604-449-9024|4707.47|bove the carefully regular requests s +1026|Supplier#000001026|rggfDIM7RY1sdPft |5|15-655-974-9432|4418.65|fluffily slow requests. accounts grow slyly. carefully final acc +1027|Supplier#000001027|z7udzhYewB47VSqZrhYfek6Jz|18|28-200-973-3276|1178.68|eans boost blithely. slyly regular accounts wake carefully after the regular deposits. +1028|Supplier#000001028|nOkvPq5WfeN87ASUIgQNKT2U0TUX 4WzZQ3f|11|21-570-218-8213|8106.47|riously bold accounts haggle slyly after the blithely u +1029|Supplier#000001029|6quaw3pjSuzrj5d|5|15-507-720-2035|8476.39|ly fluffily even pinto beans. bold packages across the doggedly unusual deposits slee +1030|Supplier#000001030|t8woLlk9uN3WTwa7zo|6|16-100-461-1195|-278.32|yly about the quickly pending requests. packages hag +1031|Supplier#000001031|IqSuMmZ2GwR,Zp8z8|8|18-423-362-3331|6240.69|tions serve blithely blithely even +1032|Supplier#000001032|EAEZHF7prT4nExxWsBPaSsOtWWytNfLdVHcIbIGA|5|15-328-403-4879|360.76|ously final instructions nag requests. express foxes detect slyly after the doggedly regular pac +1033|Supplier#000001033|A6x1P,56sVHkb4DqVyUw32n,ChI|3|13-785-290-6801|293.52|quickly regular packages arou +1034|Supplier#000001034| L3kwDU2COZtFFiv wR1669zh4KrEzZ5xNlc6Ag5|6|16-686-979-1675|3184.54|ously final deposits. furiously regular dolphins detect carefully. regular platelets sho +1035|Supplier#000001035|irCdtj9FHyuwJKYnueu0HC4G1Yjgam2v 1vA2gk|7|17-864-859-4104|5609.67|realms about the bold requests wake acr +1036|Supplier#000001036|SZ8jSo8Mr34|9|19-113-246-8057|9759.45| atop the express hockey players. ironic requests sublate slyly. final theodoli +1037|Supplier#000001037|QrDKVeIxzNFoCDf4|8|18-415-126-3978|4176.78| final deposits after the carefully ironic pinto b +1038|Supplier#000001038|JKW9kNWHDXhY4L u3PAd|16|26-767-195-4479|6071.99|s. slyly bold theodolites a +1039|Supplier#000001039|GXFzvmVvzg56hhqmVfwH7XvN80xwx5|6|16-217-795-1403|2973.47|arefully unusual deposits x-ray ideas. furiously pending braids +1040|Supplier#000001040|R0BOFsUB3j 8NrouhzaD5EYPFCP|8|18-799-389-4112|63.96|l requests sleep furiously ironic plat +1041|Supplier#000001041|ZNny0BQ7n2qc7b6V5|4|14-565-600-1255|5829.16|ntegrate blithely. slyly pending deposits of the carefully regular instructions nag among the i +1042|Supplier#000001042|uk58OAP,HfaV4HMKV3YIDCPfZvNf3IJ0gC|9|19-314-738-6331|2741.25|ly pending requests cajole blithely. pendin +1043|Supplier#000001043|9k7mX00VqjfxiQZLC|6|16-533-948-3796|2156.88|he ironic notornis. doggedly ruthless ideas sleep carefully after the slyly regu +1044|Supplier#000001044|imHHzVmeNI,OwowfxLg5lJzQOROqT|14|24-230-793-4577|3654.82| ideas integrate even, express asymptotes! ideas +1045|Supplier#000001045| GYqHNiVWsfYm9U|18|28-801-627-8967|530.42|y idly final requests. carefully careful requests cajole slyly accounts. deposits against the car +1046|Supplier#000001046|4EmyZsmUCKuMN5mkCrobX5IWXZ5|20|30-110-909-6014|2713.80| haggle furiously even accounts. excuses grow carefully regular accounts. pending p +1047|Supplier#000001047|8uI1e3tZbzJ3YYtevqITi3V|20|30-137-531-7508|2753.84|re. carefully express requests +1048|Supplier#000001048|f172BTkk1PgyO06NwAt|9|19-493-938-3406|4694.02|out the ironic packages. regular foxes sleep quickly alongside of the ir +1049|Supplier#000001049|hJjbktopDAAM|17|27-362-693-9939|9408.38|furiously furiously ironic platelets. blithely even deposits use carefully alongside of the final +1050|Supplier#000001050|Eu5ETUhMtI|14|24-271-128-7378|-249.59|ithely regular deposits. final, final deposits are slyly final +1051|Supplier#000001051|DxBxQBR5cbx50lke0A9nr5pkkRrt7Sp|12|22-964-969-2212|5098.67|ests. slyly silent ideas lose +1052|Supplier#000001052|9Hju7eTD4D9L|4|14-562-816-4934|1410.15|nal accounts. fluffily bold packages in +1053|Supplier#000001053|F69fPuCNvPpsoA8gP3RgIMa 9D1a|20|30-774-136-2488|4399.63|thely silent ideas. deposits cajole. furiously pending theodolites slee +1054|Supplier#000001054|1TaZy4jWpPn 8z4Lm0vB|20|30-373-143-7187|566.83|p evenly. carefully ironic requests use. furiously bold notornis across the final pinto beans sle +1055|Supplier#000001055|euFgXSBA1XoI,V48YwL|15|25-369-552-4996|9722.62|its cajole blithely. regular deposits use after +1056|Supplier#000001056|BC9t,3GX80PtaQIrtqf dxxIbFMk|21|31-814-871-6323|1093.16|counts haggle slyly boldly b +1057|Supplier#000001057|2,cJszqUjE|9|19-146-393-5146|649.87|excuses are idly final req +1058|Supplier#000001058|fJ8egP,xkLygXGv8bmtc9T1FJ|20|30-496-504-3341|8230.12| requests haggle? regular, regular pinto beans integrate fluffily. dependenc +1059|Supplier#000001059|4tBenOMokWbWVRB8i8HwENeO cQjM9|10|20-620-710-8984|7741.10| to the carefully special courts. +1060|Supplier#000001060|0D46fGOyD3nAIccLgHmuW v1p0N|15|25-445-428-9650|5771.63|ending instructions according to the slyly ironic theodolites nag against the blit +1061|Supplier#000001061|6ncJ56M2LAPDEBGz|10|20-204-356-2184|3751.59|tions around the ironic platelets cajole slyly furiously special pinto beans? fu +1062|Supplier#000001062|RzD13PXq2yeZm|20|30-916-179-1075|9940.76|y idle packages use fluffily toward the pending deposits. furiously ironic depths affix +1063|Supplier#000001063|cUoXUBJfIxIWhYL 7lLHunPU7U|16|26-234-743-9528|4454.44|uses cajole quickly slyly regular requests: slyly regular instructions doze ag +1064|Supplier#000001064|QgmUIaEs5KpuW ,oREZV2b4wr3HEC1z4F|11|21-440-809-7599|8750.12|sly even deposits? furiously regular pack +1065|Supplier#000001065|0Xqs0Cx0BC|9|19-665-446-9549|2283.85|ding foxes. furiously express requests cajole. fina +1066|Supplier#000001066|KWmIJTcux94j32QEA0LIg|12|22-175-705-3125|3573.66|y multipliers. ironic, final deposits wake carefull +1067|Supplier#000001067|2VAvD0OEr0B2eYt108NStNUicBOJ5TXKJ6Ej3Hu|13|23-594-463-4398|471.79|regular pinto beans haggle slyly against the blithely regul +1068|Supplier#000001068|,yIRtm8YNahsojbY4zSe57Bs|5|15-332-440-7893|7253.77|iously ironic packages can haggle fluffily special deposits? carefully regular pains are careful +1069|Supplier#000001069|VlfbeQ8ZEEaAYXcoXBft1TQ|18|28-959-318-6130|-878.43|ts sleep fluffily pending dolphins. furiously pending platelets cajole slyly across the furiously s +1070|Supplier#000001070|K pkDMZV75MUPhz|18|28-178-598-4130|7087.97|ven theodolites are enticingly ironic deposits. blithely special excuses boost blithel +1071|Supplier#000001071|1n,xZC4Q5gbT|15|25-400-481-5359|6519.70|e blithely unusual platelets sleep quickly unusual ideas. unusual, regular foxes +1072|Supplier#000001072|OXPUJIe70k|17|27-579-309-4798|3583.12|leep slyly even dolphins. slyly regular tithes cajole carefully blithely silent acc +1073|Supplier#000001073|ZRBpmsZGGuJBoE|23|33-809-500-1650|5516.46|lar excuses haggle furiously around the packages. furiously ruthless theodolites na +1074|Supplier#000001074|ee63e4oZmfmrtmqbxR90Jc,brxT|23|33-579-343-9586|5093.89|ounts use furiously pending requests. even, ironic sheaves affix fu +1075|Supplier#000001075|xL8o5RMVz5|10|20-925-260-1988|8972.50|lly regular excuses grow slyly even deposits. pending packages sleep blithely +1076|Supplier#000001076|JAOvjKSP 5NvmT9skNrI,ukWr5Ah0SBMv8Eg3ogR|1|11-392-708-2761|-127.82|inst the ironic deposits haggle furiously across the even packages. fluffily ironic gifts +1077|Supplier#000001077|Uty1fYWj2tt8 1aE5 SwPe2BvdSZjYrMixasTH8|15|25-195-244-3146|5587.72|theodolites are carefully around the blithely ironic foxes. bold platelets impress furiously at the +1078|Supplier#000001078|DD0ZwJwGoXk5raQZLfe4RG0|24|34-217-422-4794|3780.29| requests nag quickly against the requ +1079|Supplier#000001079|UaWFEXgDo775PQkJo|14|24-896-212-6974|8365.19|lly final pinto beans. unusual +1080|Supplier#000001080|RPVbFa a5wCXrpH Y8V|21|31-148-197-8002|836.52|s dazzle slyly about the carefully even pinto beans. furiously pending packa +1081|Supplier#000001081|TnYIax25gGgVkrbBg9gQPhyKynKLiMRy YQkZ|15|25-575-555-8112|-656.19|regular deposits sleep after the slyly unusual requests. blithely final acc +1082|Supplier#000001082|tAz8EuLYT5Mr,xNW1ipX ARdj4Q0VqmxIKTe6drU|9|19-217-624-4050|2283.43|onic, express requests. fluffily even deposits sleep. bravely final accounts wake +1083|Supplier#000001083|8KwL2XsO h1JL3oGR2K|20|30-470-533-2200|7512.72|hely bold requests. express packages must have to use: carefully ironic requests against the silent +1084|Supplier#000001084|BIOIy5q9CGeI|1|11-649-814-3238|6342.39|reach carefully regular multipliers. close cour +1085|Supplier#000001085|BVrNbKyP0FyScBYOXaYVsCx6mPgT|2|12-989-354-6526|4643.30|inal foxes. express sauter +1086|Supplier#000001086|x6yrc1npoiJBCG4m0CIqC18|24|34-943-139-4830|3889.84|ns: blithely ironic excuses haggle carefully bold de +1087|Supplier#000001087|z ckHESeMGv6QSd4gTp0G4Z2ckaUVvo82UyV|2|12-511-658-5708|1551.91|of the furiously pending excuses. ironic, final platelets sleep: special accounts mold. si +1088|Supplier#000001088|4GAszSHeEDCKAjzBHRsI79Tv|5|15-493-974-1062|-20.28|ackages should have to ha +1089|Supplier#000001089|lbAfNAr1HpY2LGSPclm2dIlpRethjTL|17|27-487-835-2285|9024.46|ts. slyly ironic instructions above the carefully ironic excuses wake instructions +1090|Supplier#000001090|zmTmXSUDDsZhMdPklz8cv9JRplqUgMA|21|31-523-106-8117|4126.63| are. tithes sleep carefully. carefully express packages c +1091|Supplier#000001091|qz7KNEAyDOkl8jrU|13|23-359-544-6544|-764.86|instructions lose. pending, regular packages integrate blithely. silent dinos are slyly against +1092|Supplier#000001092|EkMrbRGvQE91Ljv7sTBWhEtK|19|29-567-800-7497|1911.17|uiet ideas must affix. blithely express accounts cajole carefu +1093|Supplier#000001093|aOjHKEAu3tjFIlXMoX9F3,ahpotMrHcu|11|21-323-214-6195|-935.13|al foxes are according to the even accounts. regular, special packages hang +1094|Supplier#000001094|Fwxdag75rAW0LrFJp1utUP52S|7|17-465-579-8886|175.68|lar requests. ironic, final accounts wake slyly above the furiously final accounts. entici +1095|Supplier#000001095|g03W1pFPhUtYCsSQQT1ZYEYQxkkjiyWfT|4|14-128-328-2874|2506.96|rhorses are. express accounts nag about the furiously even accounts. slyly ironic accounts aff +1096|Supplier#000001096|Nf4FN7Tb15YUSlV|9|19-682-493-2497|2476.22|ependencies play to the blithely special requests. carefully pending accounts wake +1097|Supplier#000001097|1FeaDqTTemSqxWvrsh58K3YCwiG|0|10-636-809-3781|2856.30|: furiously regular ideas haggle. quickly eve +1098|Supplier#000001098|gBFyTaxwwnJNvWpX,Zpn,e5KDnfv7 P|14|24-202-272-6922|-111.22|requests after the packages sleep alongside of the silent, unu +1099|Supplier#000001099|Ttj R 9PUekFZI 3zq|22|32-784-328-6730|9651.40| the permanently final foxes. quickly express excuses around the furiously close requests +1100|Supplier#000001100|WXvtvdCRMWZVJXwNHX5Iokh7Q|13|23-993-221-6400|3224.52|fluffily unusual foxes cajole blithely. quickly express deposits lose ironically +1101|Supplier#000001101|Hr1EBv8bSuV0wcG|10|20-392-415-6037|6509.11|ular, regular deposits. packages haggle blithely fluffily busy accounts. carefully busy packages +1102|Supplier#000001102|LRXta86ao,jqc3x6PcLdhE0QH28kXkU|10|20-630-132-3089|1710.29|y special requests are regular foxes. pending +1103|Supplier#000001103|dtQlaRi69HTRWj9|12|22-349-205-2657|-121.34|ial requests are after the sl +1104|Supplier#000001104|fzGxlItJO,KgTJDCOKTJnwniTU LqvU|19|29-656-386-5539|4584.37|ach carefully. even accounts boost ca +1105|Supplier#000001105|bCpmcscVvexCmGtp1w0 Picp,9huGL|16|26-916-849-7204|3430.48|thely even instructions sleep carefully about the furiously +1106|Supplier#000001106|s,fPdPCojkKGWBJZvp|18|28-882-315-2915|3079.79|rate furiously silent packages. +1107|Supplier#000001107|osYWPXbB4Jvh68UvinCY06SkHE18gqzDAG0rYT|17|27-902-546-2619|5050.34|es play furiously blithely even instructions. even pinto beans haggle blithely above the +1108|Supplier#000001108|9cIxntXdykwaYWA2|7|17-726-989-5062|3145.86|g to the pending requests. silent asymptotes boost blithely. quietly iron +1109|Supplier#000001109|Dc95AaAhNEePMBhnE6 u8fk|7|17-389-465-7531|4370.66|ns are across the carefully ex +1110|Supplier#000001110|RAo58clTXY7Z3IZCtGIGa3HLD84J,9NuUpXOh06Q|1|11-569-316-3148|8684.91|ng the even requests wake blithely a +1111|Supplier#000001111|W1uUa109YUVRw6Ch6NuvFnaRcm|13|23-375-309-6302|6387.92|regular instructions use regular deposits. platelets believe furiously carefully busy +1112|Supplier#000001112|hjJ3hLM1PDH4KZR1E|23|33-503-785-9446|-244.96|ages nag alongside of the unusual requests. furiously ironic dinos nag enticingly. blithe +1113|Supplier#000001113|,NCWfNrXfNY7x7U4mKbA8V7ZIByJel7eWveJF|15|25-564-930-6336|5843.59|thely bold accounts wake blithely theodolites. unusual accounts are after the +1114|Supplier#000001114|f,UYBv5a7bpz E4v08Z0i|5|15-513-472-2380|3309.30|quests along the quickly ironic deposits are carefully about the ironic package +1115|Supplier#000001115|6O0aDsTBnt1VIzyewtHhfqPSNVVi wSkF|23|33-889-944-6438|1322.80|y ironic ideas use fluffily express requests. regular instructions sleep carefully about the slyly +1116|Supplier#000001116|0or65bKUZqty|4|14-523-628-1676|-737.92| the final platelets. regular +1117|Supplier#000001117|o5w8DGLyxgw7oKoZjoB1uWvV5P8Rm5on2|18|28-544-639-9998|3620.41|carefully bold instructions hang slyly. foxes wake carefully excuses. express packages wa +1118|Supplier#000001118|JaLt5YYck8DDtpMr2nQlYEWPKHI|7|17-160-108-4158|5246.77|y. express courts wake doggedly requests. carefully +1119|Supplier#000001119|EmB2voA2Bz34yLqrzj1,t5|11|21-468-735-8433|4459.62|en foxes above the final, regular packages are according +1120|Supplier#000001120|q0wqSfqabg2SIqziLb1Y YM|20|30-865-209-5361|7851.05|ornis. foxes against the unusual dependencies cajole furiously i +1121|Supplier#000001121|MjWB5MI,wMRY8XfkdCmVkLmi8BuAlcSv1ATjllH2|22|32-479-804-7210|5637.21|l, pending platelets. carefully ironic deposits af +1122|Supplier#000001122|d0WAMoTMEtSs8F6TUghy9 dJzbVaHXCkL OiPO|20|30-654-873-7237|-387.57|ously furiously even deposits. slyly silent foxes slee +1123|Supplier#000001123|n5L 4Otbff3oXbr0Ibs|3|13-405-995-4269|7091.28|instructions; bold packages cajole regularly even platelets. furious, bold accounts ca +1124|Supplier#000001124|I ptruqPB5c1felgrY9dsu|1|11-219-426-9501|7147.22|uternes impress. regular, regular instructions nag quickly carefully ironic instructions. s +1125|Supplier#000001125|P1KNMzx4s22L3D|16|26-691-670-8697|1377.60|sts thrash closely. final foxes sleep. unusual theodolites haggle +1126|Supplier#000001126|LnUIRVC9QeubdNCC1ZyMP7mem,isjhBZWG8HW|12|22-523-951-5183|7243.05| special, unusual excuses. special asymptotes wake blithely. blithely express ideas b +1127|Supplier#000001127|5LRB7nY9PHnTdzPf59icKT|11|21-390-257-1851|5929.81|thely silent deposits wake blithely final requests. foxes wake above the furiously p +1128|Supplier#000001128|v CXmGPsW8enaePo5n3OM32zSe DMg|10|20-916-812-8305|329.70|unts mold furiously against the fluffily final deposits; blithely regular instructions boost af +1129|Supplier#000001129|DV4fcTLWjb2mu|11|21-213-197-3416|6175.63| bold accounts haggle furiously. carefully ironic packages boost a +1130|Supplier#000001130|gaw3h9cwNJTDhzgND3Ivew9mM|22|32-181-510-9827|6841.61|nto beans. regular theodolites haggle carefully dogged accounts. u +1131|Supplier#000001131|bPrtcG7uO5xNHFfFh2omVjC|17|27-622-519-5482|1922.70|s. bold theodolites according to the special requests boost slyly reg +1132|Supplier#000001132|0rxuQ74Sqy|5|15-583-125-3480|5825.44|eans-- pinto beans wake above the sly +1133|Supplier#000001133|cnqLejGYqbqrMVlxNiaY,JdcqQkHFYeyfum2Nv1w|23|33-858-158-1956|6206.46|lithely bold requests nag. regular, even requests integrate. requests cajole s +1134|Supplier#000001134|WkRjMQOKvjH4g4RsFZomI9UKUjTw9gq|4|14-183-918-4545|7977.48| alongside of the orbits. pending instr +1135|Supplier#000001135|3bXoqoLQxbG,JGz7Wh2NDeULP2gI4kOv97wF8|11|21-809-107-4714|7628.25|oxes cajole fluffily carefully careful deposits. regular, pending ideas +1136|Supplier#000001136|33cXkwLN,9qMqFfH,yscLe,EssjSrmO|1|11-934-752-5769|1058.12|g dependencies. regular somas boost above the carefully final deposits. quickly special acco +1137|Supplier#000001137|PaOfcxOq69vJcEPjP0dKQ3zh2nhZdeU8,rST|18|28-418-800-2417|5705.25|ss packages use carefully abo +1138|Supplier#000001138|cgnz432oKyPA0g7Y26bYorMwHFjbki pjALkt|18|28-977-762-5445|-825.98|ages cajole carefully except the packages; fluffily ironic re +1139|Supplier#000001139|ig1S4mFmFcNa6A6GqSRSdZViE3myXCcU8vaeduoS|12|22-286-747-9130|8602.51|slowly special packages use carefully iron +1140|Supplier#000001140|8y7JNE7iXHBQhVaXtdye5iKZUI|23|33-927-817-5355|7384.86|equests. furiously regular deposits after the requests sleep carefully special deposits. qu +1141|Supplier#000001141|xCOuPvOd565r9H31,xuZcpE|3|13-287-397-7030|2416.34|arefully special foxes. quickly regular excuses boost blithely. slyly ironic deposits nag +1142|Supplier#000001142|RQ5DqXCWlmO Msp0ZPujoj2D4x|24|34-230-244-7118|8362.41|y regular foxes. carefully silent deposits wake blithely during the even deposits. slyly enticin +1143|Supplier#000001143|Lm h,MliyudNgZYMKKEE1,CUh21kOOdrqdDPZqm|0|10-761-431-1486|9032.45|yly permanent platelets. slyly final ideas sleep. fluffily daring packages haggle slyly regular fo +1144|Supplier#000001144|6cmLWYN1jl pFXSrHuyrFryFWqB|6|16-730-992-3332|6383.22|e slyly ironic realms. blithely ent +1145|Supplier#000001145|GaJB 0FOAPBeqwPA5J7GJcVnT6MvPe|18|28-769-367-1159|8752.68|regular, final accounts around the closely final packages wake furiously along the courts. furious +1146|Supplier#000001146|UKWRbMRr47499Kta|0|10-208-700-8880|1294.18|ructions. slyly pending accounts around the slyly silent instructions integrate against the final +1147|Supplier#000001147|tI7YxKpJOUEQAuZ rCyjKKQwC7F72xs|11|21-843-925-4037|2301.43| the regularly even requests detect stealthily af +1148|Supplier#000001148|yaK1tZuQ8Oqhnll1Doit6kM07LZhmrLw|22|32-341-240-8284|2145.46|uriously. pending platelets among the ironic accounts cajole blithely bold instructions. re +1149|Supplier#000001149|Nuno37wiZOjNGHF|21|31-169-782-3259|9783.31|le. unusual packages print f +1150|Supplier#000001150|Ea5h8EyGzY|15|25-243-708-2531|9170.16|foxes. blithely regular gifts integra +1151|Supplier#000001151|BSqCbpIU0taQWNnD23Z0MhygBLy p0|10|20-604-120-2619|-33.29|ages. bold packages nag. carefully bold courts about the quickly silent deposits doubt above the fu +1152|Supplier#000001152|D19t3SK5IpY5q0gJOf4rYAjjTpBjZlJl3dt|20|30-686-799-5287|-639.90| final accounts. carefully regular deposits wake furio +1153|Supplier#000001153|bmE8XNqYx2n228 QHTeUr|7|17-587-574-9364|1118.61| nag. unusual deposits boost +1154|Supplier#000001154|lPDPT5D5b7u4uNLN, Rl|6|16-156-502-9672|-572.40|e carefully around the excuse +1155|Supplier#000001155|rI94W20Qh,s8pQyfQQqO2pHJd6Qvbx8THji9v5J|4|14-780-531-5179|2527.96|ies. regular, regular deposits solve about the carefull +1156|Supplier#000001156|5UTqs47kipJo7voG3Ecj,M,Ms9fu d2to0AzYH|15|25-305-106-6287|3333.99|c accounts wake slyly regular, unusual instructions-- idly unusual theodolites sleep specia +1157|Supplier#000001157|O69Bgymkxg1mACtNmQ,5|17|27-342-536-4024|9345.76|ular patterns are blithely among the carefully regular deposits. blithely ev +1158|Supplier#000001158|, tVY8orI3|0|10-282-223-1239|3321.96|r pinto beans. blithely express ideas wake furiously special deposits. regul +1159|Supplier#000001159|xe9u2TY09V7MTsl,dC,kS,5kdys3Ongi|9|19-658-920-5724|5130.03|ajole regular ideas; special packages above the carefully final deposits are slyly about t +1160|Supplier#000001160|4Z4HmpP2dhWI8vTzhjFTZGyGHa,7yu|12|22-871-346-4321|362.52|c pinto beans according to the quickly ironic accounts cajo +1161|Supplier#000001161|dO6wpXbKpZjRCRo|20|30-202-116-8696|7738.90|s the packages nag furiously pending ideas. bravely even mu +1162|Supplier#000001162|p5 1bgT,F7cZaIH|11|21-669-219-5577|1751.01| ironic warthogs haggle even accounts. final deposits about the carefully bold requests p +1163|Supplier#000001163|yvREBryfATXdmjXGXzKXyUh9JOCo|16|26-582-974-8266|2181.53|fully regular pinto beans lose carefully even accou +1164|Supplier#000001164|4zRI bPJf5ST2fBFZw,|5|15-830-566-1096|-89.70|y the blithely bold packages. slyly regular acco +1165|Supplier#000001165|Hgzhk7UM2dKPaXk07rZ4Vn91A09 |16|26-181-600-1076|8337.11|ly ironic deposits cajole fluffily: final, ironic fo +1166|Supplier#000001166|PeY0yTTUfEa84Gf|23|33-137-834-2015|1787.49| special accounts was about the final, regular accounts. unusual theodol +1167|Supplier#000001167|ErCT6PR9WY8|11|21-521-753-2637|-655.61| the furiously final courts. final +1168|Supplier#000001168|SF8uvvbHRyOVTb8APmgFEL6|18|28-390-490-4295|1154.17|refully silent packages nag packages. fluffily even theodolites sleep. +1169|Supplier#000001169|bYC92PkKrICY1EfgBm2wcNlX|17|27-259-701-4743|3075.74|ag furiously pinto beans. furiously ironic ideas poach slyly slyly +1170|Supplier#000001170|C5SCYBXvOJATuamJB|14|24-150-726-5287|2864.43|t the final, even dependencies. slyly special asymptotes may are thinly ag +1171|Supplier#000001171|lIEeTSlZFfoWspupjEWITxwraW7sWYqfy|10|20-726-223-9192|4528.08| sleep slyly carefully ironic requests. blithely ironic asymptotes nag furious +1172|Supplier#000001172|1XIgPffEfiZasmaJM20b|17|27-150-360-6842|3453.94|riously even ideas. bold requests doze ca +1173|Supplier#000001173|kkpX42nzsgFHO9VCYjCA|16|26-105-912-4569|1381.06|ly unusual instructions boost against the pains. quickly ironic requests wake quickly after the +1174|Supplier#000001174|TPHtSDcMBog8C1IQVBa2s|4|14-123-849-3557|-597.60|excuses. ruthlessly final deposits haggle against the p +1175|Supplier#000001175|Iv98FgNxQ1e8hWfVM3flCuAEAJt8MbAHLAj9|2|12-637-911-1584|8067.21|: furiously final forges within the quickly regular instructions use against the iro +1176|Supplier#000001176|6pw3sCEqKssTz,CxcSC ENPfwR|22|32-634-542-7820|8622.41|kly regular deposits; idle courts are slyly. blithely even accounts s +1177|Supplier#000001177|2x0ov,dvKpVcgK5PrG7XKHkv4zLIP|8|18-769-434-2561|870.59|press deposits solve blithely. slyly +1178|Supplier#000001178|QAjFojIKZdFJsyoE6vBZP|15|25-232-102-3450|-132.66|kly about the special, fi +1179|Supplier#000001179|l9JHnPjgaxrY11cPiCE|18|28-156-806-5348|1303.98|t alongside of the silent requests. furiously unusual accounts haggle. quickly final pack +1180|Supplier#000001180|CUYcEIZwjc4JJUofc1sw6OLXtwzRN,N yk3,9C|0|10-437-532-9777|1965.64|regular deposits boost blithely slyly final packages. dependencies poach fluffily f +1181|Supplier#000001181|e,Ld995DWYXRrqQRLS9MtmWinb64wGm5JjMhRXF|9|19-963-905-7803|1405.91|express packages cajole after the packages. slyly even foxes use according to the quickl +1182|Supplier#000001182|mSIDktpv g4FhMuhXWjugfe|12|22-403-723-5565|-854.27|ously even packages run carefully acros +1183|Supplier#000001183|XcR1BiGZ0K8xnbF|2|12-456-466-4198|6215.34|sual warhorses. fluffily unusual +1184|Supplier#000001184|gKEtnB,QWGHCcwL2P9i|11|21-880-957-2610|3230.37|uriously unusual foxes. unusual, expres +1185|Supplier#000001185|khazl747u4Qc4h,OL4BqmkEH4LzrmDsu6vq|23|33-194-433-6517|5727.03|al requests. furiously regular ideas are. quickly +1186|Supplier#000001186|Tm4Tcgzh0opEX0|1|11-965-958-9493|2177.90|as use carefully around the silent requests. regularly ironic +1187|Supplier#000001187| j9rBbrIDyaBSLLRH4oMCSpVD7Ln|17|27-361-403-1251|5008.55|final packages! ironic ideas haggle above the carefully pending asymp +1188|Supplier#000001188|5zUkIscDkb|14|24-233-306-9011|4701.05|y slow accounts. carefully express foxes are furiou +1189|Supplier#000001189|xYOLJtZstk3lh 2O8H231cTkSQ8rKbNCC,i9vZY|7|17-828-994-2511|6611.85|ithely even platelets. quickly express packages boost. slyly regular deposits above th +1190|Supplier#000001190| N3Dt82gW2i2qKlZF9uk|11|21-875-802-4348|1349.97|y regular accounts eat upon the daringly express instr +1191|Supplier#000001191|bRGVgPurgzrGQwKlqrLojLtJlzON7tN YJ|0|10-746-144-5600|4824.61|o the ironic foxes sleep slyly pending platelets. final multipliers along the theodolites haggle ab +1192|Supplier#000001192|qMXIndghNz|19|29-248-593-6081|4300.77|inal accounts are furiously pending notornis. instructions are carefully quickly silent request +1193|Supplier#000001193|Er,GdM4NurXk0L|11|21-374-321-5484|55.50|gle fluffily according to the slyly unusual acc +1194|Supplier#000001194|rA0oBAtw1GxPROoJQn1qPMo|16|26-269-485-2796|6737.51|ole fluffily thin foxes. foxes against the fluffily unusual instructions cajole furiously even, e +1195|Supplier#000001195|ktmmzrPFz26tD1AlrKBfe1JL6OaZ4 O|16|26-123-742-5451|1428.62|nts. express, regular packages u +1196|Supplier#000001196|fB1EzFKsyF iBFX7AkImgUp|17|27-437-746-4848|6729.76|ckages. ideas haggle slyly. even f +1197|Supplier#000001197|8DWD2Hby2bbzEu6p fMqSks|13|23-558-564-7859|3184.68|ns are carefully pending deposits. carefully +1198|Supplier#000001198|vRfsLGzF6aE2XhsqgmJFUHGmMHepJW3X|6|16-393-309-4863|7951.58| final requests. blithely even packages affix regular depths. close theodolites boost blithely. ins +1199|Supplier#000001199|Ak51uQVtsJig3QQ7iDEZdBEhBRCH2ACuik|9|19-199-275-7669|1206.42|lites. fluffily ironic pinto beans along the regular pinto beans sleep final request +1200|Supplier#000001200|iNmOfNJF ScWy|19|29-286-106-3390|1224.16|y furiously regular packages. quickly ironic foxes will use carefully at the b +1201|Supplier#000001201|Seh4D7pi9UdK,XQkF46A0O2N|21|31-700-109-8793|2326.92|lyly furiously regular depths. slyly even deposits against the ironic +1202|Supplier#000001202|oH24kcQfvMg1mR46kugriDI837|2|12-542-821-5279|1512.22|after the ironic asymptotes. carefully regular ideas unwind quickly. blithely regular pains d +1203|Supplier#000001203|jHCBBvYw1DqzKRAV7T1bGz|0|10-295-590-8708|3878.15|ully final deposits? bold excuses detect blithely +1204|Supplier#000001204|fROOqRgBPrHagHhP9A9XvGMWSm9iONll|14|24-450-294-6355|3130.81|round the deposits. regular accounts use slyly ironic, regular req +1205|Supplier#000001205|Y9,0L5ec9j2C3DiQ|11|21-322-258-8638|4467.07|ests. blithely even asymptotes across the carefully regular packa +1206|Supplier#000001206|pFMSZD5MiEv|0|10-518-682-4417|1058.29|egular foxes haggle caref +1207|Supplier#000001207|FpOgzEElFhWJDr5OfTAsshWlv|17|27-833-133-3956|8614.15|lithely fluffy pinto beans along the furiously pe +1208|Supplier#000001208|bdkG0vZa7Kgsuj4iOavmPBvdjw26CXgnTyZu |6|16-739-665-8270|5007.57|st the fluffily final packages cajole furiously slyly fin +1209|Supplier#000001209|2jJpTlv5dG9pAsULUFv1J|12|22-547-685-5761|-766.94|iously. blithely bold pinto beans wake blithely final foxes. regular, ironic requests us +1210|Supplier#000001210|vt9s6P1sT9ncs84dMKz4Gful7YLzU|9|19-547-533-3573|7.25|d the deposits. carefully regular instructions haggle furiously about the regular deposits! special +1211|Supplier#000001211|g lldwVuium jP79qGZT6ztUjTuCZmZKrNlcHglK|15|25-752-441-6583|9810.37|ong the idly even packages. furiously special acco +1212|Supplier#000001212|SNWZ4sgC8Gh|24|34-917-244-7693|1893.83| instructions are slyly alongside of the regular pinto beans! fu +1213|Supplier#000001213|e,v9Ub gKIgvaC|1|11-983-419-9576|4528.36| special decoys. accounts across the carefully even requests sleep furiously reg +1214|Supplier#000001214|33rk3mcWaDeMzZcSF|13|23-171-373-5539|8142.32|ests along the express, final accounts sleep blithely fluffily special theodolites? bold foxe +1215|Supplier#000001215|VAqI,tumyKBHkXAIU9v4luexhsYXvH4s|23|33-651-865-2529|5961.50|ular ideas. furiously silent instructions detect enticingly unusual pinto beans. sometimes exp +1216|Supplier#000001216|KCFNQqfBmcB7kk2kaiK YefaYnKvBbeiapFVFOrQ|14|24-453-215-3742|5015.05|l packages doze slyly above the speci +1217|Supplier#000001217|iwoHJbL2n9kAPj2lAJlzIK9Xujbbz6JP1c|13|23-603-542-8035|3970.09|ts. special pearls wake quickl +1218|Supplier#000001218|A bjeBaM3ftph BcnLk,XepDz4MuV4eo |12|22-384-682-3996|8303.21|uffily. fluffily final waters are according to the furiously express theodolites. bold, even +1219|Supplier#000001219|yYAruzzZdw8WCBRCunZoq0|11|21-649-821-2202|4152.19|ual Tiresias was carefully. blithely ironic packages mold q +1220|Supplier#000001220|ZTb9JaYshJrAYaEDBINYzaTAg5co0b258ggYF|16|26-680-291-1875|-102.36|ts sleep. blithely permanent forges along the even, regular packa +1221|Supplier#000001221|QXZgKZvn3lGVagCs,Py7guY9b2cfJ hEvmFE,a|18|28-332-756-9313|4533.75|ve the accounts eat slyly about the carefully express ideas. furiously exp +1222|Supplier#000001222|xYNDEfSSuo5O8wjt7qBs38qkFnvRoljRxdMXlSO|4|14-643-664-5410|-201.74|ffily silent packages around the quickly busy packages haggle even accounts. q +1223|Supplier#000001223|3OAireaY3KvX|6|16-769-448-3298|6250.53| bold requests. furiously special instructions along the blithely express pinto bea +1224|Supplier#000001224|phrbm9zLBi4Lon0|1|11-130-646-8654|681.75| deposits. furiously regular theodolites cajole even, express dependencies. ironic do +1225|Supplier#000001225|EcOF0LtTFCmRbiJMzEAEACH4uYvcxzseopylu|16|26-277-191-6429|5073.61|fluffily ironic instructions. silent, final accounts use carefully silent foxes. ironic fox +1226|Supplier#000001226|qsLCqSvLyZfuXIpjz|7|17-725-903-1381|9189.98| deposits. blithely bold excuses about the slyly bold forges wake +1227|Supplier#000001227|cXdKu5KSQ1yaF,EiRLpcP|23|33-515-927-3294|3115.27|ithely even packages cajole slyly. slyly +1228|Supplier#000001228|8xkbypW3UaiQvRe9Avx9JJgHV9CstYiCF4|10|20-663-301-2975|7316.91| silent requests. quietly regular requests sleep blithely quick dependencies. carefully iro +1229|Supplier#000001229|WVPJeTRrbGr1C08izy7oaVt2 srgX3znhZ|23|33-448-290-4429|4379.67| carefully final dependencies along the even, iron +1230|Supplier#000001230|YTNwF6K9kaxyH8NGGrD9G|5|15-922-198-8652|6068.43|y final instructions. regular platelets hang bl +1231|Supplier#000001231|zP989Ly4bXqz1m,dXRMLpnUiQigImQQdc6u|17|27-192-272-3256|9202.60|ideas. final, regular foxes sleep afte +1232|Supplier#000001232| pjlAf6tpYw4eDCwY|10|20-443-322-1450|6112.13|g to the final, final accounts. pinto beans nag. deposits haggle furiously. even foxes wake +1233|Supplier#000001233| V26hJzvCaekpg9WP8DL7f,AXiL|10|20-440-730-2308|6600.16|thely even accounts around the blithely ironic deposits play pending, regular dependencies. +1234|Supplier#000001234|pbSHCpwfeRTI0amiSd2vmkK6,io5P29uCLWv1qds|21|31-946-737-7255|2856.29| according to the quickly final instru +1235|Supplier#000001235|DN xKA4F4qK2n4YEZe,3 2wR|13|23-387-665-5120|-437.93|y. regular instructions acc +1236|Supplier#000001236|qpYv0hQmP8aiXnIAm1o|10|20-247-189-6514|7606.44|s; carefully unusual deposits affix quickly. silent cou +1237|Supplier#000001237|KskDRrC5imTdN0DCTm3BykGJWZzsNumC|5|15-830-628-9526|8480.34|er the fluffily express deposits. ironic requests poach blithely +1238|Supplier#000001238|o3IxY ehZtQ64ubysr84V|18|28-223-865-6534|159.36| after the quickly ironic frets. regular, fi +1239|Supplier#000001239|w7zZA8K5TcHMmFtojh3uGVOPHTaPIwE|6|16-805-237-6265|3341.97|e furiously regular foxes boost blithely against the regular dep +1240|Supplier#000001240|2qtAAK9sdgjLXrUPXHKqhKk5|13|23-199-378-9923|427.43|gular asymptotes haggle theodolites. fluffily special warthogs nag blithely. sly +1241|Supplier#000001241|TsZ TYfcF5d,PRf9|8|18-845-548-4756|9706.86|lly special theodolites will have to use. regular packages are slyly carefully special reque +1242|Supplier#000001242|G61HMzAXlu,1AplHHDbLnRJH,qh53XsSrtdoN|9|19-424-816-2448|222.88|arefully regular requests. accounts nag furiously regular dependencies. carefu +1243|Supplier#000001243|SvosmhflASO pHHbybL,VNT 5cyXZp1HNaxnW35|15|25-455-924-8412|3851.33|doubt carefully along the final foxes. furiously ironic packages cajole sometimes ac +1244|Supplier#000001244|K8PU 4SwFaJaory4RzC3 o8R9|19|29-105-609-5897|3295.40| the foxes; ironic, bold excuses boost after the bold re +1245|Supplier#000001245|bztTCfHZnsLafgtZWe,isV Gr 4IFg|3|13-597-902-8280|9851.62|s. accounts haggle slyly. f +1246|Supplier#000001246| RZdhF4CPJneS0VsFpBTAcltHuwpEaxQpEYj|15|25-726-260-4668|2334.21|special, regular requests bel +1247|Supplier#000001247|szRGANkh3vAfAV|0|10-650-158-9355|3036.54|nal requests wake slyly against the blith +1248|Supplier#000001248|LBMD4udQVK0txG2UE2uO2u6bmq9SfYd57rgkSAw|4|14-263-397-4453|4175.17|haggle carefully around t +1249|Supplier#000001249|aoY82cGgJZpmWsBQaPARId5|18|28-505-573-4475|999.08| are. express, ironic braids cajole idly along the furiously regular th +1250|Supplier#000001250|uum91lnpIZa9Yv3Z IMXbf7TyYlwPYI1ApCYtx|5|15-552-400-9891|3509.03|counts. ironic deposits haggle +1251|Supplier#000001251|6vY5KGVIl6VW8dM0uGAM9MgVtgEl9KT21jYiS|20|30-842-722-2771|3178.26|gle slyly across the even excuses. bold pinto beans haggle. special requests haggle unusual, b +1252|Supplier#000001252|K4NQhYOSL2fnDgLS3|18|28-109-976-7603|9962.01|n instructions integrate furiously carefully even instructions +1253|Supplier#000001253|SvNEki89cFf2tMquxN,LdGLH,1vvy3h06|0|10-918-255-4434|9575.73|! blithely even platelets thrash fluffily. fluffily regular accounts hang upon the furious a +1254|Supplier#000001254|NHNCT2HbOpSNm2gPz2Lqh|16|26-486-845-9440|7291.20|y ironic packages. blithely express accounts sleep furiously c +1255|Supplier#000001255|I3YzBXBkSGYEV4dIEt76GMLWhIP26AWqrfmiBUE|14|24-621-423-9439|1294.77|y final dolphins. packages boost deposits. slyly ironic dolphins are alongside +1256|Supplier#000001256|CledM8jaL7bzwwTmeP3xDLv7MRUpACQx Qf|4|14-524-830-5425|1132.75|above the ironic platelets. regular, bold requests detect carefully. silent deposits integrate furi +1257|Supplier#000001257|FEcdXx9dQIygWfKnJMi,hl3|13|23-545-989-4692|6021.09| are slyly blithely pending ideas? unusual, regular dolphins after +1258|Supplier#000001258|JGMy3IYbAxJNQDH1zn6pw8EhnYSQ8qwBSW7|9|19-950-879-1357|5565.74|kly even, enticing accounts. slyly expre +1259|Supplier#000001259|32cJBGFFpGEkEjx1sF8JZAy0A72uXL5qU|11|21-427-422-4993|8375.58|ironic accounts haggle slyly alongside of the carefully ironic deposit +1260|Supplier#000001260|D9vpfn0l3JdOg1HD|9|19-431-553-8802|-550.54|ts sleep furiously. slyly even +1261|Supplier#000001261|cDxqWVt0QNlfIdZRciz|17|27-516-126-8540|7072.86|haggle slyly along the quickly even accounts. slyly final pi +1262|Supplier#000001262|xxUP5uBcJ6|4|14-305-294-8872|5530.51|y regular requests are. blithely ironic packag +1263|Supplier#000001263|rQWr6nf8ZhB2TAiIDIvo5Io|3|13-620-886-5427|-650.10|ss the ironic theodolites. blithely regular accounts solve. slyly unusual packag +1264|Supplier#000001264|GCfirAZS3kqchx5T7fFzH5qEDCN82iaZu|10|20-989-349-5053|5728.44| carefully final deposits integrate carefully furiously ev +1265|Supplier#000001265|5OzDMA8VgiJeximuX|11|21-513-678-3138|3986.63|ven requests among the bold deposit +1266|Supplier#000001266|lwHRpzssxFZW9|10|20-985-871-8162|218.47|even packages according to the special asymptotes cajole +1267|Supplier#000001267|LTW0fttLJZM 4TDrgiUssql|17|27-423-854-3318|6741.39|ly. even foxes use furiously fluffily bold deposits. regular theod +1268|Supplier#000001268|W6nIT0xqfg1|2|12-446-614-8039|516.58|he platelets. furiously regular platelets cajole furiously at the final ideas. +1269|Supplier#000001269|wsNVDR nlBgHbyQi8hHzUPk6ZRwd7|4|14-158-967-8025|1087.17|ymptotes snooze furiously blithely ironic requests. ironic courts wake caref +1270|Supplier#000001270|KCFX9tQ sql2ed6fy|1|11-361-350-6820|-931.30|its integrate furiously against +1271|Supplier#000001271|hKhoKhSiqgKKIfEF|12|22-869-665-6701|9480.47|s cajole carefully bold a +1272|Supplier#000001272|s6ACcmRU,Iv|7|17-399-346-8089|1272.52|ar platelets serve slyly. instructions agains +1273|Supplier#000001273|tyy7K3b5, 76lCMtSPkfPXc7BA6Qp|10|20-911-524-7070|1202.46|le-- ironic, ironic packages sleep. furiously final packages haggle quickly. regular, exp +1274|Supplier#000001274|BhaVtechwkjnu2Bl9SOj|20|30-238-172-4433|642.45|. fluffily ironic foxes promise carefully ironic requests. carefully b +1275|Supplier#000001275|UDPJ3Nhq1YnrOrYMKlug48KwfFGFStAsIE5|11|21-179-324-2213|4535.73|atelets nag. bold accounts instead of the final, even packages use slyly alongside of the +1276|Supplier#000001276|R5qlyLCMRzeMVw5gofE7QqA5vNgCAOItAEqO7qIu|7|17-879-359-9140|40.82|ourts. bold foxes are furiously a +1277|Supplier#000001277|o1TrsGXKksJOyWcY|0|10-211-466-9198|1926.44|y pending accounts use furiously according to the fluffily ironic requests. carefu +1278|Supplier#000001278|fvx7QQ5hGaBLdlD|8|18-641-214-4344|2577.01|gside of the unusual, final requests. excuses boost slyly. furiously regular pinto beans snooze bl +1279|Supplier#000001279|9tXl9nR6SxUgFCe80RbhQkpiYK HTEi5y1Dv|10|20-874-743-5507|2719.51|lithely. furiously pending deposits sleep carefully fluffily special excuses. blithely unusual c +1280|Supplier#000001280|R5rJqsrqclgZpLJ5QCPEg3x7VpfqTb10XNaam F6|1|11-150-679-8962|-239.73|gside of the slyly pending accounts wake dogged, s +1281|Supplier#000001281|xSwSjAwhonVYZtC1oN2O|23|33-432-130-8820|5791.91|hin accounts are permanent, ironic asymptotes. unusual warthogs w +1282|Supplier#000001282|UA gsripU3RMLvlJ|2|12-744-273-6433|3653.60| against the quickly final pinto +1283|Supplier#000001283|JexqsJ5tcT2z8GeJUzhx0w9GaCNLb90|21|31-392-289-8673|2016.62|deposits. quickly even deposits cajol +1284|Supplier#000001284|vE2U7NoqBkspwrOZFQoXoTBRuDy|16|26-812-750-4830|307.31|gular ideas detect furiously pending theodolites. pending, re +1285|Supplier#000001285|6GzzLGh7I9P3LhBWnTz,L2gECjp1P1I9mq4TaaK|6|16-140-738-6321|3820.96|lithely regular foxes. blithely ironic requests cajole slyly +1286|Supplier#000001286|YKA,E2fjiVd7eUrzp2Ef8j1QxGo2DFnosaTEH|7|17-516-924-4574|9870.78| regular accounts. furiously unusual courts above the fi +1287|Supplier#000001287|8OpFL7nZsfGbAX1V9q8c|0|10-112-209-7433|5276.43|ly bold instructions-- quickly even deposits mold furiously. p +1288|Supplier#000001288|zE,7HgVPrCn|6|16-646-464-8247|7980.65|ully bold courts. escapades nag slyly. furiously fluffy theodo +1289|Supplier#000001289|xrtlwRNxzz 8J,6|8|18-840-902-3456|5843.53| sauternes wake foxes. fluffily final instructions sleep blithely-- slyly ironic depende +1290|Supplier#000001290|Bvnw5RTAURgqNceiR7a00JYiyM1Tdyt|6|16-979-442-3949|2275.53|ess packages haggle furiously among the regular, silent packages. even courts wake reg +1291|Supplier#000001291|fUt0REYwFPK2rR8wjESb dZ,1sBn|2|12-277-232-4134|6759.44| sleep blithely carefully regular foxes. regular pinto beans use evenl +1292|Supplier#000001292|Bjfw20ywjwGEY3zRv9OAS 63rqoRObND1etPnUf|13|23-341-490-2439|6866.27| pending, pending theodolites alongside of the regular deposits haggle furiously silent ideas. f +1293|Supplier#000001293|p3ygA5wAuoGH7a8mqwGpvQycLu,ag|10|20-132-692-3206|9441.78|ic requests. carefully regular excuses according to the regular, pending foxes integrate blithely f +1294|Supplier#000001294|BvnGVXACBisiqCr8EsDa|15|25-658-545-7916|87.92|ironic deposits. bravely final requests cajole carefully whithou +1295|Supplier#000001295|Vd7EojBhBkzJunfg2e6vgPKfqj81mtv|4|14-271-709-3448|1578.62|ke according to the even theodolites; carefully regular pinto beans afte +1296|Supplier#000001296|dVrQR8phPaMmr3AABR4CrbcvG5|7|17-626-445-5639|8821.78|es! blithely special instructions cajole. blithely special requests haggle furiously around th +1297|Supplier#000001297|khvyfuPS1FV1MEhh7FaKyrnUn9D|14|24-442-929-6552|7754.49|nto beans cajole blithely. slyly bold accounts unwind boldly ab +1298|Supplier#000001298|4imjXzdv0Ydsu4b2FXl4QT,Gv|16|26-903-838-5675|-291.22|packages. slyly special accounts wake carefully bold theodolites. furiously bold foxes ag +1299|Supplier#000001299|3Le1h90D8qPIW0fUaT1ZpSM5vJ28FB3a|14|24-375-891-9460|9376.04| pending asymptotes. carefully regular accounts +1300|Supplier#000001300|UqX4pArRmxHi3LSB|23|33-890-567-7030|6124.81|counts sleep regular theodolite +1301|Supplier#000001301|VxrmO6Eyn7uNjaSaiFiPN 3ugFx|21|31-187-331-8224|-292.15|riously pending accounts haggle +1302|Supplier#000001302|CnA5dkqpoQq cUp|13|23-807-890-6717|4692.41|ach blithely. bold decoys kindle slyly carefully final a +1303|Supplier#000001303|7Kij80Aft2L WAdyeH,zh9rSF1 tU0RYo2Nh|12|22-688-457-2776|4368.88|usual accounts according to the ca +1304|Supplier#000001304|KlLipfqqkc8Ktz4CwZlYl|5|15-452-488-8513|689.25|ironic packages. ironic ideas unwind thinly even deposits. bo +1305|Supplier#000001305|lGVHX2h8ASYiLtSnXZQBq8s|4|14-959-834-7559|-863.50| the blithely unusual packages. fluffily bold ideas boost en +1306|Supplier#000001306|Ad4ljVj851hzx9F|2|12-192-484-2306|5061.76|gainst the carefully even accounts wake about the e +1307|Supplier#000001307|MPZXf,9fmn0H5Bgtfyv2c74pCy|17|27-897-389-5613|8084.68|lly special requests. regul +1308|Supplier#000001308|AnhN44X4KIZoTUAgc 73R0bMS|8|18-839-553-9361|5218.96|. ironic, final dependencies wake slyly. carefully final pinto be +1309|Supplier#000001309|72RNUzKzbniUnnsSs24ZzGDvmcv2Pd|21|31-910-723-6054|-851.00| boost blithely. requests are. +1310|Supplier#000001310|,reTHSiknxGFziNNyokCy2jUpzwXRl|24|34-340-834-5192|6090.09|about the blithely final pinto beans are carefully within the u +1311|Supplier#000001311|RkSRZhG0WUBIvJSU1X9js0hOmfx6SL6b1hmKW4bf|0|10-472-262-9154|-805.77|dencies. slyly special instructions wake blithely-- fluffily final exc +1312|Supplier#000001312|Qp YSPsVy9ZGyzK3|19|29-634-823-4679|7311.31| carefully final packages. fluffily regular theodolites impress bli +1313|Supplier#000001313|8Rh,5SSqCH|12|22-203-596-3566|9209.53|ckages wake. slyly ironic instructions wake accounts! quickly ironic epitaphs hag +1314|Supplier#000001314|aJDmqSRKvJC1Kgj,ceIfkh|15|25-594-221-9764|1681.78|ously regular pinto beans. quickly express depo +1315|Supplier#000001315|tG2KWW3xDZaTc22|13|23-296-446-2673|4634.14|ptotes about the carefully ironic theodolites haggle fluffily even theodolites. f +1316|Supplier#000001316|aQM,o55YRkzb|9|19-604-439-2043|3926.37| ironic instructions of the carefully bold platelets are furiously even ideas: car +1317|Supplier#000001317|SS8UKbD2960F1hLK5X97J1233,|23|33-841-922-3781|-431.55|s lose quickly special requests. final pinto beans sleep. ruthlessly final accounts nag. +1318|Supplier#000001318|pUYV3GlBkN9O4w,u,hc0o|21|31-779-241-5392|4839.36|ts. blithely ruthless theodolites play according to +1319|Supplier#000001319|BmPKP5ZotUl2k8lR0J5VNovROzp HFD|16|26-578-880-6781|-830.78|er the fluffily express requests must have to nag instructio +1320|Supplier#000001320|QotM4v9vsoLqNSw6LbpqdC,sb8re5GFzNe|8|18-732-433-1443|4925.36|ole. quickly regular deposits may haggle. deposi +1321|Supplier#000001321|vz9IwHx4DLiaFXFveKzv2E1|22|32-708-579-1992|3538.01|cies. quickly special foxes cajole +1322|Supplier#000001322|vlYKC9ekkzEycaHik|5|15-363-480-8363|6403.15|o the furiously bold pinto beans. furi +1323|Supplier#000001323|PAh52FWueSB04tXU|7|17-522-987-4040|2808.75|tes will have to boost above the carefully final t +1324|Supplier#000001324|DZCfy2VGbQK9iSfAK|2|12-786-643-1313|3561.49|excuses. blithely final instructions use slyly ruthless requests. slyly bold din +1325|Supplier#000001325|5WM czPfNenJJoIQOfmUFlOh1W7V0FzBOvk5o|14|24-712-179-2193|4980.85|ers nag always alongside of the pending hockey players. packages haggle. fur +1326|Supplier#000001326|bRQddxqzSIWF7QnfmlMZDV1tdHR|0|10-828-513-1019|1207.01| wake instructions. furiously furious instructions must have to wake acro +1327|Supplier#000001327|muW6V1krHMhQ T9mkugDpE|7|17-564-249-1981|218.44| the sly pinto beans wake blithely ironic theodolites. ironic sentiments snooze according t +1328|Supplier#000001328|GMmS3QncZY9r5znw5pAZjgMudX5IobDzLE4Dz|8|18-226-676-9265|6213.91|eans maintain slyly alongside of the ironic packages. furiously express theodolites against +1329|Supplier#000001329|1DD sDixDoCQwZTn|24|34-807-184-1989|1264.27|le furiously. regular ideas haggle slyly platelets. slyly +1330|Supplier#000001330|SId6O3H5X kgMUvmNMQWAzpACakbqenDohX4Acr|11|21-579-249-1360|5555.27|ven, regular packages. furiously regular dependencies haggle blithely ironic deposits. re +1331|Supplier#000001331|6 n,NZ875vge3mSHRgD,|6|16-106-285-4570|8425.75|ts alongside of the foxes haggle t +1332|Supplier#000001332|8MAqYAB7PhOvyTdhzaGV9SLvt|16|26-378-727-6261|6566.97|y special forges sleep. theodolites +1333|Supplier#000001333|gKunwze79R7QX8j|7|17-962-950-1375|6080.24|sts was. blithely final theodol +1334|Supplier#000001334|u3lmHGf,NrD|15|25-818-546-8787|9536.45|lithely across the doggedly even ideas. furiously pending deposits breach a +1335|Supplier#000001335|s408vkeaaJB1P2J1VbH |11|21-137-592-3243|7197.12|lly after the furiously stealthy pinto beans. carefully ironic p +1336|Supplier#000001336|g8umUPg62XgXBVaRwm8ANdagPoIDF,E|20|30-604-756-5958|3995.87|ly final platelets wake fluffily about the instructions-- ironic requests +1337|Supplier#000001337|wike85v4NC4|20|30-475-141-1305|1543.71|kages wake slyly instructions. slyly final packages cajole furio +1338|Supplier#000001338|swI DIAuF0vaY6jORaD7snVV5C,sQM|9|19-362-282-1978|5036.92| against the blithely regular instru +1339|Supplier#000001339|5c4jzbVZLzvxrok|24|34-581-968-4641|8471.77| packages cajole quickly above the carefully special requ +1340|Supplier#000001340|5FNwQfJ5nR4eJvEieJy9XA oY|8|18-205-441-7392|-851.92|ly unusual pinto beans. ironic, regular packages across the quickly final theodolites are above +1341|Supplier#000001341|yv2NImPAHKZrk,Bb0VfZP1193Q1|22|32-219-885-5812|1703.74|nal courts. enticingly regular accounts use above the closely final theod +1342|Supplier#000001342|PLHRQAf4AK okYJGKWODElzeupU4y,ijWOd0Q2q,|0|10-614-132-7491|-361.90|ans! requests haggle blithely fluffily final ideas. close instructions wa +1343|Supplier#000001343|e3NKukdF0m3Ecc7iuIbk5KEif8B8eGYghOcQm|24|34-518-260-7874|9449.66|xpress requests nag against the blithely even courts. fluff +1344|Supplier#000001344|6iF,zVDNTykohVKcb7FKvn82s74ez|21|31-213-133-9448|561.58| slyly ironic requests sleep furiously. bold, even packages after the silent account +1345|Supplier#000001345|gc,ysJMB0DxN77SAyweKRXl3b,opUqZTHV|12|22-581-403-5559|9472.64|ly even asymptotes mold furiously. blithely regular foxes +1346|Supplier#000001346|Q5qTZ1vLbfaotEonpMxKf5V|12|22-426-840-6527|1756.35|ructions snooze slyly furiously ironic dolphins. ironic +1347|Supplier#000001347|9kz,LqAWi1VaotiC kRwSYokpaCrDU2B7GDoC|20|30-156-120-4484|3904.50|beans print carefully. carefully pending ideas sleep slyly. carefully bold courts boost alon +1348|Supplier#000001348|FA39Bg0jdkCczUKsM1aNng|10|20-477-577-1250|5974.21|en deposits haggle fluffily quick accounts. regular, bold theodolites cajole. pending, ironic dolph +1349|Supplier#000001349|EGhB0CmnI35wLfZaVQ9|22|32-303-475-7333|9560.15|eposits boost regular excuses. slyly regular foxes cajole fluffily quickly p +1350|Supplier#000001350|JeFDZXbYECk|13|23-431-834-8413|9584.73| slyly regular deposits. never regular instru +1351|Supplier#000001351|zXdoBMmmRx1wOD7GKoHHBtemXGuYKLDb,U2KP|21|31-971-224-5549|4647.09|fily special packages cajole blithely silent th +1352|Supplier#000001352|7JO8jWge6JfhYb NYQ60piM|8|18-242-440-5683|1649.52| among the pending asymptotes wake silently after the express, +1353|Supplier#000001353|5o3mbJp gcUks7jnAJv7O3h2t3WTdZT9ovM4L6|20|30-139-819-1786|1447.45|ven ideas boost after the unusual ideas. furiously ironic package +1354|Supplier#000001354|Z3rwfjGIybZu399XSQMDDVMVTzcoakpx5oMfe|5|15-626-165-4902|4284.04|foxes. regular asymptotes x-ray carefully regular, bold request +1355|Supplier#000001355|OmfBiXdFaTc|2|12-815-691-5142|4911.61|ake along the quickly fina +1356|Supplier#000001356|cRqdKTBb5I1ibGGCnAvnGKpeC5fOaSDkV|20|30-964-855-6270|3469.96|in bravely. quickly regular foxes play slyly above the special, regular +1357|Supplier#000001357|OCkpJFqsRW63|24|34-899-141-1216|6679.29|f the regular excuses. fin +1358|Supplier#000001358|MI,jpwfa1iSVDDJUtThNB7M3RuPZnnE|16|26-489-989-1065|1861.48|ld pinto beans. regular deposits cajole slyly car +1359|Supplier#000001359|iDX4nJSNdkVO2FNGAa8,7VDkUARO|13|23-167-859-3900|-152.50| regular depths doubt carefully. T +1360|Supplier#000001360|elN50rcw Ktxr2rzSF9rsUto8FNijhOjO|1|11-456-537-2079|8831.62|es. carefully pending ideas behind the doggedly unusual theodolites maintain busily ac +1361|Supplier#000001361|ir3ySC2MMfyEw5mK9U01r|15|25-805-202-6041|8158.98|deposits sublate furiously idle requests. regular requests solve. bold in +1362|Supplier#000001362|3l5jqvUos9Zbu|0|10-584-690-8765|-2.93| ironic theodolites haggle after the special accounts: final, +1363|Supplier#000001363|gn CXPzlZVpy|0|10-409-763-8909|3690.27|he slyly pending dugouts. blithely dog +1364|Supplier#000001364|hZloe XgNqB5f06b74VCiDRd1bea5tMvc,Q1ACF|14|24-101-128-6679|2427.32|equests. theodolites nag! attainm +1365|Supplier#000001365|NysYJeiKia3El,ZtdccG,fwqe|11|21-621-404-5417|5891.72|long the slyly unusual foxes. st +1366|Supplier#000001366|t0m50fUcmWC8ak3ZUmM|23|33-398-860-6097|9189.82|fily among the blithely unusual tithes. ironically ste +1367|Supplier#000001367|42YSkFcAXMMcucsqeEefOE4HeCC|3|13-725-732-3458|331.38|es are. even, regular requests detect; carefully +1368|Supplier#000001368|YvFoaTqE0jQbfYeHBEuOQfH,QufJXD|21|31-486-424-2686|1404.91|ress packages haggle regula +1369|Supplier#000001369|NWOVSURnsQ5tfYI2CsAItodX4dG|14|24-285-954-7293|4277.33|ncies. blithely daring pains sleep carefully. warthogs u +1370|Supplier#000001370|,H9mpO11e8m2WKoAbxTg|22|32-377-514-6171|2005.96|lar deposits. pinto beans after the requests dazzle quickly carefull +1371|Supplier#000001371|rcGLCrh8vvL6mQCGQ50F52hYY CR3 kycXKgh60|17|27-738-276-9354|8166.88|beans. regular, express deposi +1372|Supplier#000001372|N1Lx,wLzbTujlrznIDS|22|32-492-818-9099|796.84|d dependencies do haggle slyly about +1373|Supplier#000001373|1EtjxE6YcZ0|7|17-403-116-4558|3500.86|ess packages. slyly final requests could have to haggle slyly regular theodo +1374|Supplier#000001374|uHLiKIkGMCK39DBiPLySDhM1|4|14-105-774-9435|8924.44|pecial deposits along the slyly unusual excuses wake ironically +1375|Supplier#000001375|Y3olicCNxG1jeNRqlx|18|28-156-731-1843|6075.43|al accounts affix. slyly blithe dinos along the pen +1376|Supplier#000001376|RRPIyyvGjLfRy20DmcdsEl6|6|16-127-723-7329|5800.50|final accounts maintain carefully final deposits; stealthily bold theodolites +1377|Supplier#000001377| UqDARB1LpswZQirXRMsde|8|18-578-693-3940|5106.87|foxes sleep about the furiously bold packages. slyly pending r +1378|Supplier#000001378|cyTlnVEoMOYxy1JjQ1jm2Wh6vr3ycFrSI21jiO|18|28-787-963-1462|7119.90|furiously final packages detect carefully ironic requests. carefully unusual gifts are blithel +1379|Supplier#000001379|c60gSoCo9bUv8vd7aILFp5|4|14-392-742-6536|4151.60|ackages sleep slyly regular, ironic deposits. blithely idle inst +1380|Supplier#000001380|ROSZK39fNCa1|17|27-767-854-8761|885.71|sly regular pinto beans haggle slyly according to the express asymptotes. fluffily final pinto +1381|Supplier#000001381|tvRoMpnWdQ8gWwsNo4Da|24|34-642-738-4308|1029.05|ld excuses boost slyly ironic ideas. bus +1382|Supplier#000001382|g5ztc13RCzp1h|24|34-626-282-4782|1156.21| packages about the furio +1383|Supplier#000001383|HpxV1sNupK1Qe cNH0|6|16-583-139-3061|4317.37|unts kindle ironically. quickly ironic deposits boost ca +1384|Supplier#000001384|fjgJwG4DViJrxMxJbO2kS2|6|16-195-562-6135|3996.68|ages. ironic, ironic packages after the carefully even re +1385|Supplier#000001385|UUD4ymFUhLSjsYHaBwlWf,1zdlh1vFFQH6Tqo|13|23-503-768-4613|4220.75|lyly regular instructions affix blithely ironic requests. fluffily regular re +1386|Supplier#000001386|iNxyxvmqmNB|5|15-186-107-2835|8082.42|nstructions nag among the unusual theodol +1387|Supplier#000001387|rhjiTEU33edkiQ5BFQsHLQ1h9Zv8Vmb|13|23-419-849-2677|353.03|nts. quickly unusual packages about the even, special ins +1388|Supplier#000001388|v0fVFlu5WyRtsqaZxvEBNH|15|25-563-139-2747|2313.97|out the slyly unusual platelets cajole quickly unusual packa +1389|Supplier#000001389|,46GsCL39UI3iVUDe8sTTqTPy|24|34-273-968-9545|3884.38|instructions alongside of the regular excuses wa +1390|Supplier#000001390|C12jsyvOT9qCpq Y|20|30-135-678-9353|8734.79|tes. slyly final excuses haggle b +1391|Supplier#000001391|hkWoAM561QlLjBNk,SdFdIgFx|21|31-636-911-2012|8692.13|tes. ironic pinto beans are. furiously pending requests thrash ironic requests. regular p +1392|Supplier#000001392|h6vpqedAQqjbMYHf0bYgl,BkLl0|10|20-693-215-9434|6707.32|n, pending sheaves. final, express packages according to the packages wake about the re +1393|Supplier#000001393|CnZ79ciDbd4|20|30-366-849-7076|20.34|nusual accounts are fluffily special foxes. carefully final theodolites are furiously. slyly iro +1394|Supplier#000001394|T4czIZ hTOWX2g,8AaSeMKn|23|33-490-926-5968|9738.88|uriously ironic accounts. careful +1395|Supplier#000001395|,ost3cczaEuPkvhSitkUPr3pGraXk0xNg3WZ|8|18-631-127-1559|-588.46| special pinto beans solve slyly doggedly iro +1396|Supplier#000001396|G,oUQvwTiNudoQX|13|23-927-155-8644|7729.96|ct carefully regular ideas. furio +1397|Supplier#000001397|8EdI75YBpNagM4ztO6yU7PKDGcIq|11|21-333-798-7814|5529.44| the furiously final deposits. slyly unusual accounts sleep. +1398|Supplier#000001398|H1l294pHv2YCA2hQztBZsLGsBmhVBRRh|6|16-465-263-9578|3387.64|final packages. quickly special deposits +1399|Supplier#000001399|LmrocnIMSyYOWuANx7|3|13-337-788-5605|9008.56|y final requests haggle carefully carefully express packages. furiously express deposits nag carefu +1400|Supplier#000001400|i6nWwfxMJ4gIaenOIIt9UThgqj|7|17-876-117-5908|989.69|refully final theodolites +1401|Supplier#000001401|hrguVJZAGvKvT2MFdj2HECB3gDtNiysR02Pd|23|33-586-658-5202|-406.06| use blithely above the regular instructions. furiously blithe depos +1402|Supplier#000001402|v1m55JSrqZva3dgy|1|11-772-363-1023|4318.64|y blithely regular requests. pending, even theodolites unwind quickly acc +1403|Supplier#000001403|f,SYhItYUfLM1XY2LKRuWlGQV4G79BpNXY |8|18-126-355-8171|3382.83| boost fluffily regularly even deposits. furio +1404|Supplier#000001404|FzktwABs1P,AmZCOeevCO0fi|0|10-719-682-8224|5223.72|t among the carefully regular requests. bold, final +1405|Supplier#000001405|te5joWcj1Wsc pi|4|14-348-753-2527|2051.33| ironic deposits snooze quickly +1406|Supplier#000001406|xvfVP4UqVFTqXGjedP7U86UuOU1QzQla3Wj4O|8|18-819-576-1262|695.92|odolites play daring excuses: blithely final deposits sleep quickly final a +1407|Supplier#000001407|WK03co4CUF2cG2,hv rnQ|0|10-436-924-3833|1280.51|ages boost carefully express accoun +1408|Supplier#000001408|JI9kGjt23U1npVcOuEpWFpm|16|26-919-551-9560|3501.92| unusual braids boost above the carefully ironic ideas. quickly special depende +1409|Supplier#000001409|VfixB5ke,JR6T7tn7cv,G uBn12IBUw7HWy |11|21-416-486-4797|4723.99|sleep fluffily special deposit +1410|Supplier#000001410|PKAsnwIIXk|9|19-295-387-9337|7124.69|thely regular instructions wake carefull +1411|Supplier#000001411|6rVS6actf5ztxVtq|14|24-192-595-8487|7762.80|fily slow asymptotes are carefully according to the blithely final theodolites. blithely special f +1412|Supplier#000001412|XeiPrl fw,N5doHeADl8md4kE q9tkCatjNoD|2|12-927-942-4044|7395.15| blithe ideas. even, unusual excuses about the asymptotes are slyly quietly special foxes. fluffi +1413|Supplier#000001413| fVaD46c59u|24|34-243-851-6239|-688.21|oys. deposits sleep furiously. requests after the always final theodolites prom +1414|Supplier#000001414|mOHKvfkuuCQXiGO1EzYJFyCqCahY5,fn sQwehh|8|18-900-205-6049|3123.86|eposits. blithely express requests after the ironic platelets wake against t +1415|Supplier#000001415|N,XLjJ8gysivwWBfUOgluVUGM|16|26-475-658-2523|8691.58|s. carefully pending requests affix blithely. ironic deposits impress carefully blith +1416|Supplier#000001416|PYA5L32bQcST4 vGIH,VEEbSIuAxk|13|23-420-251-7666|8448.94|inal deposits. special dependencies boost slyly final ideas. eve +1417|Supplier#000001417|ngxGU4WeFjbUPPSEzfJhL|4|14-311-119-5389|368.48|lyly careful deposits cajole alongside of the blithely final courts. +1418|Supplier#000001418|FVmNlHh2wrXJMQUDZqnp8GNIlQCnmHHzplBS,CF|0|10-133-287-9555|9044.07| regular deposits. regular frets do +1419|Supplier#000001419|9wbge8NxXVd5LbIlNA,DjR0sjasMyxC7oANgSQw|22|32-865-313-5333|3689.03|uld have to affix quickly after th +1420|Supplier#000001420|v7306BzuNRhYkR1SWyUxuCi5|16|26-377-840-4186|3472.82|ess theodolites ought to snooze around +1421|Supplier#000001421|NsA09mqAFTnTD6SMdDpLSuRfq4LwHT|24|34-950-623-1049|-276.08|ickly after the even deposits. special, fin +1422|Supplier#000001422|J48g9qobTEuBQPvZa6DH3TEHlL1VD11xtutv36pF|0|10-968-396-2949|-123.14|lar pinto beans use carefully fluffily unusual pinto beans. slyly regular asymptotes wa +1423|Supplier#000001423|NMfB5m6mUZaUrq5BQVixbffpKmHmLcZo|2|12-727-158-9103|7377.86|nt grouches hinder regular courts. quickly final accounts sleep carefully. bold, ironic pa +1424|Supplier#000001424|LNOsByzhAnOy8AY8VEByhbs7lp5|10|20-558-793-6049|5682.58| requests. packages haggle furi +1425|Supplier#000001425|bN7yR,f6FV9rdf5Ds|2|12-710-340-6871|3134.75|bout the slyly ironic ideas. ironic dolphins cajo +1426|Supplier#000001426|bPOCc086oFm8sLtS,fGrH|3|13-561-673-6974|4715.53| blithely pending ideas. slyly even foxes boost quickl +1427|Supplier#000001427|ouHXuQHr3lURlvT2w, Jb6NvP7rNs6dlG|21|31-437-708-4779|7838.91|ole against the silently final deposits. blithely pending +1428|Supplier#000001428|luS0rWgXC0XOY2U00gRU5LIz|7|17-687-376-8613|-344.66|mold quickly. slyly even requests haggle carefully. +1429|Supplier#000001429|4pWaHM0NNE0j GmfgT8|21|31-457-951-3566|2178.02|y pending excuses should have to print among +1430|Supplier#000001430|H7JZy,Px156gtqkfZGSK4PzX8ciU|12|22-190-359-2449|7025.12|eposits are across the furiously regular ideas. blithely pending p +1431|Supplier#000001431|,EPYNn1We9nbuIIdsRf0Qj x2b5h6YL|9|19-277-237-3709|6253.70|ross the pending, bold accounts. ruthless, final +1432|Supplier#000001432|gIECERmEUGoiGrKmSz|21|31-671-456-4156|9901.58|ongside of the furiously final +1433|Supplier#000001433|y4BL0GSGtpaJCG2036cdt9V7zsz7SMMID|2|12-312-363-6186|5977.55|ake. regular requests among the +1434|Supplier#000001434|NkTyGdEmQkoXsd7PaKGtjn,K8W9WDNz9F6pyR|4|14-435-373-2338|-237.92|ld instructions are pinto beans. packages haggle agains +1435|Supplier#000001435|4dQ34bnQ8aVnkQlyBeU |20|30-445-855-7067|3200.56|ess excuses are fluffily pending, final platelets. blithely spec +1436|Supplier#000001436|iDMEIlE8DQxPsRZM3zadhdJXcD2sauZgP97S|7|17-138-841-4100|-240.51|sits boost after the slyly regu +1437|Supplier#000001437|LF89vzsf3o|8|18-713-285-9012|3735.92|side of the ironic pinto beans haggle slyly across the careful +1438|Supplier#000001438|qjNivLd5oZDMr1BN62A2uR0cbDgsO,,Sh2|12|22-243-449-9368|-342.27|ar accounts sleep regular, even accounts. foxes doze. unusual excuses are slyly bold instructio +1439|Supplier#000001439|Ahg9t34SgEUY4Ha4jlcYoR|20|30-706-667-6250|9440.10| requests could sleep-- furiously even deposits boos +1440|Supplier#000001440|bv5HJUqVkW|16|26-480-361-1751|4863.98|kly special theodolites along the fi +1441|Supplier#000001441|uWzJDzjltr4ZX5PJLbe7g8zNtho8C5Awq9Chuu|10|20-682-239-5942|5851.67| use carefully against the requests. evenly regular foxes serve iron +1442|Supplier#000001442|j52dbhkWtbBg54wmB51gGQkAdu4|4|14-787-478-6073|7061.35|ole furiously according to the bold, pending pinto +1443|Supplier#000001443|mRFGwS7y1HfgWC6styUuSst8orq2JlJhCWHCsr7r|15|25-723-252-9388|-787.56|accounts. bold, pending ideas haggle blithely along the carefully ruthless deposits. qui +1444|Supplier#000001444|fQ3e8q0dg |12|22-785-978-9406|5465.65|wake blithely silent somas. packages are blithely. furiously pending requests +1445|Supplier#000001445|DhcB aCCxE,4rEWuOU5WWYTebpx9|10|20-769-712-3738|9182.83|tes across the regular pinto beans sleep according to +1446|Supplier#000001446|lch9HMNU1R7a0LIybsUodVknk6|3|13-737-922-1377|3913.05|r courts cajole ironic, bold deposits. blithely pending accounts nag +1447|Supplier#000001447|Y6nbeZmxx9J0j,1PLvpljga qvtV0e3ehThVjXMf|3|13-524-189-5007|6676.26|s ideas. silent, unusual accounts cajole slyly. blithely silent requests de +1448|Supplier#000001448|3vllw 89Y2IDAE6MbyqjplZ6|21|31-269-267-6837|5877.31|its according to the slyly regular deposits use quickly about the ironic deposits. furiously final r +1449|Supplier#000001449|NFQzLwdOyj0LqRnlVKF|11|21-744-540-9302|8896.57|ffily ironic realms are quickly according to the blithely regular requests. slyl +1450|Supplier#000001450|nUHaPsG8kBoh6|5|15-869-845-5599|-279.72|furiously final theodolites. furiously pending requests about the +1451|Supplier#000001451|GBV9BcM0YQqJNkE2XQ|16|26-750-504-1538|4611.73| requests cajole blithely in plac +1452|Supplier#000001452|1yAiNvo7ZJYhJ4V1|20|30-704-615-5588|1310.77|osits integrate carefully about the special +1453|Supplier#000001453|e1XQux,y BE|8|18-402-504-6984|5996.22|riously pending grouches use slyly after the regular accounts. carefully even frets use qu +1454|Supplier#000001454|TOpimgu2TVXIjhiL93h,|3|13-524-204-3262|5556.05|thlessly even requests cajole among the final pinto beans. even, +1455|Supplier#000001455|mHzGwa5hFeTvvGY4zEsD8,QyWs1ifW0|12|22-794-481-9846|-529.62|jole fluffily quickly ironic requests. quic +1456|Supplier#000001456|EZw3KWqv94b1lqocrz5F4XQpoVE|20|30-154-456-1570|6569.97|frays alongside of the blithely regular foxes boost carefully s +1457|Supplier#000001457|fBc,hLeb5G2Mb8sDVS9ExK3Wv|9|19-690-888-9450|1517.82| wake quickly. slyly unusual accounts boost carefully abov +1458|Supplier#000001458|3SmErYBeN9EFj|24|34-790-720-6395|-980.21|e carefully ironic ideas detect after the carefully final instru +1459|Supplier#000001459|9f0o97h4BZp2EFZdILZKMcHDa5C8K3|24|34-200-372-1503|1588.38|ly pending requests cajole carefully about the blithely even requests. furiously even requests +1460|Supplier#000001460|wug2y2DTPZxvm2NbOGw7xm7e|8|18-947-539-5061|7876.55|ic pinto beans impress fluffily. carefully final p +1461|Supplier#000001461|k88iU7GXk707r9eKF9Jc9LMHGyumYWsFGpH |12|22-519-392-1119|5298.91|as. furiously pending packages are special, silent deposits. busy packag +1462|Supplier#000001462|HgxOeUIzzWk7BTRw2ax8oHi|6|16-983-112-6138|840.36|r the blithely regular accounts sleep inside the quickly ironic pint +1463|Supplier#000001463|yD5,1wUkwuvV4gWRloVVudgeUpO03et0|5|15-484-394-3837|8794.09|xpress excuses use about the fluffily +1464|Supplier#000001464|yZoKE4Ac8szLlvH0cYaqp6paOV|8|18-984-442-6908|4336.46|. even, even ideas wake abo +1465|Supplier#000001465|euBDI 57f1TH1e|14|24-441-729-1194|5941.89|y regular pinto beans. furiously final deposits boost ironic, regular +1466|Supplier#000001466|ARHmacYHsfIRRxVdonchVtjf6zLe1thHGa|18|28-580-167-6865|5434.49|. quickly stealthy pinto beans about the blithely regular +1467|Supplier#000001467|cMpeOPD1hbd015 mppaxRtbjMtJ7Dza4Q4Y6om|17|27-991-433-5367|2433.20|furiously ironic platelets dazzle regular +1468|Supplier#000001468|IfGLGB5BHW|23|33-379-279-9526|5730.52| quickly pending foxes. regular decoys are car +1469|Supplier#000001469|MYQJxMgJtkzqOv9xIZRZuKweoHtgTaQwyG|16|26-598-883-9977|5813.24|e quickly. bold frets wake furiously. silent platelets cajole blithely about the permanently ironi +1470|Supplier#000001470|YLMgP6bJXx1ny1hXz6BFm|4|14-682-476-2984|-857.31|about the quickly regular instructions +1471|Supplier#000001471|34mG3MLXJGiFEm|2|12-450-672-3679|-846.13|e final deposits; quietly pending deposits kindle slyly instea +1472|Supplier#000001472|RHjAeeLxoC2VX7qxlFbuK7Sq8Dle|6|16-460-172-9906|-50.82|onic asymptotes sleep blithely; quickly express pinto bean +1473|Supplier#000001473|o9392JZgpmHc5V4iV2GLeFk9TrNqgoWZIF,ZO h|4|14-377-224-1276|1746.30|are slyly alongside of the speci +1474|Supplier#000001474|X2ye iuJe 0ghenms,j |2|12-456-585-6521|84.65|s use final, ironic foxes. carefull +1475|Supplier#000001475|FuslzvEOEkZNhVJrhtnarr88mzUnt|17|27-989-740-2417|1647.48|egular deposits. quickly +1476|Supplier#000001476|6MLxydyGdgAuvzTiSmvLRPcIV3693PkAi9A6L3|21|31-854-366-8239|128.17|refully special platelets wake about the carefully regular instructions. fin +1477|Supplier#000001477|hTs1G2aU6anxQb|2|12-867-686-8944|7033.86|re furiously. carefully regular instructions sleep after the regular asy +1478|Supplier#000001478|b6TLE65sryHFVFjTIukGqvuefvY5|17|27-123-588-7355|-897.05|onic requests. furiously express foxes boost fluffily ironic deposits. qu +1479|Supplier#000001479|txoNHYrR5zTPAqlt3t3V|12|22-442-690-3533|2114.87| quickly ironic frays above the fluffily even platelet +1480|Supplier#000001480|lb1yJFg3zCsjaIAP OHRC08sLRxnWEDZSvA14uA|23|33-173-673-9553|1590.79|silent courts cajole quickly of the care +1481|Supplier#000001481|ARqVvJHMxBNKl2LrfPsR Wq9ZUXh,14|21|31-207-618-9010|4436.86|foxes-- final foxes wake slyly pending, brav +1482|Supplier#000001482|xucSHz1vIirMdJBjNFyNGXWhJzoSlDOue36cH|23|33-579-158-3822|-746.35| believe carefully. blithely special deposits across the carefully regular packa +1483|Supplier#000001483|P23Nxp kYtqrv imDiUHVZtbUnz|8|18-701-650-4334|5023.52| alongside of the furiously pending packages. ironically ironic packages +1484|Supplier#000001484|oHkcb,ydehrJs7GC9y8|16|26-211-416-2003|3358.49| final instructions maintain. pending, pending ideas cajole ironically near +1485|Supplier#000001485|L8P733QxcUToM79zEUVnOgaU|4|14-394-892-9890|3134.16| express instructions. ironic accounts after the instructions are slyly across the doggedly +1486|Supplier#000001486|49e4IjfGtKnxW5qYdNhbIuFqGdh |19|29-277-611-8007|5866.30|ts. quickly regular accounts wake ironically. slyly special asympt +1487|Supplier#000001487|0ISabDJ8Fj7H8qu3qPvx|18|28-878-390-8065|4.83|rint final theodolites. furiously express excuses haggle according to the regular, ironi +1488|Supplier#000001488|qBIS15OXCdIZ7fR64xTY1XSN4aM|23|33-581-958-4335|2047.93|ages wake carefully about the ideas. slyly ironic instructions across the carefully even i +1489|Supplier#000001489|08AMqEGmciIVqoF5jI4OfDFjpt|15|25-494-666-3642|9581.10|ent foxes lose. even excuses haggle slyly. quickly final deposits cajole pinto beans. furiously +1490|Supplier#000001490|pK,FClsQ8cq wa sGQJ3ra|2|12-213-362-8295|9570.92|s. special instructions nag fluffily. quickly special package +1491|Supplier#000001491|6m1IpNKdXYgKZYfeLDBtGybod3vVZ8ZrIL1OO7H|21|31-795-458-3281|3562.32|uriously deposits. blithely special platelets are blithely carefully even pains. slyly final depen +1492|Supplier#000001492|z2uNRsyJJ,ezj,7lW1C1|24|34-345-702-5433|7851.60|ly regular ideas grow. special deposits affix silent +1493|Supplier#000001493|MEIytTTK27Z1YIyJ4fRh3FsLUJEzQxaM|0|10-862-591-4491|3812.78| slyly final pinto beans cajole regular i +1494|Supplier#000001494|nqNP5GmByEQ496Y3MgUngQ|7|17-962-207-6063|4331.41|fter the quickly even requests. blithely bold deposits haggle blithely. blithel +1495|Supplier#000001495|jCiaZxCbu0u5NXQGPU4I5u|9|19-924-825-8388|8970.57|slyly special requests integrate slyly. bravely regular theodolites ac +1496|Supplier#000001496|E4P0XwvHFUx,3aWTAqufAMepX|0|10-918-381-1812|434.04|de of the quickly regular packages. carefully regular accounts cajole slyly regular waters. exp +1497|Supplier#000001497|mB9z,cnNICcr 4EiYuUJArHLUL4ZzySsg2kll9|18|28-563-741-5493|-108.16|asymptotes. regular pinto beans bo +1498|Supplier#000001498|i2cBOyGFyMMcbR6jv1SX9L1Dl P7 2BoT1|20|30-541-982-7603|6022.47|ns wake blithely final warthogs. quickly regular requests a +1499|Supplier#000001499|2k vqZ85Unw,PPeD7e8j|12|22-833-725-1232|6614.46|y pending requests. slyly regular th +1500|Supplier#000001500|wDmF5xLxtQch9ctVu,|3|13-685-210-1639|1213.97|boost bravely bold excuses. carefully bold instructions are. ironic packages sleep. blithely re +1501|Supplier#000001501|xL06,Y 2M3CjPr|3|13-257-615-9493|7168.16|packages across the final dependencies shall haggle furiously pending excuses. slyly even pinto bea +1502|Supplier#000001502|UdhLqlClUU,G8r0FCi6mELF3eeQ|2|12-226-454-8297|1380.58|. deposits along the quickly special deposits solve furiously about the furiously s +1503|Supplier#000001503|bmkFWwm85BlTbkdPFOWKKs|24|34-300-112-6485|2306.83|ronic requests wake carefully furiously special ins +1504|Supplier#000001504|UPEAScgZa17W8yD8Wp|24|34-307-220-2727|4905.31|s. ironically final platelets sleep slyly. quickly even packages nag sly +1505|Supplier#000001505|,JOcArGRA6o61VZ53KRQf1hYTIWq0xyzMwQdwE|18|28-989-257-2308|7499.55| carefully unusual packages. final theodolites affix carefully. c +1506|Supplier#000001506|0mjuNYVOEOzcX1EV80CX6IwlB|19|29-282-871-8851|5008.72|jole blithely above the blithely si +1507|Supplier#000001507|JnqoaK,6Qg|8|18-859-766-6843|7568.02|its. even deposits about the carefully special re +1508|Supplier#000001508|Uik2a28GvBBd7R7Y21wjT5|6|16-380-863-8334|797.48|e unusual, unusual requests are carefully regular asymptotes. furiously ironic realms +1509|Supplier#000001509|3tq,gpu0XXFIFwNDhULw0s0h|1|11-279-103-1417|3202.80|ove the carefully silent fo +1510|Supplier#000001510|q0zLFh0,TNuluEigsmt9zVg|14|24-567-454-6556|4458.24| even dependencies nag blithely daringly special +1511|Supplier#000001511|KCSUIx2zYSDn|20|30-284-618-4021|6660.71|ndencies. orbits affix blithely against the +1512|Supplier#000001512|YXuyZTLmJh|23|33-670-389-3311|8622.55|slyly bold requests against the carefully express deposits use furiously slyly final +1513|Supplier#000001513|AUhtpbh 7oUdo5Gy5AqhR0c1r3p|4|14-417-652-3628|3215.42|. final notornis about the regular, regular pinto beans sl +1514|Supplier#000001514|mUAsH0lOz8YYbXMl1VFP qNFK9,|8|18-748-482-3964|7509.42|olphins nag. pinto beans integrate unusual accounts. b +1515|Supplier#000001515|Eg98ocIKS2VH1hYsFQl2 wp,PnSQ3Q|18|28-919-465-1784|5947.74|luffily unusual sentiments affix stealthily after the final pinto beans. instr +1516|Supplier#000001516|C1SzpVp67oxXWyL7reHB3kJ|8|18-431-532-9957|4755.71| carefully unusual packages along the requests are according to t +1517|Supplier#000001517|oJqQSTf8YH4k0|9|19-783-446-7725|1199.61|nic theodolites alongside of the accounts haggle thinly within the regular instruct +1518|Supplier#000001518|ONda3YJiHKJOC|23|33-730-383-3892|7850.66|ifts haggle fluffily pending pai +1519|Supplier#000001519|smukvJcjJUSOJOIU7N9H1x|16|26-892-479-7094|-888.32|e furiously among the slyly even foxes. ruthlessly express asymptotes should have to +1520|Supplier#000001520|wfx,kh4HNP|19|29-367-271-5757|5851.59|s nag furiously around the special, regular deposits. sl +1521|Supplier#000001521|OdAz Nj92X2 CqnZQAbfl5XZHvzok4|2|12-207-928-2351|5685.29|ctions cajole blithely. carefully i +1522|Supplier#000001522|W0C5TrNw89U|24|34-440-334-7894|4712.38| deposits. regular requests are fur +1523|Supplier#000001523| vqDQGhm58v0|10|20-315-869-6648|9915.48|ly regular theodolites. Tiresias cajole +1524|Supplier#000001524|aqGwAL1xCDiPhixZzS47IR2rfuXCa7o|12|22-928-971-3778|8482.04|ecial pinto beans according to the slyly final instructions doubt +1525|Supplier#000001525|yH9JvdfaPgPVqHMm|9|19-313-350-2767|6646.30|ress accounts. pending accounts wake express instr +1526|Supplier#000001526|bkKFtZanK27y,IzY|22|32-122-337-4827|8396.28|y regular ideas according to the ironic, even packages integrate above the even, ruth +1527|Supplier#000001527|373oKr,Hok,M1fz5pFs,DGt00NeCNUKvqsqB|21|31-496-199-3385|6304.70|blithely final packages about t +1528|Supplier#000001528|it4EmP9yhW0IZvUnapbYfoKXVvY2|3|13-570-757-5808|9186.59|refully express platelets; d +1529|Supplier#000001529|qQJmdEj7AcwaQxBXxlgZ,f|16|26-785-696-3319|1109.02|ithely bold requests are furiously slyl +1530|Supplier#000001530|gMeWFxQBzD1YJGuhdoSJV41RgCVI|8|18-277-371-1378|3167.99|gular requests are carefully. carefully bold platelets wake daringly fluffily b +1531|Supplier#000001531|SRV0tsqpwsgBHM,0LJRj9NWJ9SBDl4PSL|5|15-945-324-4201|2441.49|detect quickly alongside of +1532|Supplier#000001532|b CkwJ3by69KW5TZgmau41,sKcsID br4QZkzWs|12|22-449-543-8612|2132.02| patterns. carefully final instructions wake according to the fur +1533|Supplier#000001533|kPJJrNAF8d85BDu8L8|1|11-951-380-7283|3905.03|g instructions according to the blithely special requests cajole finally blithely regular +1534|Supplier#000001534|,rRNXQRJQd6JXzpwnEuKkKn|20|30-891-630-2295|3863.61|ounts believe around the f +1535|Supplier#000001535|Cv51IQjK9TVKYB5gIGxWiTQs|4|14-840-750-7001|731.46|otornis boost fluffily regular sheaves: blithely ironic theodolites +1536|Supplier#000001536|1j EhxcXVVIzVlc9hN4f7JMHMXi3QAAhvPemzY0V|10|20-837-590-2286|4511.03|regular ideas about the instructions use slyly ex +1537|Supplier#000001537|ZgxRuN5Jp,X8JmW3WHJbjy2nlOgUqlKEDO9a|23|33-900-377-4530|2850.48|ts use quickly. accounts cajole carefully. carefully silent pinto beans ha +1538|Supplier#000001538|pB38h5flEq7qC7sFqTYjhMuFvJxoRc6eY|5|15-344-384-1691|1062.49|hely sly ideas according to th +1539|Supplier#000001539|taspo8MUN0dFhw|12|22-641-865-1602|2203.18|y regular requests above the quickly express requests boost fluffily +1540|Supplier#000001540|7KP04dsYoPPZRJ6UgAD|11|21-846-405-6709|829.35|yly along the even, bold pinto beans. regular instructions about the slyly ironic sentiments ha +1541|Supplier#000001541|rPUV63BMAmT8Y2qhs 5Z9IT D8zjCJeBHZjW|6|16-290-201-5378|-213.62|ests play carefully. quickly regular i +1542|Supplier#000001542|RsQTMZbEs4j924|9|19-766-450-1320|4668.72|to the accounts impress carefully furiously even pinto beans. blithely special instructions cajole +1543|Supplier#000001543|sLFbJx l9VO6B,EXd2LWzWnm3VCkBD2YTM|4|14-270-284-3752|2107.21| quick deposits lose blithely along the unusual deposits. even theodolites about the +1544|Supplier#000001544|Z3riyyeH0kbAou1sG79iH s|1|11-583-499-2115|4875.65|ithely busy packages haggle furiously after the quiet accounts. slyly regular packag +1545|Supplier#000001545|MNmemS8F3k46UJchrAjJMAo9pRjyoKQC8qk Nfer|13|23-323-451-6760|3987.13|c packages nag quickly even pearls. pains accordi +1546|Supplier#000001546|g199RfU vqU|4|14-723-857-8326|8707.80|eodolites are after the unusual packages. carefully regular packages are closely a +1547|Supplier#000001547|bgbQboa0uUWjyXQlmtmHvXh|0|10-509-209-3829|6095.56|ts nag final, final accounts. quietly even requests solve fu +1548|Supplier#000001548|1v5gKgtjgE6BAATavPOmjzus20h3r4u|10|20-589-983-6951|3165.62|press requests integrate. blithely regular +1549|Supplier#000001549|MYsM43isIm8Y5u|0|10-983-665-2259|6471.77|kages. furiously even platelets cajole blithely even +1550|Supplier#000001550|WbdqrA NXZqJz4e74|5|15-207-965-2975|1018.79|after the even platelets. final ideas wake about the care +1551|Supplier#000001551|eUU5Yh2vBpBKaw93MkAui6rLrkjbs9|22|32-712-140-2807|4699.47|kly final accounts integrate. blithely iro +1552|Supplier#000001552|E3oQtBVw0HR T3WujhZTTsfNDd3ASxP9EyaJ|10|20-315-287-9110|2038.17|ackages affix blithely even o +1553|Supplier#000001553|GmS1yJTxKntAPG|14|24-430-927-3515|-450.51| carefully. slyly sly platelets haggle furiously alongside of the +1554|Supplier#000001554|EbPwlxxpCsxoKvJduekha8s mOxumWg|12|22-764-566-1120|5246.82|s. furiously even requests cajole accounts. final pinto beans use slyly. carefully regular packa +1555|Supplier#000001555|fG2uWzzAgeimoRAlr2PG5W|9|19-336-916-3670|5266.89|s sleep across the slyly express account +1556|Supplier#000001556|4WGfuUewnE0EzhrFO|8|18-515-199-2701|1579.29|ickly. regular multipliers serve about the blithely regular courts. unusual, bold asympto +1557|Supplier#000001557|ZxA5pFdT2aBu6|18|28-384-844-8031|9059.38|to beans cajole; carefully regular instructions are. theodolites snooze stealthily. blithe +1558|Supplier#000001558|rzsNTnQnD7oSWaWr|16|26-135-274-9529|7997.58|riously. slyly regular instructions are quickly slyly ironic foxes. bold theodolite +1559|Supplier#000001559|cTgcN,Vi N|7|17-553-423-6440|4433.52|ic deposits. regular requests nag sl +1560|Supplier#000001560|sFprTfclLzUUXcEZT5e8mKMHHPi7Vy1qshb|19|29-662-669-8767|9130.41|ut the even, ironic instructions! blithely close accou +1561|Supplier#000001561|rhXxtK22rXkYrQuR7,|11|21-689-890-4790|2888.63| unusual accounts are carefully above the carefully pending id +1562|Supplier#000001562|3fYy4SyTZSM58OyYaxhw6o2uogW1|14|24-756-533-8039|4102.00|ly regular packages use carefully against the final, unusual accounts. furiously fi +1563|Supplier#000001563|grObTJRTIz1LDICvxmD1XjMNbu6E|7|17-527-374-1722|3316.18| poach carefully alongside of the quickly express packages. blithely final theodolites are f +1564|Supplier#000001564|4w6z11S90E8QbtCylGdxhh|9|19-656-743-1239|7097.83|nic requests. fluffily special packages doze stealthily. carefully bold accounts sleep f +1565|Supplier#000001565|bOwKHdBteMkZoZcxdigk4Tnu07w1gDztmV7hvCw|11|21-791-273-8592|7264.42|to beans. express accounts nag around the +1566|Supplier#000001566|FxGHzfvgqFzHSvl94ir CrV TzfDHA|24|34-449-468-8660|2082.14|nusual packages. express, sp +1567|Supplier#000001567|r37zhS4ghrSTDkfFfFYPF4OwHk694pkunMedMnb|16|26-561-181-2965|1505.26|ies. final packages was slyly furiously even pinto beans; pinto beans haggle carefully. +1568|Supplier#000001568|V7Pk75s64GORPR1BbwN 2I1xawVm6C7OpGRa|12|22-726-915-2985|7834.92|g foxes. accounts sleep. +1569|Supplier#000001569| SmIEexog6MESmTTKRLYzmmUDDeTsHi28NjYBb|21|31-720-753-1283|9613.73|y pending ideas are slyly ruthle +1570|Supplier#000001570|QF2XhOAWwJeg21avqIRl8E2gi1mXAN75U9a|12|22-348-472-7053|7963.33|ins. furiously ironic deposi +1571|Supplier#000001571|08Pd2gUuOUK8aRZiHWv0IDCP1 ZIkxIt5|23|33-273-607-3674|673.78|hely above the regular foxes. deposits print never quickly even deposits. regul +1572|Supplier#000001572|Kh0CQlTLUlsXmjPM1nJ7qSexveaS6gZXo,UtMNM|20|30-582-134-2161|3102.93|foxes along the ironic instructions h +1573|Supplier#000001573|yYUX fPcpR9TBPjz|1|11-442-876-5702|8879.38|as. furiously ironic accounts grow. fluffily regular packages use qui +1574|Supplier#000001574|010G9zS4hWlWkl1X3 F7Ha2 Vnq2Qc2|13|23-997-787-7515|7750.28|yly dependencies: furiously regular packages w +1575|Supplier#000001575| A2VO9TlYeAr6XeBELX8e9Ubcfh|13|23-960-614-7292|6028.32|quickly ironic requests sleep carefully special instructions. regular, f +1576|Supplier#000001576|3dj4fsF5fNQ2boo1riXOA7N9t|6|16-116-644-2882|5929.82|ic accounts cajole slyly ironic accounts. pe +1577|Supplier#000001577|gVkhzP4car7D2Eg7AML90y6|10|20-430-554-8789|590.12|into beans. instructions sleep quickly carefully final foxes. quietly ironic deposits slee +1578|Supplier#000001578|MI,JjL08uqYHU9tH,agj1wpSWQq 9hcP,VCC6vcR|24|34-838-683-3491|5095.53|nal accounts alongside of the slyly final ideas wake against the slyly ironic instruct +1579|Supplier#000001579|EA256SSBFlnNj6D,4dRgFrm|22|32-223-473-2267|3608.71|ounts was furiously sly theodolites. even, final accounts wake carefully packages. furiously pendin +1580|Supplier#000001580|gqwJchO919h1NPd8IBWvNahMtg|5|15-814-392-7616|6505.84|equests cajole slyly along the furiously pending pinto beans. slyly unusual reques +1581|Supplier#000001581|Mayh8ynSzPga4xRjsvbA a,IH59HBwE,2i|13|23-726-930-9372|1655.27|foxes wake attainments. special, regular theodolites use +1582|Supplier#000001582|QctT8WYfXx,fb5lmhhd2EEvNFbvIlNoyisz3i08|2|12-782-597-9543|-926.61| blithely; even ideas detect special dep +1583|Supplier#000001583|2Nmh1yFFQPxH4nUw06iqQ2FC5eM8iM|14|24-117-692-4897|387.10|luffily blithely final dolphins. unusual instructions boost furiously ironic packa +1584|Supplier#000001584|gJbTkijteJxSMLmdzBSzeMAH|21|31-213-675-8655|2851.71|the slyly unusual foxes. regular d +1585|Supplier#000001585|ItpE5Is5uV5yPDNEOwWYscn n8V17ee|9|19-884-914-2018|-353.98|al dolphins cajole. even, bol +1586|Supplier#000001586|m LYj4Omt,luWdRK8pY|18|28-163-124-8856|-388.63|he regular frays. fluffily ironic deposits cajole slyly fluffily bold +1587|Supplier#000001587|mJGiVq4lo012P4Qd|24|34-523-646-3372|205.64|courts cajole. fluffily final grouches are expres +1588|Supplier#000001588|JxR1ZHKemou|6|16-985-426-2117|182.29|sual theodolites grow across the express, pending courts. express, final ex +1589|Supplier#000001589|DFyI,OICFg|17|27-520-853-7657|3756.76| theodolites sleep. even, special requests wake fluffily. pending accounts nag again +1590|Supplier#000001590|rEYMbl67PIcccPfu0jIi4BsqQrb|11|21-214-808-2728|8803.31|ing accounts are quickly! special pinto beans wake furiously after the asymptotes. quickly expre +1591|Supplier#000001591|CJejUJNK0ET|22|32-781-225-2968|5298.32|tegrate carefully. blithely ironic requests haggle furiously past the furiously final packa +1592|Supplier#000001592|R7WTSZx6J5eE0QdCP6Xrvwuz|0|10-352-443-2162|1806.39|riously enticingly final dependencies. slyly +1593|Supplier#000001593|nyrNd0vqtRsA2mB8 N0B WGupWrAy9WK3l|5|15-528-294-8640|8878.97|s breach blithely. slyly ironic ideas above the furiously final packages nag slyly along the b +1594|Supplier#000001594|QG2VOcf1nf|23|33-733-772-1150|909.83|s wake carefully above the even pinto beans. deposits after the closely +1595|Supplier#000001595|Aiaxm9qhlurHBe bFojSTsFL3fS0s|20|30-498-157-9257|-550.04|es are slyly among the regularly silent dependencies: furiously fina +1596|Supplier#000001596|HBhVIq44Ayc8uf|1|11-975-377-2012|4476.88|deposits will have to are. ironic ac +1597|Supplier#000001597|WdfeZ1VeZcDLxXFTEA5YaGn|0|10-575-361-2743|991.27|pendencies. unusual deposits hang. regular deposits haggle. blithely bold re +1598|Supplier#000001598|tBpG0u4Yf5tBsJwe6Oo5,ayiO19Lb|8|18-847-850-8215|4145.32|platelets at the doggedly even braids are silently +1599|Supplier#000001599|rjcb4JsZbnkwBX90NGz90GjZUE01wp00eEB|11|21-693-626-9990|5367.96|ns. slyly even instructions affix quickly accounts. re +1600|Supplier#000001600|Wli6lXwUVA4SSle lr0iA2VaJZt|3|13-807-407-4084|6241.06|onic requests. quiet instructions haggle across the quickly regular pl +1601|Supplier#000001601|6tH4HvB9R4fqljDeAprIspXqR7MUFtRueDIWd|20|30-869-889-7813|6563.37|nal grouches nag pinto beans. +1602|Supplier#000001602|uKNWIeafaM644|3|13-598-526-1114|673.68|. permanent excuses cajole carefully. bl +1603|Supplier#000001603|QKE6q UdY8JD3d44aRQ0 r8xX|0|10-246-381-9259|1559.33|ainst the carefully regular requests. furiously express dolphins alongside of the pend +1604|Supplier#000001604|JQqqQ1FeZHi2UV9Ji2o8WW,1w4ZdYHoA v0m,g|6|16-709-747-9026|3681.43|le quickly: unusual, bold requests about the ironic excuses +1605|Supplier#000001605|A5Jf2STTfJObp eGoChS0KYYNhe1vJsJ8V|13|23-611-695-5721|7712.58|theodolites cajole at the fluffily reg +1606|Supplier#000001606|NhCLoGdGq6Mtfa7gVfzi|24|34-460-502-7798|-537.20|eep above the blithely final foxes. slyly regular depos +1607|Supplier#000001607|t umXULPQNsFgOks6Ssxp V9y1 EOpYLICa|21|31-587-363-7648|247.60|ounts detect furiously alongside of the regular accounts. unusual, +1608|Supplier#000001608|IvOUzvA72x4I1mObouptJEsTca6eHWhz|2|12-309-145-4668|1269.80|haggle slyly. regular excuses +1609|Supplier#000001609|dqTbeKACaikTyyytcKQtjaHXasj0M7I|9|19-497-494-7241|-514.12|beans. ideas wake quickly express packages. blithely ironic +1610|Supplier#000001610|zg69 Tqwah38L7u23eK|18|28-416-322-9988|3120.60|s wake slyly. slyly express requests boost idly across the fluffily ironic id +1611|Supplier#000001611|s2UJPFHH46tf,QWRXLNuN4UkAL|0|10-123-465-1292|8875.71|special deposits. pending, even ideas wake carefu +1612|Supplier#000001612|79DAQCPIgHZ|2|12-797-511-5255|1175.40| after the furiously pending instructions. ironic ideas integrate carefu +1613|Supplier#000001613|u5Z9UgWdfELP,hrWtPfj9lsrvGGNHvE|8|18-197-961-1832|7958.64|le furiously about the regular, express theodolites. quickly even accoun +1614|Supplier#000001614|Qw0p3FZDls8WqXRjvs2Q6wu7|12|22-788-900-8919|9896.02|packages are blithely. blithely final packages sleep after the furiously final packages. fluffi +1615|Supplier#000001615|8xlm4dZEH51da2i0ws5QC9n,D Mr|20|30-882-687-4686|556.50|lar theodolites around the slyly even requests wake carefully busily exp +1616|Supplier#000001616|uT6XNcLkO6ZETegNRj0QWIm|14|24-729-958-9586|988.66|uctions. quickly even theodolites use carefully regul +1617|Supplier#000001617|jnnhvsWAe9I5qanmlc|6|16-550-365-8843|4373.08|s detect furiously about the bold, special foxes. blithely silent deposits na +1618|Supplier#000001618|G0hSiC,5kKfQ0N5IZpTgMVvJcoIY,6|17|27-687-308-3845|5436.51|. even accounts are furiously. quickly ironic deposits are blithely blithely perma +1619|Supplier#000001619|woJZMLuz6fECsf2,YE BWOh|23|33-900-729-2366|3389.27|en, special accounts. dugouts wake. fluffily regular packages boost furiously even asympt +1620|Supplier#000001620|yD0MbgagtO2|24|34-133-815-7983|3443.49| packages? bold packages according to the furiously express deposits haggle furiously ironic +1621|Supplier#000001621|QrXudFjJgUNHGLm BUzakaXGp|7|17-495-537-3207|1734.52|beans sublate deposits. slyly silent excuses nag quickly after the special pinto beans: unusual ins +1622|Supplier#000001622|xPza7deL4BK2xjF9o|10|20-858-165-8949|3.81|the quickly bold packages sleep slyly even deposits. unusual, pending theod +1623|Supplier#000001623|7QdxGgnQRvPDGSTMtD BpStZts0Xk,pPkqjtN6|9|19-525-377-5806|7047.64|alongside of the fluffily regular accounts. ironic, close accounts are alongside of t +1624|Supplier#000001624|ISkTuv6KLp9AcxdWRGJ8 ey,r56tQ9DozGP|3|13-316-879-7795|-303.08| quickly bold epitaphs haggle slyly slyly final foxes. regular pinto beans hang +1625|Supplier#000001625|yKlKMbENR6bfmIu7aCFmbs|11|21-769-404-7617|8046.55| deposits. ideas boost blithely. slyly even Tiresias according to the platelets are q +1626|Supplier#000001626|UhxNRzUu1dtFmp0|3|13-239-682-5714|2460.11| final depths. idly unusual courts wake slyly about the ironic fox +1627|Supplier#000001627|74kMjyK1cWzXBtjZK u4n5I4UP|5|15-112-462-8537|6981.07|uickly special pains. express asymptotes wake even pearls. blithely even p +1628|Supplier#000001628|GRsapf3mv7Vw|14|24-736-353-7342|2828.46|o the slyly bold instructions. express theodolites throughout the blithely final accounts snooze sl +1629|Supplier#000001629|kUeaD,ZEdnrWnc3qKMvBetl,y1cGrBaBvFI|7|17-883-412-5490|4909.91|y bold requests haggle furiously! furiously re +1630|Supplier#000001630|yA6yXkUCKvJI|10|20-657-455-2820|7931.02|press foxes x-ray. fluffily ironic platelets wake blithely. unusual courts whithout the q +1631|Supplier#000001631|3JwfERzppDc6h7BV0I|12|22-255-355-8658|7687.91|ven asymptotes. carefully regular req +1632|Supplier#000001632|6KZLzKm0GjIQgF2AZ4HDKRcffj9RTw|14|24-118-318-6660|2742.44|nusual gifts wake slyly ruthlessly thin requests. carefully final requests detect blithely abo +1633|Supplier#000001633|0O0likCf4cm7pwJeWkzXaf9cxUb|9|19-561-753-4533|96.29|xes serve blithely final foxes. slow instructions thrash. asymptotes +1634|Supplier#000001634|4B4 Q2fCNjyWqt|10|20-955-177-6323|-760.03|ideas haggle above the carefully +1635|Supplier#000001635|2sUtri6Km8xGmFaoRUET 9|11|21-208-115-2881|6077.48|s. slyly ironic accounts are carefully bold requests. slyly special accounts wake fu +1636|Supplier#000001636|7iI6fY7ItF0T2QuE0Y5XZ6Oo,y2Gl9AZz|10|20-226-408-1320|7286.55|riously furious asymptotes caj +1637|Supplier#000001637|2D8ZlHb5bLmK,YIrlEIOY8R1|21|31-921-265-2668|32.71|ages. final ideas wake fluffily above th +1638|Supplier#000001638|Kvwg58SG5NmUo1Ovy|12|22-635-832-5981|8611.17|ly final ideas. slyly regular pi +1639|Supplier#000001639|cuGHE O3ck4poWFVVq2gYNxG|15|25-980-131-8609|9965.05|urts detect. carefully regular f +1640|Supplier#000001640|38CdHXvKtF5mE D,a AKJ|13|23-867-826-6801|5645.73|uffily according to the final epitaphs. special instructions are bold +1641|Supplier#000001641|gzc7jnvUL6Bro9j9XPJ4UArgAT VYYFBCxe6V|17|27-855-991-9814|-182.95| deposits. furiously even deposits integrate quickly. quickly special foxes haggle careful +1642|Supplier#000001642|cnOSXV9ehy3u8soGk9P3KHhXNzcT|14|24-426-583-5934|3129.69|gularly regular instructions. carefully re +1643|Supplier#000001643|yXTO,21uKjrpvOmX|15|25-145-982-9580|7695.99|usly after the accounts. slowly ironic deposits i +1644|Supplier#000001644|ChjhHjLPsOyLPxmE|19|29-474-678-9070|6565.69|furiously unusual pinto beans: final pinto beans wake furiously above the packages. account +1645|Supplier#000001645|3dq6lQRmb6oukvgSbMUgBPt|13|23-665-168-3606|6585.18|ccounts cajole furiously according to the +1646|Supplier#000001646|f0X8oohw5jcmKtacTCX71xgc|23|33-353-633-6687|4185.16|eposits affix quickly. deposits nag quickly around the final dolphins. c +1647|Supplier#000001647|uXPMu8BN6KesU|1|11-385-158-8420|3980.16|er the carefully ironic requests wake quickly bold deposits. furiously pending accounts n +1648|Supplier#000001648|u8gBXkSvmx1zO9rJcI,GH|11|21-957-133-1311|1536.39| final, unusual excuses cajole after the furiously unusual hockey players. spec +1649|Supplier#000001649|,pmB7HW,5DuG|9|19-665-467-7538|3121.06| carefully regular asymptotes. ca +1650|Supplier#000001650|0R0dQzwyoH6vgewbsib0u6wnkuj,JCdR|18|28-235-797-5129|5602.30|ccording to the carefully regular instructions. pending depths are blithely against t +1651|Supplier#000001651|6rJNoWL9YL|21|31-836-779-5458|6856.74|equests. dependencies use pending frets. ir +1652|Supplier#000001652|RSKj9qiJ0xZBwW4|10|20-895-421-3569|3312.98| regular sheaves. fluffily ironic dinos wake slyly. furi +1653|Supplier#000001653|BgzbkhhpoAtd2kEQ8ogc9HPNwuKNF2tMdOOFtQAe|17|27-477-199-6864|8238.77| boost. silent deposits doubt. courts would sleep. regular instructions nag blithely. bl +1654|Supplier#000001654|5F6ZEzmh6PWDS|2|12-219-980-3825|-988.37|ouches boost about the even foxes. regular packages cajole blithely. ca +1655|Supplier#000001655|S4coCIhdb1y,xbBL4NbuTjXaDgB6 hchz0|5|15-449-908-4417|7445.36|oxes-- bold deposits alongside of the slyly ironic pinto beans run furiously slyly even +1656|Supplier#000001656|HPkORETlxwqmcVjpGe1nbuzQ3Zr4R 5,b7Oks|6|16-238-996-5507|7170.44|uriously blithely special pinto beans-- ir +1657|Supplier#000001657|nkiXvgyvJgMRprx9,Wyz7q1SGYV|16|26-307-373-7484|6628.23|losely final, silent requests. carefully even packages d +1658|Supplier#000001658|CGwZVa1qTQ|7|17-111-389-8658|1765.93|. slyly unusual deposits sleep bold pinto beans! carefully re +1659|Supplier#000001659|Y,MPrFueR16|11|21-972-375-2358|5.01| regular pinto beans. unusual platelets cajole final requests. ironic, pendin +1660|Supplier#000001660|7BQeyUSCsBkA5GBfO8|1|11-776-129-4391|5160.52|rts sleep carefully regular requests. packages are about the slyly do +1661|Supplier#000001661|86KqLVZL5YoGn2Q9l,n11J6T2V|12|22-167-548-4025|6817.13|its wake requests. furiously final f +1662|Supplier#000001662|ATRQSPG3ww5 AC2PrxUEn7eW0k FS,DXzbONe3dx|2|12-164-286-1152|8585.59|s sleep. carefully ironic theodolites wake! ironic packages +1663|Supplier#000001663|DNIbw7A8B Ll6nq|13|23-466-505-1744|9013.03|ts integrate blithely. final, regular requests haggle. sometimes special accounts use carefully. +1664|Supplier#000001664|M5qDkAFsr,ppM41RKZ7cc|7|17-466-353-4789|-954.24|ests along the regular, ironic epitaphs affix speci +1665|Supplier#000001665|ZH9hZfa0baJVYmfXu3uoudL X|20|30-377-968-5888|8975.97|carefully final requests haggle quickly special, unusual packages. packages +1666|Supplier#000001666|z4t8jiCjahT7K E8l|22|32-154-546-8208|3210.50|bove the deposits wake blithely above the eve +1667|Supplier#000001667|fdq,u8n6WtGHv3bi|22|32-975-396-4279|1862.80|ets across the carefully final requests are ironic, bold deposits. blith +1668|Supplier#000001668|oD5tUdEIM5vtt92IQAn2HVQh1QP2BPVNZKNSZCj|5|15-214-784-9215|11.13|slyly final courts use. slyly regular packages are q +1669|Supplier#000001669|9AWkRm7kFNL3i86eTiZDjQDf,K|16|26-762-352-2798|8930.01|o beans sleep. carefully even ideas boost across the unusual ideas-- blithely exp +1670|Supplier#000001670|mQibb3TPZ2mrBK0|14|24-170-743-2189|716.20|ully around the instructions. blithely unusual dolphins are slyly even, u +1671|Supplier#000001671|WoqU0wH41AR|23|33-932-795-4391|1102.72|yond the blithely close dinos cajole quickly fluffily special theodolite +1672|Supplier#000001672|eWGLmD19vbE38Wu80O0Uz|7|17-914-195-9061|7286.94| braids. slyly ironic instructions integrate blithely. slyly ironic accounts boost. carefull +1673|Supplier#000001673|Sza30kYca,udUggnvZv4EjpjEr5Pq TXVzYQd6|4|14-961-472-9331|6669.38|onic ideas about the regular, pending instructions cajole blithely s +1674|Supplier#000001674|nepm7O58V97AD|18|28-893-890-1902|340.14|riously ironic deposits. carefully ironic packages about the fina +1675|Supplier#000001675|ivZoa,172A5gp4dgA93YI6l96Ksh5XkKeEcy0C|23|33-133-387-6972|5722.22|fter the requests. even foxes affix carefully after the blithely +1676|Supplier#000001676|mZ7D3z736uNNMs66KaoK JkGOnjdgRJAklq|18|28-184-394-2196|9371.10|. regular frays among the ironic theodolites integrate pending +1677|Supplier#000001677|z1hO0o9EWm5BpN8uRqvGeml48v0V1FxsiD|10|20-213-199-9893|106.71|ic foxes sleep blithely after the quickly even deposi +1678|Supplier#000001678|1hEflU5Nt,EVDdB6LZO7514 zJdRqjhc9jK|16|26-905-311-6289|9330.10|uses are regular accounts. final pinto beans would wake of th +1679|Supplier#000001679|zN8g,w0gPBysLnz,EjOOo6RJBvh11oShl|11|21-252-692-3949|4060.10|y against the quickly ironic deposits. slyly final accounts haggle slyly about t +1680|Supplier#000001680|9r1tXehZwE72p0i|24|34-230-784-5114|3056.95| ideas wake carefully. even dolphins against the even ideas haggle fluffily across the spec +1681|Supplier#000001681|,,MX631mJ3b,|12|22-369-410-8905|6144.37|f the carefully regular deposits wake blithely even packages. blithel +1682|Supplier#000001682|pXTkGxrTQVyH1Rr|3|13-856-656-2476|9413.98|furiously bold theodolites. final packages need to sleep slyly. carefully i +1683|Supplier#000001683|e0AmK5Spfib dgbwokqeuiFuxn2lOVnrup|2|12-303-331-7460|5906.85|ons haggle. carefully special accounts after the finally regular deposits use theodoli +1684|Supplier#000001684|wHKQO Je0CnUyrIDt2EsyCJ5|12|22-161-480-3715|-574.25|ly according to the furious +1685|Supplier#000001685|YZZuwx45JDn6 QsARZq|16|26-877-710-5379|4424.91|xes among the carefully pending deposits cajole slyly after the carefully regula +1686|Supplier#000001686|LotsVSpeE7V6C|18|28-786-924-2891|7406.30|cuses doze slyly blithely ironic epitaphs. blithely even accounts dou +1687|Supplier#000001687|8NXUtXce9tLl LwaLanN8RG8ykJf|11|21-653-389-4620|-211.35|olve slyly unusual platelets. fluffily even epitaphs nee +1688|Supplier#000001688|WpC9wCj7bYZwJGMrs|9|19-389-800-2516|9779.92|ular foxes wake slyly even, express foxes. regul +1689|Supplier#000001689|eLZWHr5DsW8|0|10-837-991-9041|7204.97|ular deposits sublate about the blithely ironic foxes. blithely brave +1690|Supplier#000001690|Pi5u7Jxd0IMcfwieStSf|2|12-596-971-7617|3996.55|lar deposits are furiously pending accounts. ironic foxes promise above the quickly bo +1691|Supplier#000001691|nBaw4F r,qo2Px7uQnD0inobFV|2|12-221-939-2938|2083.05|o cajole carefully special requests. fluffily special deposits beneath the courts h +1692|Supplier#000001692|vLZfI8OGE,SQXHr3PytUrNhxY48e6JJIBCz|0|10-789-325-3069|5719.43|ly final deposits. pending, bold instructions doze blithely. always express requests boost blit +1693|Supplier#000001693|eyOiCRnurWys75HSWkBONbM3m8u9ESsN3Rml,|3|13-317-825-9696|6559.71| requests cajole furiously alongside of the even instructions. quickly special requests are +1694|Supplier#000001694|ozS4Fm3Do Ld5ZZ9Rt|8|18-587-433-4644|5158.29|above the deposits. slyly special theodolites detect along the regularly spe +1695|Supplier#000001695|SWxXwwAGOSIA4W Ah6|13|23-471-557-4483|2308.39|mptotes use slyly slyly bol +1696|Supplier#000001696|1 8XLqwCH,fbKVtW6HVGt3bl17PQ9vAj4|9|19-782-434-5708|9003.68|pecial deposits doze across the furiously ironic requests. blithely regular requests integrate blith +1697|Supplier#000001697|37nm ODTeHy0xWTWegplgdWQqelh|11|21-377-544-4864|6878.62|ironic theodolites. furiously regular d +1698|Supplier#000001698|KID8KLG,W4NdRgCrfqdUeF5BLX|0|10-428-669-8316|7919.69|ke alongside of the instructions. silent, pending instru +1699|Supplier#000001699|Q9C4rfJ26oijVPqqcqVXeRI|3|13-533-890-5782|4320.75|requests; ideas among the carefully even ins +1700|Supplier#000001700|7hMlCof1Y5zLFg|3|13-910-616-9417|5786.59|ly pending pinto beans are-- final asymptotes haggl +1701|Supplier#000001701|9vBGAcWT7tM4ZQ2pyj0|15|25-366-807-7745|2270.59|he furiously final requests use furiously after the pinto beans. special asymptotes detect fur +1702|Supplier#000001702|moV8JuhcrBxAXOJzmtGb0maE1Mmjrw6t|9|19-747-481-4905|822.69|efully quickly final deposits. carefully even ideas shall have to use quickly b +1703|Supplier#000001703|cWe6NOKoU2JXIhHAGzUwsZ5|4|14-921-609-7101|8823.84|al deposits grow. accounts above the final instructions sleep regular excuses. +1704|Supplier#000001704|9D4 qo36WR11wNoSpCQQGASZiRy4T|18|28-363-744-2071|9326.79|kages sleep across the instructions. furiously final hockey play +1705|Supplier#000001705|h6 ,T,bEwXf6aww|12|22-419-713-3468|3444.11|slyly final pinto beans solve furiously +1706|Supplier#000001706|rLkt6qcRQZbVMFJRSwo |16|26-523-228-5007|1149.18|pending realms are carefully unusual packages. pains are ironic requests. blithely final accounts wa +1707|Supplier#000001707|zNAdyW4VZuWqv5qTFjZnI2tvGndL3hqK0ELmQw1|14|24-329-756-9859|7180.02| was alongside of the slyl +1708|Supplier#000001708|U9TfwBmHzE8Cwksop,,d37vxvd|4|14-852-212-5681|2628.36|xes according to the slyly regular courts nag slyly ex +1709|Supplier#000001709|IkbS20TCZCrjwpJ3L9NG3SAu,MOqDVzVdocrj8JG|15|25-400-333-6639|7848.56|carefully regular courts +1710|Supplier#000001710|J,sdOOJwUhwPv2mrEiNEA0UZlmu5IRmgz|21|31-452-330-7832|7868.12|s. special excuses sleep above the regular depths. carefully express excuses alo +1711|Supplier#000001711|Jx,uiFpJX8FOhY2ntifFomrTkbwfmJc17|13|23-217-649-7762|6514.78|ual platelets sleep among the blithely regular +1712|Supplier#000001712|,wgzZrY56tRQ37G4dL1btANp|9|19-817-134-4251|8910.49|nd the slyly blithe deposits hinder enticingly blithely pending excuses. daringly ironic deposit +1713|Supplier#000001713|YDrf04Arzkk wv t4txebRVoOyAA5lHGxhJ98|12|22-586-592-1690|2611.30|y permanent deposits cajole slyly along the +1714|Supplier#000001714|r0Gtk9pV8Xh,tu8tnflUHKlRTaYt865|9|19-720-775-3951|7147.31|ffily pending theodolites integrate. blithely even instructions according to the blithely even acco +1715|Supplier#000001715|zeMs3O9v3TNv|11|21-323-286-9026|2853.35| quick packages use quickly after the carefully ironic packages. unusual deposits along th +1716|Supplier#000001716|l1HQoaudWVp6|21|31-280-113-9095|5271.46|y after the regular theodolites. slyly ironic pinto beans breach over the unusual t +1717|Supplier#000001717|iLO76fgaDH6DFsBfb6ZMnOK7F9LnqoZRvjQzVpj|0|10-501-359-4359|1380.58| ironic instructions affix again +1718|Supplier#000001718|,OgZ,mA3MSew1 OJvM6UBn|21|31-256-122-2861|2315.87|unts. regular deposits cajole slyly according to the daringly regular instructions. fluffily final +1719|Supplier#000001719|5YnBSxnBsZDkCv8wynfBFc Jdt2Rz,NcZ3uzlb3|7|17-891-726-9678|2831.45|tructions! finally stealthy requests a +1720|Supplier#000001720|ZTDR6fE4HR3fDSKs|0|10-339-599-1876|223.71|cial packages. even packages are; theodolites at the slyly quiet deposits haggle inst +1721|Supplier#000001721|VCPSMnjrsVDMT5,eVI0zMOgohV|23|33-248-237-6674|6749.62|al deposits are slyly even instructions. carefully regular ideas boost silent +1722|Supplier#000001722|h3CJRmiahabx|3|13-756-728-4652|6914.18| boost furiously; quickly final r +1723|Supplier#000001723|6L8Bjhpa4461|4|14-361-743-9306|695.34| pinto beans haggle slyly around the +1724|Supplier#000001724|U 4H6a,MAvCo,EVwrl4Et1D|20|30-408-880-1688|2539.33|. slyly ironic foxes boost blithely special dep +1725|Supplier#000001725|vZiLtETl7Rq,a42N5fyl|23|33-551-143-6319|7546.99|posits integrate. pending attainments +1726|Supplier#000001726|TeRY7TtTH24sEword7yAaSkjx8|3|13-630-597-4070|-751.93|ly alongside of the daringly iron +1727|Supplier#000001727|YI0bCXSwEXpz,SUoF iveZ|5|15-658-818-9847|-807.15|t quickly. ironic foxes wake evenly. pinto beans nag furiously agai +1728|Supplier#000001728|0AHX,VyDusbxz|7|17-263-886-6350|6160.31|blithely express dependencies b +1729|Supplier#000001729|t77dnGUN7PRN68GNkfLtegp8CemKvdDbLaaE3UNd|18|28-932-759-4965|2435.34|nding deposits cajole furi +1730|Supplier#000001730|Rc8e,1Pybn r6zo0VJIEiD0UD vhk|3|13-788-703-6101|8712.72| pending ideas are carefully furiously unusual asymptotes. express packag +1731|Supplier#000001731|Dqy8LQtY5i8GygrdOC1lt,OVsIgrGoL8Z3PMs|7|17-115-638-8685|686.50|lar requests. final, final platelets around the carefully even deposi +1732|Supplier#000001732|GbsJPNJVFf1BfA|18|28-899-515-5026|7947.80|ounts among the carefully pending instructions poach furiously +1733|Supplier#000001733|khvlGwIT6onpM|19|29-242-518-2857|3654.39|special accounts. furiously final excuses cajole regular, pending requests. express +1734|Supplier#000001734|ggP 9OBjmz0tJRRXk,wtQhim2xO3qFxKatx9|1|11-870-214-4915|4289.21|e slyly with the theodolites. blithely silent packages against the blithely even notornis +1735|Supplier#000001735|4fWMQbbJV9L6ROr1tA5cP3zJej VTxPWg|12|22-513-837-7699|3743.00|en, special instructions. furiously final instructions use always express, pending escapades. +1736|Supplier#000001736|cbzU5P1b4c9k7Hr0YTPt|9|19-566-122-9621|8530.37|side of the blithely silent packages +1737|Supplier#000001737|PYmhWHl0BbjFYyExQIJsL ,6LFobxXaOQPH|5|15-199-724-3432|4915.16| wake blithely across the slyly express accounts. ideas haggle +1738|Supplier#000001738|KFD3nE8IHWbZS0prC FPV8FQnF|15|25-692-297-8993|8111.78|tithes boost slyly regular deposits. blithely express accounts sublate furiously deposits. sly +1739|Supplier#000001739|Txm 6j3ooPLq9kID9dQ5KHmgn4nXT71A8|4|14-800-180-3736|9704.52|ly unusual packages promise furious +1740|Supplier#000001740|Lhf8YneZzZ5SHEo1JL3KSA3K4X|7|17-187-935-1871|4914.32|ly. carefully regular foxes believe about the carefully even theodolites. slyly regular dep +1741|Supplier#000001741|c jSLhH3ObhUreddSp,9P4HwE 5zm7rU5oN|12|22-642-922-2439|5050.43|side of the carefully final ideas +1742|Supplier#000001742|sbLubkV9ZTNbLsak6yM4oMz|21|31-253-408-5060|4627.46|des impress carefully final accounts! furiously regular deposits +1743|Supplier#000001743|z6LShIN5U71h2wxKr,NzNpbLzyvu21glBlMoWj|1|11-207-443-2441|1789.34|s boost furiously carefully special pinto beans. even reque +1744|Supplier#000001744|oLo3fV64q2,FKHa3p,qHnS7Yzv,ps8|19|29-330-728-5873|8328.46|ep carefully-- even, careful packages are slyly along t +1745|Supplier#000001745|HpEkG64m4W|4|14-694-581-3111|-555.60|after the carefully final deposits wake slyly about the slyly ironic ideas. slyly regular reques +1746|Supplier#000001746|qWsendlOekQG1aW4uq06uQaCm51se8lirv7 hBRd|3|13-538-692-8661|9920.74|ful requests at the ironic pinto beans are regular deposits. +1747|Supplier#000001747|UyDlFjVxanP,ifej7L5jtNaubC|4|14-141-360-9722|9007.16|maintain bravely across the busily express pinto beans. sometimes +1748|Supplier#000001748|gJWDmBjGrnXcA|19|29-115-992-9045|-674.40|ounts nag carefully specia +1749|Supplier#000001749| NQpuJ3JXjn7fzJW1jmC5SqvgAwP6GHEm7c|6|16-227-721-6187|4202.35|arefully unusual tithes. regular pinto beans wake bravely. even asymptotes wake about the sl +1750|Supplier#000001750|r5RrPrq7oX2xG2xqhHM6tddxzXX4kAPUtwWvYTcb|6|16-689-978-9301|6864.66| fluffily bold packages. even, final foxes na +1751|Supplier#000001751|6jC4PcP6HCs9NMKN|23|33-309-349-1317|4332.54|ular foxes haggle after the even platelets. furiously final accounts haggle slyly fluffily regula +1752|Supplier#000001752|Fra7outx41THYJaRThdOGiBk |3|13-725-402-4911|6025.06| regular packages. carefully final packages boost regular p +1753|Supplier#000001753|Hl8Hn83R16EKFoedi31LQKu5 2exdrLzFeWb|23|33-108-335-5944|6173.02|ess depths wake furiously. furiousl +1754|Supplier#000001754| cXPInp4HEXQsROE6ogwSSREHQ|8|18-437-961-2317|6900.72|thin, even pinto beans. furiously bold deposits against the i +1755|Supplier#000001755|QstBVfnY,93NsbWXCqO|21|31-491-754-4421|5483.45|n theodolites sleep blithely regular accounts. dependencies +1756|Supplier#000001756|7yyqBNCFy3s,NB4eoKo|4|14-424-423-5341|434.56|ic theodolites use quickly final dependencies. deposits among the regular deposits sleep sl +1757|Supplier#000001757|CJh,bMlz8mGtATv12|4|14-574-969-3525|8634.88|ctions boost blithely express sentiments. express, pending packag +1758|Supplier#000001758|RK82Bll97WNO,zhVHO94|12|22-588-254-2879|1437.13|. carefully ironic requests wake +1759|Supplier#000001759|IMUk7aHEg2sS8PyUqj4FsZS7gR8HX3Ix|5|15-110-929-1862|6721.22|y express packages across the sl +1760|Supplier#000001760|Jhn SKQIsJEAU9Szs9N8LRzeEf5XEJh,,w|22|32-639-688-8314|2659.90|nusual requests nag quickly slyly +1761|Supplier#000001761|a0m8NEA2aIlbFIC,54v7|9|19-593-938-9681|9833.79|ckages integrate fluffily after the slyly express pains. slyly regula +1762|Supplier#000001762|yI0sWnNPZTDCkoVzFKe9Zsc7,Fo9nV2N|10|20-193-768-6553|-400.48| carefully finally final deposits. regular, special pinto +1763|Supplier#000001763|jlElnSpkpQuxgYbev1Z35nETkj6DQCINaQOuJ8Af|18|28-174-770-7793|-882.19|ckly beside the quickly regular warthogs. careful ideas a +1764|Supplier#000001764|7nVLpQw9JWeN94kqw|17|27-200-805-4130|-990.13|efully pending theodolites affix furiously. blithely silent packages +1765|Supplier#000001765|6GWz5uUIbv|19|29-692-694-5175|-144.48|ar instructions around the daring, pending +1766|Supplier#000001766|RWyGiX4gzEdSYiDj6GOc|4|14-880-253-3174|679.49|nic asymptotes boost carefully about the furiously daring ideas. slyly regular deposits after the +1767|Supplier#000001767|x7BWRdlYmsP,94h6dhtqEl4hX2Sca|16|26-235-800-4132|93.35|ges. fluffily final escapades w +1768|Supplier#000001768|Ug4jMAaK46J31Z5GFO7x|22|32-197-236-9784|4850.96|counts. slyly even hockey players mold furiousl +1769|Supplier#000001769|u4FB5bCSMFbef3ef4hiPPzH,Dc|13|23-244-220-2703|7027.00|alongside of the regular sauternes are ca +1770|Supplier#000001770|StwobhCYQgD0h7bAo|4|14-805-395-2468|4896.36| slyly between the fluffily ironic accounts. slyly pending packa +1771|Supplier#000001771|PH1MVFYsQwnt09QsErngmmttUHqk6CKAGrH|1|11-454-451-6630|96.69|lly. quickly pending dependencies cajole carefully. furiously unusual +1772|Supplier#000001772|ueuEYuzE0mR1uw4zeNIFG|14|24-235-625-1757|6010.43|accounts. slyly ironic accounts haggle after the regular ideas. +1773|Supplier#000001773|o5pATriiuzDNwkBIMdKkn4PaW7F K|19|29-968-453-1832|2313.40|slyly after the accounts. carefully regular accounts haggle furiously slyly express +1774|Supplier#000001774|AzEk6WfVHo2W|4|14-587-735-1575|5361.09|y after the furiously bold instructions. ironic, pending grouches sleep a +1775|Supplier#000001775|a6Rpe337dA eJQl9RJZeWga0XtFv7em,3ci89|12|22-193-554-4479|923.43|ously bold accounts. fluffily pending pinto beans +1776|Supplier#000001776|T3DN kKgRFwZQAfUuH1rAWw8qS|6|16-950-181-1597|8023.98|requests boost along the ironic grouches. ruthless, ironic instructions are fluffily. fluffi +1777|Supplier#000001777|PnqBSa4xn8ZG8HnNgft7u6Zifn|8|18-460-933-6501|8053.80|usly according to the blithely regular ideas. blithely bold excuses +1778|Supplier#000001778|iAZ6grPHso1aSYofUzvt38|19|29-592-746-9719|3190.51|ackages. furiously regular foxes integrate requests. quickly even excuses sleep. regular, +1779|Supplier#000001779|BG4Zfkyg,QFtyf|2|12-680-787-3989|6821.77|are blithely. carefully regular packages detect: never bold ideas wake. furiously regular a +1780|Supplier#000001780|5VXXjHsRuGrU HNS2xdoUCPw5LMlrJVcUOd5ED|5|15-148-378-4534|9581.63|counts are boldly furiously even instruction +1781|Supplier#000001781|Cs93kCGRA6HlNVZjgrU,5Fi 1F3 vx|14|24-437-662-2009|7565.18|pendencies. slyly final theodolites use +1782|Supplier#000001782|LQqd5Enn37ar,I4vANF98Yda475ImnUtnobvPBV|14|24-892-845-3113|8076.89|lly excuses. furiously express decoys are furiously. regular sheaves +1783|Supplier#000001783|ac5hhWNSEs9oqWSDfSbjsQBogoFpZI|18|28-928-569-7762|6378.00|ecial requests. pending, regular accounts sleep furio +1784|Supplier#000001784|WwxpO7ccLORAYgPyH|6|16-690-399-1778|9010.63|eodolites nag furiously. even, regular ideas detect slyly carefully quick accounts. even, silent ac +1785|Supplier#000001785| fdCWEFK6Qlr fZd 4CFKyXvVnp5krLr5lSu8V|21|31-360-534-7977|2759.82| slyly quickly regular packa +1786|Supplier#000001786|36sVKQsSCrGnNi2Cu EjNTwF90ztyWz8g|18|28-824-614-4319|6341.48|e furiously along the quickly unusual deposits. final accounts are slyly after the blithel +1787|Supplier#000001787|y038r5l,dusbQMGAXd4G|24|34-350-413-5054|283.11| furiously bold ideas. ironic requests shall have to sublate final packages. furiously quick foxe +1788|Supplier#000001788|Wy6v5VWVKFLg9u|0|10-741-929-4244|4196.08|fluffily. fluffily regular requests a +1789|Supplier#000001789|zzNyqcHhaB0bDGk9kv2dEruI8gSDiu|20|30-693-246-6660|434.47| courts integrate. regular, unusual deposits sleep furiousl +1790|Supplier#000001790|rVMubJsz50CI6wNNYL2dDE|15|25-885-394-8713|7057.79|s wake. quickly regular instructions believe! slyly regula +1791|Supplier#000001791|UxRinfHXInd11tz4,vRoQvB9gQMHs9pAfOf3yw1|23|33-257-612-7366|5388.97| warthogs about the ideas unwind except the slyly ironic f +1792|Supplier#000001792|YbJ7B5b4NQ6UTAJEKm,vkRSI|5|15-820-745-6527|7448.35| quickly regular packages. slyly final pint +1793|Supplier#000001793|e37l8lvsJviZzWt5z|24|34-625-636-1651|2530.46|es nag furiously quickly final orbits. ironi +1794|Supplier#000001794|QY4ygC456AABXtvwTOlMsYUGm0HM|16|26-114-642-2939|1440.55|nticingly slow deposits wake quickly. special, final instructions +1795|Supplier#000001795|wmxB8RWp0XJpNqekpZEsHaD|9|19-853-380-2417|9352.84|d haggle. final theodolites sleep among the blithely fi +1796|Supplier#000001796|TY yGajPp5TJgQZK|4|14-368-172-6487|1204.24|deposits are quickly along the slyly even frays. final, unusual escapades affix. instructions wak +1797|Supplier#000001797|ibZNV0m2BveksMY|15|25-102-699-1785|8659.63|hin excuses detect quickly fluffy accounts? regular Tiresias wake furiously about the dugouts. car +1798|Supplier#000001798|uO6lXM3RIag8jfdkRnDHajQe|15|25-172-838-1197|-483.92|aggle carefully regular ideas. slyly regular courts integrate accor +1799|Supplier#000001799|G0gkN90wTzwZ1saDn2b7vh6|8|18-642-860-9158|4648.72|ckly silent patterns. bold, pending requests hang blithe +1800|Supplier#000001800|nP5LrFUS7NQ 3EMzUUEk|0|10-216-488-1532|6973.21|bold courts are above the slyly regular ideas. final, unusual excuse +1801|Supplier#000001801|vpfEiRZGacAH2 aCS40,JwCZH1E0|13|23-788-391-8151|2777.24|y regular dependencies are f +1802|Supplier#000001802|inpMhQlIW21|13|23-758-739-1576|2031.08|packages dazzle fluffily special, special platelets. even accounts integrat +1803|Supplier#000001803|6 hG CGshwoaY26KIIb|8|18-601-946-3727|1304.94|thely special asymptotes haggle slyly. bold pinto beans cajole slyly. carefu +1804|Supplier#000001804|EC02jKzDLPiMYFfQan3sgn|0|10-393-500-3856|7096.07| have to nag beyond the fin +1805|Supplier#000001805|cpQy4m COnESQrPhoBvWPbHnCaihk|19|29-305-293-3380|-345.82|ve the ironic, silent braids: carefully bold requests boost. regular realms against the final, fi +1806|Supplier#000001806|M934fuZSnLW|3|13-220-863-8920|4672.17|uctions sleep regular, final accounts? fluffily pending theodo +1807|Supplier#000001807|AHdxwZCPjNpG|11|21-351-617-4154|5107.29|eposits. slyly silent requests hang slyly slyly final dependencies. furiously final pinto beans +1808|Supplier#000001808|5BpK38HqFkGcR6fB8R2fJ|6|16-705-363-3885|3219.06|out the quickly regular requests cajole ironic, regular deposits. bli +1809|Supplier#000001809|4weEsVM4veMgcw2IQ4PjuuKygWRDxjJ ILBYP|20|30-463-470-6459|2711.54|intain except the special, regular packages. quietly final packages nag fluffily unusual dependencie +1810|Supplier#000001810|Njhy11HtKe lgtid0bI05kw mLkQxcFvqV8AtI3 |1|11-898-840-2232|8872.57|ross the even packages mold furiously around the slyly regula +1811|Supplier#000001811|PWGxEBBCod1q48XnM9YFRz0hxSV3FUdMjGk3Ps|1|11-386-195-3467|3308.01| daringly pending hockey players are slyly around the stealthily +1812|Supplier#000001812|GJEF5fzrP3oUYJYG5 0UTgd53Rx6uc|2|12-711-700-2101|1727.45|sts above the blithely unusual accounts cajole across the carefully ironic accounts. regular deposit +1813|Supplier#000001813|gwAsLzBQXy|6|16-552-716-4806|2254.02|refully final pinto beans cajole. dogged, ironic ideas detect +1814|Supplier#000001814|eHqfI3FhFG7gT2G,Kb21V4yr NyQhVKH04yfIyj|24|34-369-553-1930|8321.24|sh fluffily special ideas. ironic realms sleep pending dependencies. carefully ironi +1815|Supplier#000001815|Y6N5Kvq,En8L2ck0GFBBBl,fP4m|19|29-448-767-1310|-814.14| blithely carefully final instructions. bold theodolites a +1816|Supplier#000001816|e7vab91vLJPWxxZnewmnDBpDmxYHrb|6|16-392-237-6726|9624.82|e packages are around the special ideas. special, pending foxes us +1817|Supplier#000001817|,te1RWRzZWqPEVE|5|15-103-734-9639|7454.46|thless theodolites haggle. +1818|Supplier#000001818|CMzzFu9R7w|7|17-152-692-4204|6145.42|ronic pinto beans haggle quickly. slyly pending Tiresias breach furiously. blit +1819|Supplier#000001819|4cK2R1q8R8tbBp,tG9UqgYJsmdsjKj8,|15|25-200-577-3172|5125.99|s. carefully unusual foxes according to the express realms hagg +1820|Supplier#000001820|KpXwAkf9mJGAhkUcJUmWO7ggW3lI|5|15-664-895-4843|9442.87|lithely ruthless ideas. regular instructions wake. fluffily furious orbits was slyly acro +1821|Supplier#000001821|3vDimhSK5SNWns6phih5L3jlZyGW,a,0zhr3|21|31-696-483-8332|7431.29|he carefully regular packages haggle furiously agains +1822|Supplier#000001822|oEyc8RCrulnWTCFbEGmhI 5vrxqDL0oe95z0|0|10-996-906-4890|1794.17|osits believe quickly final packages. daring ideas sleep furiously +1823|Supplier#000001823|Whl3blLj9o29SyPJk ZJ7pXPL9uAvvOadEvIkd1|1|11-830-153-4103|1531.11|counts haggle express platelets. fluffily ironic accounts among the final requests sol +1824|Supplier#000001824|QRgNQm3JDybaKlOsMljhYcBVI52j6|13|23-798-490-4355|2264.49|eans. silent packages boost along the regular, even excuses. foxes print carefully +1825|Supplier#000001825|0Vn07MtX5Op|11|21-281-883-5238|8297.68|nic warthogs are. slyly regular pinto be +1826|Supplier#000001826|CNbWsPnmlVM95BXAo5j|24|34-686-140-8734|-834.71|e of the packages. fluffily final Tiresias detect slyly. special requests are about the +1827|Supplier#000001827|Lfnit3,Cm10Epus8RsvOWh7|9|19-477-545-3938|4919.94|e quickly furiously busy accounts. slyly regular instructions are. fluffily exp +1828|Supplier#000001828|mUsKWGSmdFwgQLp4Q|15|25-884-510-9495|400.32|osits. regular, ironic deposits against the even, final requests nag furiou +1829|Supplier#000001829|CC3jTZymrkpJSYqh|19|29-598-220-3639|5174.04|requests haggle furiously al +1830|Supplier#000001830|WkyC5TmwChZHlyt|12|22-710-425-7131|3100.84| are carefully. slyly express requests must cajole express acco +1831|Supplier#000001831|,556nufAzYia19|14|24-711-550-9642|6790.45|y regular packages. final packages across the silent requests wake in p +1832|Supplier#000001832|fkP5H03GIG58u3CTAqs0KiebBG|12|22-542-506-8154|74.02|refully silent deposits was. final forges impress quickly. +1833|Supplier#000001833|5Jw0JRVxg8g5wWHl20BY gxj6AC23Lx|16|26-441-871-5201|9992.26|onic deposits. furiously pending tithes sleep. regular excuses sleep a +1834|Supplier#000001834|DwLcgk66n6|19|29-893-932-2474|920.76|sits. fluffily ironic pinto beans wake furiously express theodolites. furiously silent inst +1835|Supplier#000001835|JcvmEAl4rgE|0|10-872-917-4931|7171.49|affix blithely. express requests use. furiously even deposits must sleep against the qui +1836|Supplier#000001836|kWStfDtS9Jub24xxLeoFtVzb1Ok9434wN|0|10-440-314-6740|2224.53|ic accounts cajole carefully about the furiously ironic accounts. r +1837|Supplier#000001837|y6OPRBPQnzsH5S9HVkTx2WZXejyXKL,8g2,|24|34-886-827-7602|7647.30|s. furiously express accounts above the ironic deposit +1838|Supplier#000001838|FFUsR1DLGv1pp|24|34-331-990-8521|2141.48|. quick theodolites along the regular, final foxes boost carefully across the silently r +1839|Supplier#000001839|qx3yuUFRa1yqQq9LB81BEeQVOdonLuJUjbKyXpn|6|16-360-113-7327|8846.22|e ironic packages wake carefully alongside of the slyl +1840|Supplier#000001840|8TUdSSF1q Vo|9|19-598-171-6086|511.10|uctions boost blithely before the carefully regular a +1841|Supplier#000001841|uaDclbMh7Cm95bDy,nPQyxF8|9|19-208-899-8962|7968.82|aggle after the fluffily close platelets. special deposits d +1842|Supplier#000001842|hu3DplLYCAH t|16|26-104-841-3792|5023.18| theodolites. special deposits affix slyly. blithely even platelets wake accounts. even pla +1843|Supplier#000001843|AZ ees0HgmszAkT1WVhBzbt8HhD0|18|28-386-659-6601|8670.08|ts nod. even pearls sublate closely. idly final +1844|Supplier#000001844|54sHjRuKYVAHc0Hy,d6Do5 4Urj,|2|12-709-800-5282|-818.70|ts nod quickly excuses. packages use furiously. fluffily bold packages wake slyly about the c +1845|Supplier#000001845|Qxx8BfLUs8c1D2umIcr|6|16-457-798-7038|-971.99|ng theodolites sleep fluffily across the regular deposits. quietly bold asymptotes nag blit +1846|Supplier#000001846|WkFUL,NrqRrCCUnchwis3t0o,zYbvUZ|4|14-323-286-1452|7622.68|hely regular packages. deposits are by the bravely express ideas. final pinto beans wake furiously +1847|Supplier#000001847|0GAs3udWD8IfclOsoK22YxTDtHj|5|15-843-949-5894|6500.60|the fluffily bold asymptotes. s +1848|Supplier#000001848|k9OeQO9gp1EKVweuiOyBLcP|16|26-144-261-5485|1570.50|lly bold accounts sleep carefully. fluffily final packages +1849|Supplier#000001849|j2lGnzQaxOKwO|17|27-423-487-7855|4367.75|ly ironic packages. slyly eve +1850|Supplier#000001850|u8APqIJf3BcRLnul4AsNH7ha|5|15-547-277-5192|3714.48| beans. carefully unusual accounts sleep blithely about the careful ins +1851|Supplier#000001851|NjxSzMQgc97zjqOtED31fVpC|15|25-797-496-9211|2241.43|nto beans. final requests boost furiously. blithely +1852|Supplier#000001852|yNq 3s,7SgnMyjX94eg|24|34-658-967-4317|5648.68|ag quickly fluffily bold dependencies. carefully ironic ideas cajol +1853|Supplier#000001853|,jm4zK6n,aUOCc565FFQulvgljuWk|2|12-746-491-1691|5433.94|. regular ideas poach boldly against th +1854|Supplier#000001854|e161D2fCjS51q ,b7hHM3UK1wf1I7G|1|11-745-535-4762|4165.95|ould wake furiously. fluffily unusual +1855|Supplier#000001855|MWk6EAeozXb|3|13-710-803-7566|46.35| sleep furiously. unusual instruc +1856|Supplier#000001856|jXcRgzYF0ah05iR8p6w5SbJJLcUGyYiURPvFwUWM|3|13-346-499-2268|7017.60|eep fluffily alongside of the instructions: slyly final requests haggl +1857|Supplier#000001857|DNyis8trJ9eIUG,URh,xbQ FLXT4JXR|12|22-268-590-9345|769.58|hely among the silent packages. ironic deposits nag blithely expr +1858|Supplier#000001858|rmDP0y7kgY6H|15|25-404-624-4225|7271.44|ages. even ideas haggle slowly along the unusual, final deposits. bold depos +1859|Supplier#000001859|9XjnGaQlQR,Z|0|10-150-792-8913|-937.11|n excuses boost silently express tithes. furiously regular requests sleep car +1860|Supplier#000001860|tw9w5n AwYCbIETps1ogy75zkAoWk8l|18|28-223-486-6014|-66.02|ding deposits. quickly slow dependencies h +1861|Supplier#000001861|LRsUq9GH8A5e|5|15-867-623-9256|-699.78|mong the accounts. foxes sleep slyly pending Tiresias. regul +1862|Supplier#000001862|kS0M,qxWpw2N0uUxqgvTI0z|12|22-335-142-3917|6697.54|y. regular packages cajole after the +1863|Supplier#000001863|1EOsrDVmYEaMXcLHogFo7V1Q0G218BQMx93a|17|27-544-212-9999|661.93|l packages use slyly sly excuses. packages sleep. quickly even packages shall wake. unusual cou +1864|Supplier#000001864|ExiJvugY2QxsI r5goq5So mi NTI Zj|18|28-506-589-3599|1922.90|, unusual accounts. furiously ironic foxes wake slyl +1865|Supplier#000001865|oul9HB4pB7tz32OMRgOwSYsfRdvR,jm|17|27-399-417-8796|9060.27|. regular dependencies cajole +1866|Supplier#000001866|gJ9bAJPfBjX0s5x9dU,qA|6|16-943-270-9329|1062.75| according to the quickly even foxes. pending, re +1867|Supplier#000001867|YvNHyMcy1wYi3XN|22|32-419-628-3837|6547.28|e furiously express dolphins. furiously regular accounts are fluffily? slyly ironic dolphins a +1868|Supplier#000001868|0rg0IKNvouoc15OPkmXFwTNae|2|12-256-389-1648|5194.03|sual ideas detect furiously. regular excuses are furiously. slyly ev +1869|Supplier#000001869|nogoCdaFQii,ri9rs3P8f5rPt1wVOMw9I7TmypxK|21|31-431-165-3867|9076.51|regular dependencies sleep final, bold deposits. slow platelets according to the +1870|Supplier#000001870|laa9 Fntbx4kHd|12|22-681-122-2311|-989.05|ously express accounts grow blithely. quickly regular packages cajole carefully. blithely unusua +1871|Supplier#000001871|,xEXYUvpZ1cK1qEF6cZMQyCs2JaCkEd4IUziT|13|23-819-813-7745|6890.01|counts are quickly. furiously bold accounts sleep slyly above the silent, final a +1872|Supplier#000001872|sGWaZk2cZDYyNazJ|22|32-523-692-8212|1361.61|te. requests boost about the pint +1873|Supplier#000001873|Tdzlitffjp,Qv7bB3mWthF8|3|13-110-428-1300|2571.81|silent deposits eat slyly. ideas solve fluffily across the regular packages. quickl +1874|Supplier#000001874|6CqUGrKfWNlY|16|26-962-195-7595|3357.55|y ironic realms haggle quickly. +1875|Supplier#000001875|Tle 4UCeN4gL|12|22-550-981-2256|9358.58|coys nag carefully. quickly unusual foxes shall h +1876|Supplier#000001876|lTZ6pI,RFTpOig,qA|18|28-885-858-3916|9804.43|c requests across the carefully final foxes integrate after the carefully pending +1877|Supplier#000001877|YdS3UX,LpA0KAihb9oOD8Tf9ccfoDg9|8|18-939-405-3809|1134.90|ss requests sleep carefully according to the furiously ironic foxes. slowly even +1878|Supplier#000001878|lXQbu8BN2TgGS|0|10-903-990-3612|9270.26|lar ideas. slyly special packages promise; slyly unusual deposits a +1879|Supplier#000001879|NHWlAv4FLyCE|13|23-420-323-6460|-675.51|y alongside of the careful +1880|Supplier#000001880|L3edwJISBKqE6M8yUx8nUW9Z08Maz2PI|22|32-701-440-8952|7633.84|hins. carefully final packages sleep carefully after the deposit +1881|Supplier#000001881|Rr7T5n7tDj0eRQ ,|9|19-653-840-6903|6964.63|s after the fluffily special accounts use even e +1882|Supplier#000001882|ptM8Xk8tPzo2|21|31-755-111-7289|1636.13|ic, ironic requests cajole qui +1883|Supplier#000001883|H0WkWfpMkORknSj4jveLNr4YH6Yonp|19|29-825-969-2240|7444.01| against the blithely final pinto beans. courts affix quic +1884|Supplier#000001884|9 DQynTCKNx1,hTiMBYVmwlg8KWs,U|1|11-365-480-4166|6534.52| accounts: regular requests +1885|Supplier#000001885|fAa rpWRmRte|16|26-276-509-7880|-249.51|y regular ideas cajole furiously according to the reques +1886|Supplier#000001886|sBHbIapvIqB46fw0mv3s|12|22-945-253-9885|6449.94|ve the bold, express platelets. final accou +1887|Supplier#000001887|7Z75U5i7LLuXp8GbXjlH41sCgWsnJ 0iGgcSj2mK|0|10-838-936-7432|8198.50|sual theodolites sublate furiously from the slyly +1888|Supplier#000001888|wAcemOwabF01DQpJotvhkQ2u|19|29-262-845-1530|4192.14|into beans. slyly final theodolites along the furiously pending accounts wake +1889|Supplier#000001889|eK1A7NhlGccTJw78wxR T|19|29-122-640-8760|264.01|olites believe blithely fluffily bold excuses. +1890|Supplier#000001890|LC65vJFKdQCXTwHvVMaMu4QU56GBvlt|12|22-138-829-8813|2182.77|, special pinto beans. always ironic p +1891|Supplier#000001891|nmvfVVyBMUitm1hiZv|24|34-334-635-8569|9710.42|e carefully special theodolites. carefully even requests nag among +1892|Supplier#000001892|Ax1g787n01E4eQ5FOy2rubD2UIxp|4|14-581-872-9823|5185.12| according to the blithely bold accounts. furiously enticing platelets wake after the slyly pen +1893|Supplier#000001893|dNN p6AskdHyftPQQrsG87kjT6Rq6owcLOYIvw|14|24-992-104-6804|2296.68|onic deposits can boost. quickly ironic excuses agains +1894|Supplier#000001894|dPRr4r43oK3EWdk1UmEmutjV6aXtcU|8|18-885-377-1684|-658.64|blithely bold instructions. carefully +1895|Supplier#000001895|lywAGDbk37fYPDS|21|31-952-984-5164|5108.27|. unusual, busy platelets haggle about the slyly final pinto beans. regular requests after the blith +1896|Supplier#000001896|HtLWILD3PCEz1CnQdO99uALDH 1,J3jpMDCCqE3|20|30-169-906-5613|8686.28|x furiously. packages cajole furiously final dependencies. f +1897|Supplier#000001897|tIwaK9T,jM3yIGVWR72|24|34-784-374-6875|7293.90|sly unusual platelets haggle carefully across the fluffily ev +1898|Supplier#000001898|kZ WbobBFaVqMfT9yeWQxhAAk3opvpQH|9|19-979-402-5149|2607.24|s haggle carefully ironic asymptotes. furiously even accounts integrate r +1899|Supplier#000001899|A136Zmkh4BV|14|24-225-411-7898|4832.44|ccording to the furiously ironic instruction +1900|Supplier#000001900|9ZjkyBLy84mg5wvsW34ymQFzovjt5,1LCiNvVM4T|23|33-454-146-4905|3755.91|c forges mold furiously blithely bold ideas. packages cajole slyly according to the ironic th +1901|Supplier#000001901|kCKOBx1Xsu,TJm1TCPOT5hh0M,lpR|24|34-949-909-8639|5490.24|ests. furiously ironic requests wake slyly carefully ironic packages. unusual notornis +1902|Supplier#000001902|T78aOvA9rDeXfNM|1|11-946-831-5480|4439.84|final deposits doubt furiously among the furiously ironic +1903|Supplier#000001903|2nXFdZB2h8sM54OylU7SdZ2yVJXsV2qj9841m|17|27-149-331-4571|5391.81|e ideas. slyly silent platelets above the even foxes serve slyly regular requests. daringly final +1904|Supplier#000001904|EDieupuG,VGESMDhDtd0w5SvfHK1WSgWh7JLZr|7|17-210-998-7416|134.16|tly ironic foxes according to the slyly slow deposits boost alongside of the bold req +1905|Supplier#000001905|VhOcLXQyJM,Z,,QAVNWUPVNS58O|8|18-973-853-1876|8801.33|c packages are. express, close accounts solve. blithely even packages sublate +1906|Supplier#000001906|hiT65fN0ChZx6,rVm1con4zTXDId1B5J7T8ZMjO9|8|18-738-147-3630|4978.10|usly express requests. blithely final dep +1907|Supplier#000001907|ybAHrX,OnohkqolW ZBY4YhW63IRCUtuIehrJk8J|20|30-901-567-4832|-987.45|totes sleep carefully. furiously unusual accounts after th +1908|Supplier#000001908|mUBhhzwrYBZj,2e0lh0O3lv6c3pcdX3r5MHo91|16|26-784-301-5141|867.71|s accounts cajole slyly ac +1909|Supplier#000001909|psjAOuc2l9PBkOND6sD3eObHL82SezS7emj|9|19-563-689-2789|6537.32|regular, ironic instructions doubt blithel +1910|Supplier#000001910|vih,zrhclXX1O9x|21|31-353-926-4923|2763.15| regular, even packages. carefully bold requests haggle +1911|Supplier#000001911|myRVbhhM7cDpErcA|17|27-157-437-8512|7287.19|ual deposits should cajole furiously +1912|Supplier#000001912|jBTlNhDe2hI Hfwi p2Vsz48cE|15|25-456-469-7524|-122.10|cross the slyly bold theodolites use fluffily bo +1913|Supplier#000001913|GhPX2FN84MFhHwQNjebsGyEq|16|26-800-710-7187|2812.65|ely across the carefully express packages. express packages affix according to the carefully ironic +1914|Supplier#000001914|wFmRY6QNUcQhjjt7JIGSdv|13|23-225-482-6493|3150.15|ole along the final, unusual requests. carefully iron +1915|Supplier#000001915|Ul4Ue,6fzHwyNkpSJZzyQI0Kmmx, jZU|13|23-119-101-1772|-400.20|blithely final dependencies nag at th +1916|Supplier#000001916|juC,4ej1nuxX|20|30-859-870-4119|3697.11|cial packages wake furiously across the silent, even requests. slyly final theodol +1917|Supplier#000001917|XUTxl8xOBRCD,y|16|26-610-148-7736|7348.16| about the furiously even courts. +1918|Supplier#000001918|TDY0AufRVvba|1|11-217-221-7821|1596.80|ickly ironic tithes. carefully sly foxes nag furiously at the entic +1919|Supplier#000001919|ZW1rqp6DpLXGzMpR7 iAPW4cJRHPzAGy|4|14-261-458-6539|6375.47|counts. furiously regular requests wake. slyly special theodolites sleep carefu +1920|Supplier#000001920|esB5ZkpVMLGLau3JHKrCEM|0|10-495-104-1252|-402.83|ans. express, special packages +1921|Supplier#000001921|pT5jR2mpIfa4CqqHdM82Vu,Oc2A9 EUZof|22|32-658-250-1634|3498.39| affix quickly pending, ironic deposits. quickly ironic theodolites use carefully sl +1922|Supplier#000001922|82oW0v1k Dx0f6|16|26-179-916-4721|396.49|the bold, special instructions. regular, regular platelets +1923|Supplier#000001923|cBl0vlisChmBdQc8Kc6FYLFV|6|16-241-142-7145|6788.57|imes regular deposits kindle against the unusual, final requests. quickly regular deposits nag q +1924|Supplier#000001924|eiXNykrhQK20rOe, xSxAuRiXEWlw|4|14-648-546-9457|1575.30|is haggle carefully. doggedly +1925|Supplier#000001925|Cu5Ub AAdXT|20|30-969-417-1108|8040.16|pending packages across the regular req +1926|Supplier#000001926|UP6w7yICw0aRxsRwDkXY,zleHx9q0EX9TiF3,To|22|32-259-128-9507|4980.97|y unusual, ironic deposits: furious, regular theodolites wake against the qu +1927|Supplier#000001927|vuBVbKLbea|7|17-479-239-9670|3113.39|nal requests wake carefully after the express deposits. regular instructions are f +1928|Supplier#000001928|w5ksuneIutUKiRFmRt|18|28-457-867-6302|8719.02|gular platelets wake according to the furiously bold packages. fluffily regular foxes affix +1929|Supplier#000001929|9HSveX9CpjMcvUFhEeLuaGQlb4zm2kg|22|32-384-446-8680|7580.71|ding to the regular, regular deposits. bo +1930|Supplier#000001930|2jCSw3KOLHol7y5omVO13|21|31-773-630-9937|5098.00|eans lose furiously even accounts. blithely regular waters along the ideas sn +1931|Supplier#000001931|FpJbMU2h6ZR2eBv8I9NIxF|3|13-822-248-7689|1130.03|furiously even pinto beans. quickly regula +1932|Supplier#000001932|YxTEoa6pXu, |15|25-765-538-7915|4380.19|e final instructions. slyly ironic packages must x-ray. furiously unusual requests sle +1933|Supplier#000001933|fdJmXGml0OgoNCsT5Wukzo0L4NlHPq|22|32-601-788-5798|3882.77|. pending foxes wake carefully across the silent sentiments. deposits according to the bold pa +1934|Supplier#000001934|qa227GUR5COkw|23|33-351-424-1808|3663.43|ial deposits. final pinto beans hag +1935|Supplier#000001935|lxfvnpWQ dvSWd6mSA Q1tDjspDq CP7y|5|15-134-339-9047|2272.18|kly even pains sleep against the regularly express pinto beans. fur +1936|Supplier#000001936|13g,mb 4fXHUPSpj0mAjav QORuK0UhAa|1|11-551-397-1062|5431.48|ependencies. furiously ironic packages across the slyly ironi +1937|Supplier#000001937|eJ1eYyleBJJHQZ|9|19-574-809-7821|6965.91|ent epitaphs use across the special theodolites; carefully final instructions are furious +1938|Supplier#000001938|aFMa1UzMRPAO5hsX|6|16-783-625-5753|1061.28|unts. quickly ironic deposits sleep carefully according to th +1939|Supplier#000001939|Nrk,JA4bfReUs|3|13-675-423-9518|3899.79|l warthogs wake slyly even, express requests. furiously +1940|Supplier#000001940|eBYt LFXqTqgXMH7MQ6KC2OhIn|21|31-293-442-3659|3684.75|ronic foxes. slyly ironic theodolites about +1941|Supplier#000001941|h8VV3OEcSykk|6|16-316-553-7611|8546.98|eposits. carefully regular dependencies need to cajo +1942|Supplier#000001942|sSxh15mxkySVE0W,Ot0EWNWkUZiSyDn jCro9L3|4|14-453-767-2173|9195.62|s accounts about the slyly bold accounts boost ironic requests. +1943|Supplier#000001943|ROT u2yO2E8cIFvz6a18vuW3DvF,SvHIFKESc1V |13|23-574-621-8188|9889.66|s cajole slyly alongside of the quickly express asymptotes. blithely sp +1944|Supplier#000001944|X MP6hjp7dZHg5hyIfAXqMW0pTiFg4hNZiO0M,AY|20|30-238-135-7160|3604.66|xcuses. express realms above the slyly special accounts wake according to the accou +1945|Supplier#000001945|edI,,ekDTNDdlzQYIHnhdbkw|16|26-826-634-5171|4479.45|lar accounts cajole slyly blithel +1946|Supplier#000001946|6J33 1IV5vOq0WacqWFd4uon46NXqouP5gdS6iDX|9|19-934-602-6578|172.23|quickly silent requests haggle carefully accounts. fluffily ironic +1947|Supplier#000001947|ldbSOVHowulGQRaNF3a7zTj9y5N|6|16-781-356-4399|-340.05|ackages. furiously final sentiments haggle along the silently spec +1948|Supplier#000001948|ZcX7nDmJiBlFPa,6KnbOnRf6K3H9Hhx|24|34-241-262-1496|2748.87| requests. slyly idle requests wake blithe, even requests. evenly final package +1949|Supplier#000001949|W79M2lpYiSY76Ujo6fSRUQiu|10|20-531-767-2819|9260.78| thinly ironic excuses haggle slyly. f +1950|Supplier#000001950|np0V93T1ZvPmyuqJ|17|27-204-805-2085|5003.29| theodolites. carefully ironic requests wake blith +1951|Supplier#000001951|b2sJa7BV3EV5Gg3K5q fimm,xq|20|30-486-690-8063|4300.59|arefully fluffily regular hockey players. slyly ironic deposits alongside of the silent request +1952|Supplier#000001952|lwuffu3TjEs9kAyqoSdQKY|13|23-495-879-9037|8066.45|he slyly final packages haggle blithely regular, silent packages. ironic, special i +1953|Supplier#000001953|hpv9Dsezjk8Zw7qSlnzGhMF6kkdJ9zI|10|20-371-309-5922|6825.47|l deposits. slyly unusual excuses sleep carefully alongside of the theod +1954|Supplier#000001954|QrTQJOkhS7WEFB3L5zbvDUSGh4lV oh|24|34-861-418-3829|5643.96|ans. special accounts do use furiously. blithely express dolphins +1955|Supplier#000001955|,GeC5kNX11ItX1CUtp|2|12-730-544-7513|3071.38|y unusual dolphins. blithely regular deposits +1956|Supplier#000001956|mE38JsDrQ5KpmGiFDZ7RmpBhEvJ6Vc5B|4|14-839-216-5090|6721.37|iously final ideas around the fluffily ironic instru +1957|Supplier#000001957|Dyia9 2,aIklULv4QawxAGMW6|1|11-515-919-7083|5059.14|nt deposits. bravely regular courts sleep across the stealthy, final theodolites. pa +1958|Supplier#000001958|bJAZFisYzqSFaspq2XQOVwrEpGgtTe9 1ccU9RJ|3|13-270-328-6193|7926.59|eans along the furiously even tith +1959|Supplier#000001959|8v2xjkfTj90tWl3gGTKODt|17|27-191-843-2229|-788.51|ake carefully against the final packages. furiously final packages +1960|Supplier#000001960|D3I28h7f6Z|20|30-267-821-2732|443.60|rnes: regular, silent pinto beans haggle furiously. fluffily express foxes solve +1961|Supplier#000001961|i5FV1jSc1jj2cpwKaJC|17|27-330-708-2722|6601.40|e furiously ironic deposits use quickly after the quickly even deposits. bold +1962|Supplier#000001962|UZAdG60BMZRxwNh7dmXB N9GMgGVnW|22|32-575-740-2950|1764.72| haggle furiously even excuses. fu +1963|Supplier#000001963|8Uwi,ryUWmecY2cE2gjsLj p3WAXdKy6azS5RRMM|16|26-220-467-6960|1629.99|odolites eat slyly above the furiously express requests. carefully regular accounts wake fluffi +1964|Supplier#000001964|5OlSBcKlYeqgnneJNh|14|24-907-201-3298|3182.08|ts. special packages integrate carefully even accounts. id +1965|Supplier#000001965|Tgtqgw0UVj5ULgeIBpnkVp15|1|11-456-304-1204|8277.83| unusual epitaphs. blithely +1966|Supplier#000001966|liovqzTMyBOSQpStwfizuxgBbuu1YGaKCeBfRfA|6|16-661-169-1959|6546.49|iously. furiously even accounts sleep daringly blithely final requests-- regular packages a +1967|Supplier#000001967|yvTFsXdtp197Q74meyc 4SxKnSD1A2d2dh9E|7|17-175-981-2149|2909.73|gside of the ironic, special foxes. foxes impress furio +1968|Supplier#000001968|EpUTyZuUEb6YKRCQYhGqrp0WrsSiA|6|16-140-133-7640|6860.29|ffix slyly after the slyly special accounts. regular ideas among the fu +1969|Supplier#000001969|TsaBRyvy2wao5LV2s8kEovROAVxDF|0|10-934-830-8948|7915.97|en packages. bold, unusual depen +1970|Supplier#000001970|QqyXDIxLNM,fwA8qP1TLxQBA9v|5|15-350-889-1064|7705.03| slyly at the slyly regular pinto beans. regular, special deposits affix carefully pending asymptote +1971|Supplier#000001971|MCV9Y2Kw1SwwsWIseqWl|14|24-124-678-3750|4506.47|regular dependencies-- express instructions nag +1972|Supplier#000001972|1ufqdQLhkBkRF4nWBpgbLzVP,Be0ZwwtSIVnLx|7|17-581-211-6866|3285.00|ithes sleep slyly along the pinto be +1973|Supplier#000001973|BVI,FMyON2jPHYGbYm4oN1pDw8Mx4fc4D,|0|10-918-775-4739|-91.98|posits after the bold sentiments boost furiously accordin +1974|Supplier#000001974|1R7qIuqU1C49lcz2WW3W|10|20-845-358-5508|3002.07|bold, quiet requests sublate above the blithely final reque +1975|Supplier#000001975|nBmdXXmMEmKhYwO2PWWjQur|5|15-516-365-1831|7708.98|instructions. blithely ironic dependencies alongside of the pinto beans use +1976|Supplier#000001976|lbnEmZyu58MhIomv5jqN1P4aoTwRQYqaM|18|28-244-422-3128|4911.27|ackages sleep furiously daring deposits. packages sleep quickly f +1977|Supplier#000001977|U2su,G5WmihZ|23|33-800-614-4695|8664.48|structions. even, even reques +1978|Supplier#000001978|qnOvcc36FOHCHqO0VEWCD |5|15-541-919-6147|3921.11|l accounts. pinto beans against the blithely special accounts nag slyly even, b +1979|Supplier#000001979|UNW7nA,IC 5igvVsgUHA7OaLL,jOzUcT|21|31-193-831-8780|-832.44|eodolites sleep quickly furiously ironic pinto beans. special theodolit +1980|Supplier#000001980|ZRmkzXoXZR5yT bdwUU2iQOGN|17|27-126-336-5747|-940.61|o beans sleep blithely. quickly final foxes +1981|Supplier#000001981|Q9YktbRu7FxixQiaT1U |10|20-705-553-6657|7086.42|ach. regular, special packages along the furiously final pinto beans snooze furiously r +1982|Supplier#000001982|q5g5cl4V2Ssk6vsVTtPFBo8lT8gLcQrbojDyGsN|14|24-307-672-7764|2518.95|ss accounts. furiously bold accounts affix slyly. express, ironic packages +1983|Supplier#000001983|HD5VA,1YtUHRh6EQIKVKkoG8H90WSq1HuCj vSG|19|29-703-467-6087|6006.43| carefully even accounts. fluffily final depos +1984|Supplier#000001984|M,EWRY6BhH|24|34-298-901-5740|1026.27|even asymptotes wake above the final, regular deposits. slyly permanent packages along the instru +1985|Supplier#000001985|iNpX5StxnUW8DlgToWvv9kZ Uk|24|34-968-184-3570|9542.91|sly regular dependencies against the bli +1986|Supplier#000001986|D2d8InHEo5MjZHcD,Ru|9|19-165-166-7955|5721.91| regular deposits wake at the silent asymptotes. regular deposits a +1987|Supplier#000001987|sSw4t7fWicU9U1Zpgi8z1q|12|22-196-523-5808|4101.27| furiously pending deposit +1988|Supplier#000001988|z6jUTt,p7fpP VxegSqti21 UMSYbxFAZ dRdfL|8|18-200-839-7278|2057.82|ronic pinto beans. furiously pending packages sleep along the sometimes blithe +1989|Supplier#000001989|u3dxCM57G0E|10|20-346-853-5153|-224.35|ackages. express courts above the slyly +1990|Supplier#000001990|DSDJkCgBJzuPg1yuM,CUdLnsRliOxkkHezTCA|3|13-430-427-6190|204.32|instructions use at the quickly regular packages. even ideas sleep furiously. +1991|Supplier#000001991|,PYHN7M wZeysns8|1|11-471-846-5069|-32.99|lithely express foxes at the +1992|Supplier#000001992|MRwtJAmbv8FskuOV,LbvJhrLaOp8bO|14|24-551-800-6193|4508.97|ly express accounts. slyly final dolph +1993|Supplier#000001993|9J72jXtyUhHlwat3bsNMowm4QV|2|12-484-282-1634|1250.98|losely across the quietly regular accounts. regu +1994|Supplier#000001994|Vy2W83Uop8MgGn|9|19-410-152-4401|2101.80|xes. silent packages snooze express +1995|Supplier#000001995|EUOX4y1yxYE20JLEYPc06nADD|18|28-474-543-1631|1945.95|reful theodolites. carefull +1996|Supplier#000001996|3bPc w4ZoxqIck LaAlP2DCjf1|10|20-547-358-8914|506.28|ick instructions according to the s +1997|Supplier#000001997|gUkq5gt2A4RqdDqfe3sQvKeipbzk|10|20-573-670-8952|3966.94|p bold accounts. special ideas boost carefully across the slyly +1998|Supplier#000001998|QwwByHZ9 JLIFToI4hp9qlDianI9uy|23|33-159-218-2352|-589.57|kly final deposits along the fluffily busy package +1999|Supplier#000001999|iMJGJ97d7uZ6d4R7HueaELdtwiuv8oTO|21|31-410-167-3442|80.88|nstructions cajole blithely bold requests. slyly regu +2000|Supplier#000002000|b1,LW1CcQX7Mp2RQgKXLrKGvuJmM|11|21-860-645-7227|4316.36|lar theodolites. carefully ironic packages use. blithely blithe requests print. slyly r +2001|Supplier#000002001|Z9 IztWNMiBsnQiwykwX2|7|17-362-184-9185|8978.82|s detect furiously. slow deposits affix quickly silent ideas. fluffily specia +2002|Supplier#000002002|muqP1ywikGjms8t|22|32-880-712-2281|7165.76|etimes regular deposits cajo +2003|Supplier#000002003|8xDjpcEfQNjO8vXoWm0lr315hc7DnXTo|2|12-997-589-3384|4979.71|y final pinto beans. ironic frays against the pearls are blithely aft +2004|Supplier#000002004|kJ8ah6BTTXZCpmZq7fzHEhvmpImgK|4|14-554-575-6987|4039.92|e after the permanently ironic foxes. carefully silent packages sleep enticingly. express +2005|Supplier#000002005|McKU4Du,pmL10I3g|20|30-286-863-4689|4913.76|unusual requests cajole slyly according to the carefully ironic packages. regular packages should h +2006|Supplier#000002006|ozotk0nJqLKtn4Vbi,Ig|4|14-108-742-4060|6555.64|n pinto beans alongside of the carefully express +2007|Supplier#000002007|B,hSqG4gmGyaTmwPAkSn3Ai01BQXeOGLr|19|29-112-968-9817|2745.18| furiously even instructions. blithely ironic gifts wake blithely furiously ironic foxes. blithely +2008|Supplier#000002008|jXfmC1KyP3RL|17|27-726-611-5754|2280.16|nic requests. ironic, express asymptotes +2009|Supplier#000002009|PgjqjeHyTvmb7uUJIMJiSFp1VY14X54|7|17-826-654-1306|7620.06|bout the carefully final accounts wake above the asymptotes. carefully e +2010|Supplier#000002010|NS2ccTY26K9n4QslPncPAuJ2tQZuWRHY45,XGzqe|0|10-845-970-4551|6420.08| slyly according to the pending, pending accounts. furi +2011|Supplier#000002011|EKs5znUv,GZtfhLx |9|19-368-474-1163|1234.04|s. special deposits about the quickly unusual accounts print qu +2012|Supplier#000002012|eckc4QQiaZyIQp1jW5DDDamHTp,,L5M6uP6zT0eY|2|12-211-688-9124|3466.71|unusual, final excuses! regular +2013|Supplier#000002013|DspSlENDW6CP6ln|18|28-932-819-7573|6907.24|odolites print blithely: even braids haggle furiously? express packages amo +2014|Supplier#000002014|4iyCaDCMrkIsp3atWX|13|23-458-531-6395|5068.01|the fluffily silent requests. slyly ironic package +2015|Supplier#000002015|SvLE1jxVZXpZYUI8Na2ihM|15|25-849-796-8961|3071.86|y bold dolphins are slyly against the final accounts. car +2016|Supplier#000002016|MMEOcRhawgangGIwtLlVUNi|4|14-135-829-6919|3665.68|deposits should wake. slyly regular requests dazzle carefully f +2017|Supplier#000002017|5X3irJUk8 SA0E6RokqWc06wX9|1|11-940-342-7501|6990.21|y final deposits hang. blithe instructions boost from the pl +2018|Supplier#000002018|053ieIpiBAYZoFEmnW6Wo8Kfo55,xcaSaXLV0|6|16-354-932-1816|2465.39| slyly bold accounts are quickly w +2019|Supplier#000002019|Cv1sLRqjcfNR|2|12-972-912-8922|7742.24|t the silently ironic deposits. carefully ironic reques +2020|Supplier#000002020|jB6r1d7MxP6co|3|13-808-671-7904|2950.52| the always final deposits wake at the slyly special sentiments. special, regular accounts alongsi +2021|Supplier#000002021|t89NPGwaXrWJ6Hk7K|18|28-396-825-3921|-716.29|old pinto beans across the slyly express pinto beans haggle caref +2022|Supplier#000002022| dwebGX7Id2pc25YvY33|3|13-924-162-8911|4296.26| ironic, even deposits. blithely cl +2023|Supplier#000002023|qofT29jef,Wu4PR75k5 0T8sKKyCu6|5|15-826-899-7652|1159.28|-- slyly silent foxes cajole quickly. doggedly even depths cajole sometimes. final, ir +2024|Supplier#000002024|8PEKfAVOdGCspXSXMAZln|0|10-461-607-4760|905.46|to the regular pearls cajole quickly special accounts. blithe +2025|Supplier#000002025|5PA3gpCb,udfi3yvxlFcAFaV1SnK6|20|30-862-982-6332|3061.02|ing accounts was blithely. fluffily +2026|Supplier#000002026|ZJzy4wu,lfoZugo6,cRgWvbca gFv4SzximYFeG|14|24-814-878-2691|8616.78|bold requests. idly regular excuses +2027|Supplier#000002027|QmtD6C2kQF3xRxmt2tOjIK|13|23-333-668-7439|5367.11|ate alongside of the blithely i +2028|Supplier#000002028|2surPcRgmrPV2yO7kyHMWseCBTlc47lL25u|10|20-286-838-6640|3023.21|ages maintain carefully silent, final theodolites +2029|Supplier#000002029|UthKA FYjx|14|24-503-782-1114|-6.37|ross the furiously silent requests. dependencies haggle quickly under t +2030|Supplier#000002030|nHJl8Qp1p2|14|24-805-946-5733|953.46|lets nag carefully according to the ironic, exp +2031|Supplier#000002031|JpBWuUSDxQY0GPkjS3ckp1GaGwxU|13|23-454-380-1517|6752.02|usly beside the unusual deposits. slyly express ide +2032|Supplier#000002032|r9cG90yX2I874Bw9|22|32-659-685-6157|3219.14|ular requests are carefully special foxes. carefully even +2033|Supplier#000002033|gYRnEl6exTtA|19|29-267-366-9503|4098.19|heodolites. furiously final a +2034|Supplier#000002034|deKZihoSypb62Yi,ivMpZf|12|22-763-404-5757|5343.26|ly around the unusual ideas. expr +2035|Supplier#000002035|ONhbdP6MIcvoNV6vXS4TmtF,7d6XBvuY,KF|13|23-636-350-9994|9192.18|aring requests. blithely final deposits cajole fluffily +2036|Supplier#000002036|20ytTtVObjKUUI2WCB0A|3|13-533-270-2069|6981.87|ave to wake regularly; pending pinto beans will haggle against the bli +2037|Supplier#000002037|u736keYss0RxcTKy79yefcn|12|22-779-411-3215|5354.84|ecial pinto beans affix along the foxes. final, regular accounts +2038|Supplier#000002038|kpVo,EebfCUVJd2GcC|14|24-869-278-2651|5853.00|ost alongside of the carefully blithe +2039|Supplier#000002039|fsytgC5z,XCZlnkXmEcZTy61GN|20|30-803-992-5696|7409.82|egular requests haggle across the unusual theodolites. quickly ironic instructions hang. +2040|Supplier#000002040|HGw2YsBJku,YJplIgkY|6|16-178-801-5927|3838.77|are fluffily according to the ideas. furiously final asymp +2041|Supplier#000002041|Ntraa17DNTQ8a3SE0iruP6z|8|18-506-551-5955|1153.76|sleep slyly bold instructions. carefully ironic ideas eat. carefully iron +2042|Supplier#000002042|DYafqV5X76CbMQeW9u6a5O|19|29-192-285-2351|7228.64|nal pinto beans solve finally. slyly spec +2043|Supplier#000002043|mtL9onCBkFgpCD8|19|29-623-177-4329|5873.57|counts serve quickly. slyly final requests nag blithely against the furiously express courts. expres +2044|Supplier#000002044|L8FEgPgsTG7r06OOLepJFUIOzMa,qyT9x2tq9Cg0|12|22-222-115-3803|1077.43|ep to the blithely final dolphins. blithely even requests unwind +2045|Supplier#000002045|3GVjWVjXkGXUeF82afLa,A|14|24-956-242-7409|378.43|ter the fluffily even deposits +2046|Supplier#000002046|BiTDgHknmvQGT6FpZXfRX,xlnR|21|31-837-215-6004|177.00|rding to the requests: carefully bold packages ar +2047|Supplier#000002047|xKC4244IwbLa4qUKQ|0|10-262-377-2302|2703.22|ly among the regular ideas. quickly unusual dolphins aft +2048|Supplier#000002048|oUC5NNI9LyuEk|0|10-939-898-3098|955.65|deposits use against the express accounts. carefully regular ideas sleep blithely. instructions use +2049|Supplier#000002049|r,4H5xgCiVqZx3JKQ8|14|24-214-122-2349|-467.89|gular, regular packages. accounts about the accounts affix carefully acros +2050|Supplier#000002050|62jBCB3agVo Q0EbDwOq6hiYcXl58N7z3ODngRD|10|20-432-928-3484|-893.17| ironic forges. slowly even ideas cajole regular tithes! furiously pending dependencies along the de +2051|Supplier#000002051|XQIC,vrISmxlbW,F9YtdWcMAQC4TrhXVZxKvY i|13|23-305-823-9298|1354.69|oxes dazzle slyly. carefully +2052|Supplier#000002052|nwydan6swETj9O5Jv0iVk|1|11-224-324-5343|7307.43|y bold requests-- slyly bold requests after the instructions haggle slyly across the s +2053|Supplier#000002053|75Amlumtzs nRLz1Q lMf|7|17-894-414-3923|-913.46|es boost slyly along the patterns +2054|Supplier#000002054|6YAwzeTdf4DBy5TRZa2Se1WhrUPUx,Nz|23|33-895-796-9194|1429.70| express requests wake fluffily final instructions +2055|Supplier#000002055|h14IqRaCVKW|22|32-368-197-3979|2905.10|ly fluffy theodolites boost +2056|Supplier#000002056|mffyZqn6ghX|20|30-874-824-4537|6020.84|lithely special platelets at the quickl +2057|Supplier#000002057|bpQKpW5yJwO80Wr4,4|1|11-681-744-9905|1303.42|unusual packages. fluffily silent ideas thrash slyly. final, +2058|Supplier#000002058|ogFEO0NKn1eqK43f|1|11-980-686-5901|3580.78|y ironic courts against the carefully special requests haggle quickly unusual excuses. unusual i +2059|Supplier#000002059|Oa4FWTaIHzMV,9M7TuLT7 uX3|9|19-267-585-5906|-765.06|he fluffily even packages. fluffily ironic theodolites above the carefully special instruc +2060|Supplier#000002060|sr8M4zsrs23IzGSuq8IfVlrXkJm|16|26-246-351-1423|9624.51|s platelets use. blithely regular +2061|Supplier#000002061|E5jNmsYHNNGASXwdX2FgOoeFh1qgDCk7tHX7bEh|11|21-186-577-4689|2165.35|xcuses wake? slyly slow accounts haggle regular accounts. accounts nag. express packages along the +2062|Supplier#000002062|FlxpSFOW1fQNR05i3sQzcNwpBOTqLQM0S ZEYC|12|22-757-732-7348|2987.43|ial, ironic packages sleep furiously. fluffily busy excuses are furiously furious +2063|Supplier#000002063|vwGXWA v5XyDZA6E,1zvqAJwDFuwEFOWh|2|12-108-888-3121|3760.01|al packages boost blithely never bold theodolites. quick somas for the quickly even Tiresias boost +2064|Supplier#000002064|t840Ipd5YG7SZyQQUUEQNPZ fQfjJT5lv|9|19-450-298-8597|8473.17|fluffily regular, special theodolites. brave dugouts hag +2065|Supplier#000002065|rOxmqWB37iwJu4Wpb5xl5|0|10-916-389-3232|602.89|ter the carefully special deposits. furiously b +2066|Supplier#000002066|,XyIxY2VQ,ztru|2|12-958-782-2048|-356.67|ss instructions. ironic deposits haggle blithely pending packages. closely express foxes a +2067|Supplier#000002067|9dO68x0XLMCUDuFk1J6k9hpvLoKx 4qasok4lIb|4|14-252-246-5791|9450.15|rding to the regular dolphins. quickly ir +2068|Supplier#000002068|K3Ao0QAOGRUFgUE5JH8X8rTngbLpyKDORoO|18|28-909-115-3663|8736.77| carefully bold deposits. ironic deposits +2069|Supplier#000002069|PAVLHfjvFxC6M1tJwW5WG2bfi19Cs|21|31-674-833-1391|9031.54| final foxes. regular, unusual asymptotes wake. bold requests sleep +2070|Supplier#000002070|gZ8nCVAgQIMUfoYvIaTF X|6|16-525-827-5093|2163.36|jole close, even dolphins. ironic accounts haggle slyly. slyly even theodolites de +2071|Supplier#000002071|zLH3QAtZuuOq8AoVNM|21|31-736-983-4487|2531.46|ar gifts. slyly ruthless requests ab +2072|Supplier#000002072|XBBFJ0OV0RxLBgG ,dNl|0|10-983-966-7774|7180.16|ironic excuses sleep slyly regular deposits. even foxes wake above the c +2073|Supplier#000002073|t1cHUsmf93|2|12-738-301-8859|4976.38|s. final, ironic foxes was +2074|Supplier#000002074|IwU5OEGT1iGYacID09jnAM,yzTIMpWp04fJ|7|17-248-717-5468|3026.54|t blithely. requests at the even courts +2075|Supplier#000002075|GdtAgDNIqsot8KG4wpmBx2L247IylJKxMcrRJ|24|34-953-639-8189|1043.26|s haggle. regular pains cajole carefully regular waters. slyly special +2076|Supplier#000002076|ptfVIQCHMp|19|29-598-923-9264|8975.45|; fluffily bold requests above the slyly u +2077|Supplier#000002077|Mdq61PYieVRmBm0lXRNJwbsVIqrfZ|21|31-181-415-2972|5540.45|en ideas nag slyly even, even excuses. unusual, regular dolphins according to t +2078|Supplier#000002078|mW3gJHBiSW|9|19-659-206-4469|1432.63|ke blithely carefully even requests. quickly special theodolites agai +2079|Supplier#000002079|2oUXSqxj,oPyDvoEPcgrBKinWFqgWxBa5kglvvo|9|19-917-742-4588|8190.80|ar packages affix furiously regu +2080|Supplier#000002080| sRfXXB4zBnycf478fANs7oCg|17|27-238-306-4611|6772.52|lithely special requests affix +2081|Supplier#000002081|iHnGUIqGqHYOJ,hvR9uSqQ2Iftoj3|12|22-738-508-9858|6703.36|se blithely regular, final platelets. even, even package +2082|Supplier#000002082|7v7kwZySU297XEFptPDGuBh|16|26-179-265-4597|-664.94|dolites. carefully final accounts amon +2083|Supplier#000002083|ZvyX5ooQ7t5,ijPLuLuSuVodJNSx9ZGC|7|17-365-265-7297|9308.14|the bold packages. furiously unusual theodolites doze along +2084|Supplier#000002084|,zeVVYFb6hYoWyJ4kjqm8hT5lzLlW|5|15-241-499-4185|9014.90| deposits haggle doggedly among the furiously even packages. d +2085|Supplier#000002085|iNUkZfe1jzha0Bnaw4PX roaeI |12|22-494-673-9808|8461.05|ar requests wake deposits. careful pinto beans are. silent deposits sol +2086|Supplier#000002086| ,yBCAhvKWP21ZO1d94zY2Rcl46Z|23|33-257-695-7480|5866.79|e quickly silent foxes impress regular, bold instructions. carefully enticing deposits along t +2087|Supplier#000002087|HGgxYKF0 EkGjGK|5|15-542-152-7965|8153.16|ven sheaves serve slyly agains +2088|Supplier#000002088|3OY9G5JC36TDMOwQ6yLXF1Kc|14|24-159-426-5821|1169.14|lve idly regular accounts. re +2089|Supplier#000002089|n9HUt4McToc1CwevdABlCIPQ6wCOAw4|2|12-510-621-8936|1750.65|dolphins wake furiously deposit +2090|Supplier#000002090|mvxvcstFjFhPlOhiVcB1Ycr8|13|23-450-260-4748|7110.62| pending theodolites. quickly u +2091|Supplier#000002091|1KX2xPHrrdEwh7veDf7KBe8pnz|9|19-648-583-6246|2311.11|ng instructions are slyly. quick +2092|Supplier#000002092|ZykceCxEC5HdQ9KgWsQZo b,6nyURU7RG,,hD|2|12-100-819-9559|-200.10|ly after the furiously pending ideas. q +2093|Supplier#000002093|6vmM10xY771fxGWt3H xYNWHE80,an8xTB5N|19|29-273-500-9147|797.46|. quickly ironic depths cajole since +2094|Supplier#000002094|kEZofuvhGD8csZWD0|2|12-346-722-4762|2863.86|ly carefully bold excuses. accounts wake carefull +2095|Supplier#000002095|EsDTz0mR5I|20|30-525-757-3604|5951.25|final dependencies sleep blithely. qu +2096|Supplier#000002096|kuxseyLtq QPLXxm9ZUrnB6Kkh92JtK5cQzzXNU |3|13-621-558-5424|8661.02| slyly. ideas cajole carefully quickly thin braids-- speci +2097|Supplier#000002097|wCI0NPCvbpLQctpTYOm3J0XDr9ytCrzrIXnSJpZ|4|14-858-312-3546|861.29|requests are furiously final deposits. ironic, +2098|Supplier#000002098|2MCSyt8XGXTLfXRzKGf7|18|28-314-281-1034|8664.66| even deposits haggle alongside of th +2099|Supplier#000002099|xpHU3PIuVz5UfoiaKiQtIxqbmln5zf|0|10-763-945-1271|7043.94| to the final instructions. requests sleep quickly along the pl +2100|Supplier#000002100|oBp4v4AGuXeRh6tXeP|4|14-765-745-4694|3586.55|otes nag packages. even d +2101|Supplier#000002101|WPQIxQOridhW55 NLbnQm|18|28-766-434-2379|8161.41|counts. special requests according to the ironic ideas sleep fluffily fu +2102|Supplier#000002102|1kuyUn5q6czLOGB60fAVgpv68M2suwchpmp2nK|11|21-367-198-9930|7910.16| accounts after the blithely +2103|Supplier#000002103|6RnHhF2YEr2p3tBPCxAnNQsw,apIQyilDz|24|34-998-551-1845|9105.42|riously unusual instructions. ironic epit +2104|Supplier#000002104|4Ebf2rn8VwG|16|26-273-834-7525|787.20|ts. pending, sly grouches use bli +2105|Supplier#000002105|cChddPk6cp|16|26-296-815-2017|6486.76|ely special requests. slyl +2106|Supplier#000002106|ADXiG5y3MX75DmRvtlSKvjK |6|16-609-473-8406|69.05| furiously silent foxes. bold platelets +2107|Supplier#000002107|Yyk00UY jkOBQgxJosIcP|1|11-908-103-6980|3497.99|uiet theodolites hang carefully ir +2108|Supplier#000002108|AgLCqSngNY,CE|15|25-771-395-3556|4652.45|ly regular hockey players according to the ironic dugouts detect accordi +2109|Supplier#000002109|deW6ISdcFQkhbJqqbHhk9,ADM1SCfrdsvHzSvB|5|15-125-731-1831|3755.77|al instructions nag slyly above the deposits; blith +2110|Supplier#000002110|56LKTbXb,3|13|23-988-935-6591|3179.93|counts against the blithely unusual sheaves sleep ironic +2111|Supplier#000002111|4SAA5C3tUiXmrYIgZxruUmxRfFHZMRoOZ7,RxA|1|11-737-158-4268|5010.93|cuses affix carefully along the even foxes. express +2112|Supplier#000002112|McUW2in3FZWMkplg|19|29-740-945-1203|723.12|. packages wake furiously. +2113|Supplier#000002113|s1ZhpldDRY6Z1Uke1oPcyqBfeEL,Ly1z|17|27-164-147-6160|7449.81|st the accounts; ruthlessly final deposits ca +2114|Supplier#000002114|oYrzEAWh3pExO|18|28-461-289-4367|5043.92|tipliers wake quickly frays. blithely iro +2115|Supplier#000002115|CdIw7T8f1ur yJmvm|15|25-293-741-8460|5102.14|y ironic dinos are fluffily after the never bold foxes. regula +2116|Supplier#000002116|02IsqZ0CQCV HoKhpfyZ4UMThyCXQlfWK1|2|12-448-646-9912|4630.02|theodolites. quietly bold requests abov +2117|Supplier#000002117|MRtkgKolHJ9Wh X9J,urANHKDzvjr|3|13-319-805-9736|718.75| fluffily furiously pending accoun +2118|Supplier#000002118|PtAhBWdS5kAzACf,yuS KmPR FlWJSmPMw440A|7|17-384-929-1914|-44.49|ld multipliers affix. carefully silent accounts cajole quickly. final dugouts sleep furiously ex +2119|Supplier#000002119|E8p6oQMDNR7,NNGxqoGDZsoood9sYFt6NEH,|8|18-518-384-9729|9159.04|osits boost quickly. deposits according to the furiously idle excuses are against the blithely qu +2120|Supplier#000002120|t0pSXXQ38Rkb9EZ7GSDBX3nPRrG8kLiv2OSRr|4|14-806-267-7554|7.93|inCustomer requests daRecommendsully +2121|Supplier#000002121|0IAnSj9cJjL1sS6KGbx8Qh832VNYN3y|8|18-110-119-1013|-757.47|es are slyly. requests kindle finally packages. furiously bold deposits throughou +2122|Supplier#000002122|14qlDMJhKdsrcpV aU|1|11-783-408-5959|5166.33|ajole blithely bold, ironic dependencies. carefully pendi +2123|Supplier#000002123|xN nQ0vbZ1LUSUKX7s77 A FNiN1J9C|14|24-554-806-6564|5601.41| closely express realms. care +2124|Supplier#000002124|mT2TCWCpXJFg0ISAPlvbQPwrj Gd|22|32-731-692-4007|2938.97|platelets. fluffily pending deposits boost. carefully even request +2125|Supplier#000002125|DdeDFynrHD|9|19-758-774-5095|3318.84| packages nag quickly above the bold, final Tiresias. express foxes acco +2126|Supplier#000002126|eFM3Kvs3B8WrEi1kyx,7,WQ,V|14|24-896-206-4217|7734.58|tipliers. final, express requests are blithely bold instruction +2127|Supplier#000002127|bfBD88wohtla|8|18-992-731-7412|6123.02|eposits after the slyly regular pains detect carefully express requ +2128|Supplier#000002128|6TPUo9sOHU NyT kkbTXv61 4YXbteyhNIi|1|11-399-786-1151|2609.04|closely close deposits. requests poach accord +2129|Supplier#000002129|bXwx9YXnh08M3EiQe|8|18-204-903-3964|2036.02|he deposits. carefully ironic packages use slyly among the blithely regular accounts. regular, iron +2130|Supplier#000002130|wt8iKVoTG2 EiAhqat4oWvm6VSNoZp|13|23-963-494-9186|7308.67|y ironic packages. slyly regu +2131|Supplier#000002131|TWm8VNv3,DhkeK6A3Dorsu2,v564zzHgn8|0|10-227-104-1206|8838.89| accounts about the deposits use carefully after the dependencies. speci +2132|Supplier#000002132|zHk4g0UXmvKDpg45kmXJL6L7F|12|22-206-137-1944|9239.51|the theodolites mold furiously final request +2133|Supplier#000002133|7jVZkJSH,kwQow 6|23|33-166-898-3786|-556.19|eas. quickly bold deposits sleep blithely. fluffily even excuses sleep fluffily ca +2134|Supplier#000002134|2DRjGuS7JFDWRjRwMzZotVq khUDnDUJ|14|24-307-765-2679|-41.25| among the final deposits. furiously regular theodolites are blithely q +2135|Supplier#000002135|Y4pCXfDzFcQhpNWnEmcKfM4flq2sIE8cWtrDVE|9|19-668-614-7265|8713.53| even deposits are. furiously silent waters boost sl +2136|Supplier#000002136|JlqwzADX kuO2KNMHn9wxRGJX7aSY71ZWv6|4|14-235-236-4239|3445.50|ges x-ray furiously even platelets. bo +2137|Supplier#000002137|bq7n09XCqRQbyJ9YrejrU dPlUqfp2m|23|33-563-538-5564|2457.03|hely unusual realms-- regular pack +2138|Supplier#000002138|gccGupUzewLfMPfrztLFAoZwstgAun|15|25-735-863-1467|-765.42|platelets sleep blithely. pinto beans use slyly patterns; permanent, final requests integrate blit +2139|Supplier#000002139|G,W L ld7P 8p,gc2kxSWba|8|18-542-888-6924|3439.29|ickly regular escapades. furiously express deposits shall have to sleep across the fur +2140|Supplier#000002140|0tAvFAWv c3|11|21-202-416-4283|8062.15|ic accounts kindle carefully ironic deposits. warhorses cajole quickly. slyly silent account +2141|Supplier#000002141|Bi0LDu9ismJcMx|13|23-109-935-2681|567.67|integrate blithely slyly final packages. blithely express foxes de +2142|Supplier#000002142|5yDT8,tbsgxBG7ojieb97bAky5r5JaLg8icR,y|21|31-874-974-2673|1808.85| special requests are doggedly. slyly ironic instructions across the furiously special courts ca +2143|Supplier#000002143|Zcaa9iYhYaRhLR|1|11-501-507-4162|6328.84|to beans. final, express pin +2144|Supplier#000002144|ev3dZzeUemEOpmx7l7j2HyREEGv|14|24-503-469-8284|7844.17|ial, regular foxes haggle blithely about the regular packages. furiously idle asymptotes haggle. +2145|Supplier#000002145|,4oLt9R33bS8YBg,Zt1B7t6EbzF2um8TMdGfi9AK|10|20-993-496-6890|0.87|t slyly blithely even requests; carefully regular accounts cajole furiousl +2146|Supplier#000002146|awlVhz3J JiQyVHxvjh2ukSleqHuNfXXtqh,gHKY|2|12-356-888-3821|1591.23|uests. silent deposits against +2147|Supplier#000002147|ZuMql8hTTEer|5|15-600-322-9441|711.47|deposits. requests are furiously unusual pinto beans. ironic gifts sleep blithely final +2148|Supplier#000002148|yVDlpqCxG10tM8dm,NNVKv|24|34-880-232-8537|9627.72|ng deposits. express platelets use among the accounts. furiously even theodolites affix quickly und +2149|Supplier#000002149|7psmWKO5V2kZibASWBzU0oK5Rog5str4cDU|24|34-563-553-1854|1170.51|ly final accounts. deposits integrate blithely +2150|Supplier#000002150|4BB8gg2kf3OSmmFn2kglleRYJ|5|15-313-570-1950|2656.11| haggle slyly. blithely regular ideas sleep quickly perma +2151|Supplier#000002151|pwgaF8lCPGvZr4638PeSz|11|21-107-968-9595|1281.34|nts print slyly. quickly final foxes wake slyly. blithel +2152|Supplier#000002152|K4jEHXF,fhFizp,4fR2f0,GLPY1IJ|6|16-240-121-5978|6430.00|al theodolites. slyly final accounts haggle. quickly regular accoun +2153|Supplier#000002153|fvGJKuSkubeTJU8wOG5M|5|15-747-316-6887|2975.60|ests. special pinto beans unwind furiously above the regular accounts? even, even ideas are bl +2154|Supplier#000002154|2VL iJ2kCPHBl00eBuEoOl5yP HuDy|6|16-372-645-7267|2802.26|refully unusual instructions. re +2155|Supplier#000002155|caUd3klutQhu|11|21-880-146-1088|4590.33|the accounts; fluffily ironic depo +2156|Supplier#000002156|7aQ psAL3n58R|8|18-105-354-2253|4617.70|luffily regular packages. fluff +2157|Supplier#000002157|7Q5LjczVaV4lHl8XnbHTAZVupfk4i7vSp|17|27-236-706-7304|2743.34|thely bold requests wake along the accounts. carefully regular pinto beans could lose. sl +2158|Supplier#000002158|VxglnCKPCfx71cz0Lr|24|34-830-332-8763|4369.44|he final platelets. furiously final ideas lose after the sometimes +2159|Supplier#000002159|DhZwT2g62r5JoS|7|17-496-146-9282|-898.01| along the stealthily silent asympt +2160|Supplier#000002160|eh 5WtEASNKzD|20|30-413-142-9482|8108.14|eas. final dolphins are along the sheaves. busily express instruc +2161|Supplier#000002161|4bt4kkI29uKp8BowMqD5LiWeUoIT|7|17-321-495-2086|3560.35|alms eat slyly about the regular, bold dugouts. final, regular accou +2162|Supplier#000002162|6ya g3MW991n9JfhxSrvgM|6|16-859-508-4893|7763.74|eep slyly ironic accounts. +2163|Supplier#000002163|7,2jkkF7Ql|13|23-700-903-4782|1244.33|xpress instructions boost blithely above the +2164|Supplier#000002164|dHpsS5Q u3APFpi03kXi ouPsMd3nozIi|17|27-162-482-9692|486.55|leep carefully carefully ironic packages. special, even requests amon +2165|Supplier#000002165|R97gTJLC6froCOjvX|13|23-967-652-1331|2461.84|ct furiously always regular accounts. +2166|Supplier#000002166|AJE9Ch04InHtwRV|24|34-759-131-9038|3073.74|lyly slyly regular packages. packages use slyly dependencies. quickly final accounts about the +2167|Supplier#000002167|,DyX4f,RHR4EnsZgmw4cepMaOVdjMhwrC|9|19-375-315-3210|2381.43|urious ideas are slyly. silent instructions boost furiously. fluffily regular foxes +2168|Supplier#000002168|e7yPs4yiFCdjHDiStA9Xkb2eUiz|12|22-144-605-9504|4616.71|y regular dependencies cajole stealthily silent instructions +2169|Supplier#000002169|JKBnBaxV,0z4lwgBupYpPWIIaoCPUG|11|21-931-513-1870|8096.36|ely permanent pinto beans are above the requests! quickly final pearls are regular, ironic requ +2170|Supplier#000002170|Svxza OMMR2wreaBWPnjCscdmJHE,yy|11|21-630-104-1587|3627.16| deposits. fluffily special accounts affix slyly. quickly even frays poach q +2171|Supplier#000002171|zk,g3KRF yZL8NbFZMrrAXWxsNfw5koUenMC|18|28-725-347-5772|8624.28|nt platelets wake. furiously final a +2172|Supplier#000002172|OEtLtQ9aWxB,pCRV0brBTaqEEhatnULDNFZyiGnn|13|23-765-954-6822|-692.51| requests boost according to the furiously even orbits. furiously ironic requests haggle regular ac +2173|Supplier#000002173|, nK6DzIo1Dpyz0VhOuPJZ|14|24-895-185-8909|7933.98|onic packages! furiously pending deposits upon +2174|Supplier#000002174|xzOhWsAOU0knDpclr0SuPz7h,UCboDZ|1|11-235-588-6823|2472.94|t the carefully ironic packages. furiously pending theodolites cajole finally +2175|Supplier#000002175|ieMsCCBs2PWKsAJfbHwHS SL|18|28-659-560-9793|2604.99|pending deposits sublate ca +2176|Supplier#000002176|3CEC2V6bwVMq05nVAOLzrDBXAIhZ4e|16|26-713-680-7576|115.78|requests cajole about the fluffily regular accounts. pending courts after the ironic packages cajole +2177|Supplier#000002177|6O,8q,u1FLWiKGGZmfhGg9ooXl1AHARWZIti|0|10-750-359-7898|7872.65| sleep furiously quickly ironic deposits. blithely express exc +2178|Supplier#000002178|,7s3dA5Me2xQGXOpkNr56U6W37|16|26-113-962-5018|5700.44|ctions believe blithely pending deposits. foxes breach furiously even, special +2179|Supplier#000002179|1bSbNinI5914UbVpjbR8|6|16-270-342-6959|6273.03| express dependencies. unusual deposits should have to wake blithely final requests. fluffily regula +2180|Supplier#000002180|nBhjih4KN3Rg7B0|15|25-286-788-4765|4774.92|ntly ironic requests. fluffily quiet depe +2181|Supplier#000002181|RoBanQQ2CwVL4rIXA4|7|17-341-958-2694|5059.47|s nod furiously furiously ironic requests? regular theodolites nag fluffily packages. de +2182|Supplier#000002182|nF,UVhLaYj0B7j bhbKoaf9T|5|15-415-986-8042|2468.90|ven foxes sleep according to the regular requests. ideas cajole furiously alo +2183|Supplier#000002183|5nu7FaQYQgnPg0c3my0pyq|10|20-571-687-3289|748.47|uriously final asymptotes must have +2184|Supplier#000002184|t2Y8XM5flj|18|28-665-517-6175|4397.36|jole furiously daring requests. special requests sleep across the pending warhorses. carefully qui +2185|Supplier#000002185|ir25hJO802yN19NjyXPwlDf5|19|29-142-107-3683|6361.52|e carefully bold accounts. even accounts nag quickly past the carefu +2186|Supplier#000002186|VKhSL06Ul4kHi4hBkK8Zm3|5|15-655-903-2294|7146.03|de of the furiously ironic +2187|Supplier#000002187|5SmzRnGn8,fzW1KllxYzi4CXLNLiSX3k8wsIj|24|34-134-791-1048|3329.46| slyly thin instructions after the ironic ideas sleep furiously pending deposits? p +2188|Supplier#000002188|K2mhHbsCOZY6lV C5kYh0z9|17|27-122-809-9121|-796.58|heodolites use blithely. carefully bold instructions sleep furiously regular, unusual d +2189|Supplier#000002189|L0ECl0z03tjdB41vSGduiAPVrEcC6PAaDvP|24|34-439-957-6234|7758.00|s. quickly careful pinto beans cajole blithely final, +2190|Supplier#000002190|J fhViwOg6xww5C07Vr|2|12-935-652-3388|3730.66|refully furiously special deposits. blithely ironic theodol +2191|Supplier#000002191|9Qx wIa7LoVuaF2wjTBebj7c40OUg9IkZUQJ|14|24-554-959-1165|-838.46|jole carefully blithely final requests. silent excuses nag. sometimes +2192|Supplier#000002192|KWu5GK8oZHC7tP|24|34-619-665-9112|7924.48|ly bold deposits. slyly pending theod +2193|Supplier#000002193|1DRgmH53Uz|19|29-298-403-1384|7775.18|, bold Tiresias cajole furiously regular dep +2194|Supplier#000002194|HlAexHmxKXQNawzR1EN|7|17-821-907-5084|6462.60|es wake whithout the regular pains. orbits are blithel +2195|Supplier#000002195|aDZzLrVyjTGTrQkFbZIsVRz HvhC4Qvd|18|28-154-128-8030|1023.12|ns engage. carefully final instructions a +2196|Supplier#000002196|wdSDqS1UVwtfkm9|2|12-627-149-8954|569.55|hely express dolphins. unusual ideas haggle alongside of the quickly r +2197|Supplier#000002197|BwaIAeAPySUFnuppcYpR A0lqn05d3,nttvEu4|16|26-519-209-7510|337.53| silently express deposits. quickly regular packages sleep bl +2198|Supplier#000002198|Jw6nnnURJOzFeimLS|10|20-472-918-1953|5807.59|ole express requests. furiou +2199|Supplier#000002199|kWfU i6u5Rl91zmMkS16Dl|1|11-949-843-4841|3241.14|egular pinto beans. fluffily express requests nag across the fluffily even excuses. care +2200|Supplier#000002200|b764g4P1xzI8czRKa,7jAwqtml,vs6JIIkB|19|29-448-618-4915|-228.32|ss accounts. even deposits cajole carefully. packages around the ironic, ir +2201|Supplier#000002201|qijUKP86TLnIMjrL|22|32-191-109-6823|-470.28|sual foxes. ironic sheaves mold furiously slyly regular packages: packages cajole +2202|Supplier#000002202|l3CTXqUqnR67po0RNhF5|6|16-619-459-2620|2953.82|yly along the express, bold asymptotes. regular, silent packages integrate. +2203|Supplier#000002203|aIY2DYGsqyvzpzfdWMb|17|27-500-333-9733|-242.00| regular pinto beans sleep sometimes. final, ironic foxes sleep sometimes. furiously reg +2204|Supplier#000002204|uYmlr46C06udCqanj0KiRsoTQakZsEyssL|3|13-529-779-6608|580.84|uthlessly express, regular deposits. slyly special asymptotes must have to are +2205|Supplier#000002205|lk288b BoRGIySRu 7YOAEUeSh|10|20-499-510-7961|3133.66|ending deposits along the closely final packages boost enticingly against the expres +2206|Supplier#000002206|e3Lcyqksu2lRJu8BfUwZPYGo|16|26-433-565-1419|8371.80| special, express accounts +2207|Supplier#000002207|7UFNOe4Wt,tSg2bILer,gTn18VZNAwVL KAv|17|27-214-879-1650|1835.71|ns. courts thrash furiously. foxes sleep slyly: furiously express accounts acr +2208|Supplier#000002208|N3a93D2ALpD4hf 8DicAAi KZG|4|14-369-691-4892|6699.38| ironic accounts! slyly even ideas are fluffily. carefully special deposits sleep qui +2209|Supplier#000002209|aFowPed G cs8MH DvvqHQhH49XC08DHwTr,Jzil|2|12-524-356-3168|9487.41|ests haggle carefully under the final platelets. unusual, final packages cajole furiou +2210|Supplier#000002210|0d33LZm8ZLYk75cFPd7jxAYZb6|8|18-112-483-3374|7994.45|g accounts use near the theodolites. carefully express deposits cajole furiously +2211|Supplier#000002211|jnC20nWkKJCMKwefjbyUTVC5Zd|16|26-262-191-6279|6830.25| the quickly ironic requests impress carefully ironi +2212|Supplier#000002212|xqhdX258o1LfvMBdfu4ecQBwTEfnQVpdcGjj5fj|11|21-100-454-2429|9036.49|y carefully special theodolites. fluffily special deposits sleep carefully: blithely si +2213|Supplier#000002213|oDAPzQXc67gaRNIqaYH39RQMi|9|19-401-601-3714|8415.03|ironic requests are furiously id +2214|Supplier#000002214|0vXe0cRJIkdIkEQHrFXRXEaGPlPaWO2uOlSaah,|21|31-570-567-1838|2391.41|ing asymptotes. quickly regular requests sleep blithely idle, regular ideas. sauternes integrate qui +2215|Supplier#000002215|CI8gUDb,vXUF39Pq6Py|21|31-535-737-6571|7342.10|y. packages are. platelets kindle after the bold theodolites. ironic, furious pearl +2216|Supplier#000002216|8 AciCfJsC6l2PC3ELfiFMCJDSswf|12|22-260-421-5259|9784.19|e sometimes special attainments. quickly regular deposits amon +2217|Supplier#000002217|OKKxE,bL1tm2olSlad|5|15-801-159-5004|1944.08|t across the carefully regular pinto beans. carefully ironic requests sleep. ironic cou +2218|Supplier#000002218|nODZw5q4dx kp0K5|3|13-587-588-9916|2164.76| fluffily even requests sleep quickly +2219|Supplier#000002219|ShNYaJlJuC|5|15-855-925-7829|4082.86|inst the packages affix furiously according to the stealthy +2220|Supplier#000002220|,TFjMgaq vSThlYiKHcD mTTuNEKmEf37X1g0A|2|12-308-506-8546|559.14|s use furiously. quickly ironic ideas nod furiously. closely even deposits +2221|Supplier#000002221|2VMpv3tW0Gvg6Y0gTeRHa7jYlRXVA|11|21-889-413-9663|1469.33|ggle slyly about the final deposits. even accounts after the final braids wake above the ironic +2222|Supplier#000002222|vNEn6nM4vI|14|24-452-943-1627|8973.57|ly final packages haggle aga +2223|Supplier#000002223|cFqPbhaW4Nwm2cGw,vY|17|27-714-734-2919|6926.46|ests. carefully express pinto beans solve +2224|Supplier#000002224|08hDtcJ8EUZ0nMxMrZiEiNH9RM08gIqt0DAst|13|23-714-590-7668|9060.44|accounts sleep. express, express packages sleep fluffily at the quickly ironic foxes. slyly unusu +2225|Supplier#000002225|eBLDBdK1NpeIfBwP|20|30-440-141-2654|3940.22|ar packages haggle slyly along +2226|Supplier#000002226|rkAwkWxiDTRY9qsBzcKokXjaiR60WhoL3ptD|13|23-592-231-1431|1256.63|unusual dependencies above the furiously special packages +2227|Supplier#000002227|Xf9mtueSmyuGS0yMlSt|6|16-728-980-6194|9069.21|its. always even instructions affix slyly quiet packages. furiously bold accoun +2228|Supplier#000002228|,JV0VPi607LwPPFPM3alORcB4eh8Vb|16|26-765-912-9503|8218.62|y against the even requests. blithely even deposits cajole daring +2229|Supplier#000002229|2i8gsU1RJfqUBJmb6HDEQ4|22|32-273-826-7792|1344.52|instructions nod quickly regular deposits. bold, unusu +2230|Supplier#000002230|mE,ko1,0mMz1 sc|19|29-334-821-6402|1300.29|deas haggle furiously across the slyly even foxes. pending deposits be +2231|Supplier#000002231|WIQCMu3Jbl9wdGTa9MBrs|18|28-163-192-9960|5973.38|sheaves about the blithely regular foxe +2232|Supplier#000002232|xrkvsYF2J d1t|7|17-899-899-7619|6689.07|eodolites solve slyly along the bold accounts. tithes sleep blithely along the slyl +2233|Supplier#000002233|pyKMeYFHVDx61EeR|5|15-509-673-3425|1191.31|pending packages are carefully silent accounts. special ideas print. silently iron +2234|Supplier#000002234|,Cb2T41MV4W4au vEhDGOuyNtVs4Y2O2|11|21-741-771-8869|287.49|ever dogged packages sleep blithely instructions. bold packages boost flu +2235|Supplier#000002235|lAz4R8mtsBtqsu|14|24-678-846-5103|712.26|ffily stealthy instructions print carefully against the final +2236|Supplier#000002236|hoBL6XyHJ4GY6P|9|19-207-926-8639|835.92|carefully. furiously even frays wake carefully; final, unusual tithes across th +2237|Supplier#000002237|N7GSBC 5tBn9gywu|14|24-968-977-2270|313.28|ending packages. quickly ironic dolphins cajole final, unusual accounts! requests use blithely a +2238|Supplier#000002238|2C 77e62vlkKIv8lWx 0n2v9UZn QE1F|13|23-321-491-8444|5244.93|regular requests. slyly bold pint +2239|Supplier#000002239|NL9w9GFCzq6N|0|10-191-563-6127|6552.43|d the ironic, pending theodolites affix +2240|Supplier#000002240|hTJM3PXSdUuF0wFb0geX|2|12-390-860-8618|3656.52|ly quick instructions across the quickly regular dependencies unwind carefully +2241|Supplier#000002241|iGV,9clV9LwufQVVqiOO1plFx|3|13-275-397-2653|7103.28| requests integrate furiously ironic instructions. blithely regular theodolites hang +2242|Supplier#000002242|E4nWqXu2DqBe7 88D5B3YgFf7Ke8 xdqk|12|22-595-186-3819|-210.05|pecial requests lose among the even pinto beans. quickly even dugouts at t +2243|Supplier#000002243|nSOEV3JeOU79|3|13-549-671-9272|9666.34|lly alongside of the final, regular requests. carefully regular courts integrate slyl +2244|Supplier#000002244|oQI S3tp0bPensl5vKob7J8ueFa34c3Mxr|15|25-613-324-4089|1234.91|ideas wake. blithely unusual theodolites wake furiously. enticing requests use care +2245|Supplier#000002245|hz2qWXWVjOyKhqPYMoEwz6zFkrTaDM|3|13-943-980-8226|7013.25|eep fluffily according to the carefully pending requests. ironic deposits solv +2246|Supplier#000002246|GXk1ttT7 7BgRhvK X,7t|8|18-257-298-6662|4801.02|olites haggle quickly. packages maintain furiously pending pinto beans? asymptotes sle +2247|Supplier#000002247|sBGZhF UGzdk6st1ryZYuv 1Nkv1mVi|18|28-209-560-2049|3700.56|ajole special deposits! quickly ironic requests print carefully even, i +2248|Supplier#000002248|7ROwPIg6fkE|11|21-154-766-2784|-732.59|, even pinto beans. regular requests cajole quickly. blithely even requests sleep slyly furio +2249|Supplier#000002249|lR3,fjyJbUjqJER7tUq7Ux7BDrci3NyBUuXx|22|32-295-610-8194|5286.39| slyly regular asymptotes are. fluffily regular foxes according to the special +2250|Supplier#000002250|2BW8w iYp9DThJ,PfIqib3iwz|1|11-700-606-9453|-547.48|ans. packages according to the final, regular packages nag against +2251|Supplier#000002251|mL4HUukOkAgo1MUZ47xT8zC9,98ULRft6FxUfB5S|4|14-333-567-1078|3364.72| regular accounts unwind fur +2252|Supplier#000002252|Y 6If6HgqdTOEkGq9|7|17-200-114-9951|8758.03|nt platelets wake bold, even dependencies. packages +2253|Supplier#000002253|ndzDYSa7RzMSOk|11|21-414-387-2720|3328.89|sly slyly stealthy accounts. +2254|Supplier#000002254|LN4oMYUDiEig8PtRwFz |8|18-622-291-6863|646.99|cial ideas. ironic instructions detect slyly bold platelets. pending packages according to t +2255|Supplier#000002255|wlWWfVz9r2JiLGCR|4|14-314-708-7222|3884.10| packages according to the unusual, pending s +2256|Supplier#000002256|glIwS28AwJmxvQIRKFDVC,SMdsLtC AdoS|20|30-143-496-1014|4073.97|ss the blithely special excuses. regular platelets are ruthlessly. blithely regular pinto bea +2257|Supplier#000002257|vbr6RiyybhW4nqIxTayAPGZnLE4zeE|23|33-304-497-5207|5124.70|ular foxes. blithely regular accounts do +2258|Supplier#000002258|bXE2ygKSjCOsmijmUy5TbL|0|10-205-637-5432|9825.40|eposits are fluffily carefully silent theodolites. fluffily silent packages +2259|Supplier#000002259|nOt4M6WnJb7Rrc,kzs qqcX5pJHq2CQLkMGw1w|14|24-510-136-8669|1565.92|deas boost about the requests. special, unusual accounts dazzle daring +2260|Supplier#000002260|F1QG3S04NFHXVW0s5|4|14-828-142-1046|4398.75|intain. slyly final deposits use above the asymptotes. quickly brave reque +2261|Supplier#000002261|jhGiJYH8xFOniPzhoB4TO4HV,CEt|21|31-965-646-7762|7866.26|oost. packages nag slyly. regular, idle deposits wa +2262|Supplier#000002262|8e6XEkB 5A,m8eqAE8QPEsuNIlFLpyNLTUq4bfE|9|19-636-113-4365|8296.71|ecial accounts. requests along the quickly brave dolphins solve carefully +2263|Supplier#000002263|eowPbcmPy9jRFwmavlktYOCm7FwTL58Z58DBN|18|28-438-817-3756|4903.47|posits must have to are furiously. +2264|Supplier#000002264|dBDsb7P6EcVLsX|3|13-392-660-2172|9678.36|beans. bold, ironic packages wake slyly special deposits. carefully ir +2265|Supplier#000002265|sYKnh6xGOOFl,D8Jjch4vqcSZ 0,sgp|3|13-926-156-5479|5282.94|play carefully bold instructions. fluffily unusual excuses would boost. unusual id +2266|Supplier#000002266|AT5WYjTc6EjjhwrqkvvS6zApJglwj18sF|10|20-780-534-9845|7380.97|ly ironic asymptotes wake slyly even dependencies. furiously regular theodolites i +2267|Supplier#000002267|CKPRzKI,kiEL,C7l|7|17-409-103-9718|3081.30|ously final dependencies along the even accou +2268|Supplier#000002268|1So0dHWj0xfwuNopKvDKFHlCOcL1OvgtkhhUPb|6|16-321-990-6525|8357.03|ress foxes use carefully e +2269|Supplier#000002269|ZD1x6SWGQaaTL oVQCvuCd|20|30-969-611-8031|6688.42|ove the express accounts sleep even, ironic p +2270|Supplier#000002270|HIscbvhw8N94djn,3UbPaY4R|21|31-107-567-8575|1269.67|s after the slyly busy frays dazzle along +2271|Supplier#000002271|Vbz1ex28,Trm|24|34-749-713-1000|8224.89|usly among the slyly ironic packages. unusu +2272|Supplier#000002272|tQdSwOifi4AkfJh|3|13-434-688-3913|7774.36|he ironic, unusual requests. regular accounts dazzle furiously. bold packages boost quickly re +2273|Supplier#000002273|vWz JRynn3djWCIflK5gzLIgjzrBwaBmacSN,L|19|29-241-430-1113|3787.37|ites. decoys cajole slyly express reques +2274|Supplier#000002274|IcvbFFg0JJ9yl6O2fT05VpK|14|24-632-309-9143|6362.84|he quickly special deposits. quickly exp +2275|Supplier#000002275|n95 s1Gp37NuIaefRHIuSolsOtRLPlazF8jLS|5|15-209-306-1486|7233.12|aggle carefully according to the fina +2276|Supplier#000002276|t99Wna9evHMQ3WV99qIGSLlHxmYg73wvrRDIpF|4|14-467-324-5053|4119.81|symptotes. regular requests alongside of the unusual foxes haggle after the slyly pending acco +2277|Supplier#000002277|uxpnnDRwK1m, |7|17-762-810-4943|5696.18|. carefully regular asymptotes nag blithely above +2278|Supplier#000002278|6DzCVkMOXEYdq,vxc5Z4IMl6bUOI0JQolZY9|10|20-171-978-4320|724.01|ld instructions nag quietly express Tiresias. +2279|Supplier#000002279|itcDcATrvf2p|10|20-504-534-1040|396.52|se fluffily carefully quiet packages. blithely special +2280|Supplier#000002280| iaCJZ4e,IydT9Qb|9|19-219-933-8214|7201.72|gular, ironic accounts wake daringly bold excuses. final requests are. +2281|Supplier#000002281|u3YBtrMmGwdEbEp|4|14-474-797-8225|4323.60|ccounts into the pending, pending do +2282|Supplier#000002282|ES21K9dxoW1I1TzWCj7ekdlNwSWnv1Z 6mQ,BKn|3|13-882-982-2553|9689.51|s sleep blithely after the fluffily ironic asymptotes. quickly pending instructions slee +2283|Supplier#000002283|ZlQ9xdapnt h3gFloTbt5DU2rXE8GmvOUIp|16|26-277-101-2459|5662.78|l deposits. carefully ironic asymptotes boost. carefully final packages alongside of the fluffily r +2284|Supplier#000002284|iPrawnuWekWcu shhiSI|4|14-201-516-9247|363.09|ly carefully regular instructions. sly +2285|Supplier#000002285|o2HCnNpytSY7wQIVZ77YrG|4|14-380-268-2685|6355.51|ly regular accounts. slyly pending instructions sleep furiously. even packages hag +2286|Supplier#000002286|C6 Lohr2pUwX3cu9N3wMsTx1q|15|25-378-103-6809|5018.96|tegrate ruthlessly final accounts. carefully pending accounts believe. blithely regular asymptotes +2287|Supplier#000002287|tTsLQnKFL6BbUCmZfgp6CHfVhVj8LmJfy|14|24-393-230-9700|3687.54|as. carefully even packages haggle carefully besid +2288|Supplier#000002288|mR6FnETCHNIWau EF19Kqi kG8Xq5BqwwAp3|23|33-233-878-2521|508.00|he blithely final instructions. slyly dogged account +2289|Supplier#000002289|ZNrbGM9zL7vKnvuRIOj0gY9NhEiL|22|32-130-274-3202|-116.52|ts sleep blithely carefully silent requests. quickly ironic asymptotes sleep slyly across the furio +2290|Supplier#000002290|cLImQv999wfCm11X|18|28-672-592-1824|8573.93|usly ironic theodolites snooze carefully +2291|Supplier#000002291|cREl5ZgjwUa|1|11-877-834-5524|134.30|ular packages. ironic pinto beans cajole furious +2292|Supplier#000002292|4UFDTmVJn0 xnwoWzI1sSdMz5kGFhP9|10|20-419-398-4304|101.83|its are blithely silent accounts. furiously regular deposits wake slyl +2293|Supplier#000002293|1sm7JDyklc59d5Ogy8ZryanOpnod8,XVQ|7|17-317-510-6895|4458.63|ckages snooze carefully. even acc +2294|Supplier#000002294|dcAWrWvBzWRQ8j2lUMRKol3Eq,4xFipvykrHfu|22|32-342-857-4948|2152.17|packages haggle carefully. quickly expres +2295|Supplier#000002295|kxrtG3x8BYVlu6E408azhv7WSOozG5s|15|25-726-503-4865|1404.79|unusual, regular frets caj +2296|Supplier#000002296|WJGru63JUSL8HGmctFjG |8|18-550-820-1088|2204.62|c, ironic requests. carefully regular asymptotes haggle slyly above the pending deposits. qu +2297|Supplier#000002297|Brx9U7alFOFnfqMnQaVxiI1ON|12|22-191-279-4345|3064.30|en instructions nag at the fluffily final instructions. regular exc +2298|Supplier#000002298|tIHQJbPhLnRuom7p,7S6I5kj|0|10-207-519-6202|8675.77| requests haggle furiously unusual, regular pac +2299|Supplier#000002299|8GBH1425wDXP|14|24-705-547-4556|6730.50|refully ironic ideas boost whithout the requests. thin +2300|Supplier#000002300|cKyPkKvkeN|7|17-680-598-1034|4556.77|ly special deposits according to the pending court +2301|Supplier#000002301|vrMm6EOBe6aq,LAi2Ir 79KfCvq0AlSG|20|30-680-972-5401|1649.37| wake. theodolites nod slyly deposits; bli +2302|Supplier#000002302|wMEzrsX2KKpTaJGE3uGEUibymG|6|16-486-165-5642|9967.45|gly carefully bold deposits. accounts nag b +2303|Supplier#000002303|nCoWfpB6YOymbgOht7ltfklpkHl|3|13-358-165-6810|1984.35|silent theodolites boost quickly furiously even pinto beans. blithely ironic orbits promise +2304|Supplier#000002304|l0IUScVLpiM7,VGGGfmfp5kLqQriF5C38f PY|6|16-887-932-4893|7963.93|y after the carefully ironic ideas. slyly +2305|Supplier#000002305|,5L2eVXMzPA8KiwdDNnolpoLf|23|33-791-663-6753|-207.89|unusual, even deposits sleep furiously furiously express foxes. requests boost bli +2306|Supplier#000002306|M2jMA3sV,Ph1rbXecC rOATKGyadR|11|21-386-638-1323|1995.40|accounts. ironic attainments after the deposits lose inside the re +2307|Supplier#000002307|QiuKN71yXgsMd98bVI|7|17-178-196-1825|6731.39|inal deposits. accounts along the slyly bo +2308|Supplier#000002308| Jnv1IK2glrSU0DYu|17|27-384-167-1483|8490.64|iously daringly pending instructions. silent instructions wake slyly after the s +2309|Supplier#000002309|yg0LSO,Pjv1Cs2g|16|26-517-699-1407|8216.74|n packages boost quickly final pinto beans. fluffil +2310|Supplier#000002310|E FGFgvWTLL3beRdu5V3mEVNaYvW61|13|23-736-684-2154|2751.79|lithely unusual dolphins would sleep after the special, silent theodolites. ironi +2311|Supplier#000002311|poad6Gm9xFQ6uvBZjiriixTAWXPGtpdBnw2MxKk|12|22-866-688-9874|3316.28|ng courts against the regular, regular accounts detect quickly carefully fina +2312|Supplier#000002312|07FKNPQjr1kFmSFQZ0wJ|20|30-972-442-8281|8702.46|he quickly busy pinto beans breach above the blithely express requests; express, ironic dependen +2313|Supplier#000002313|LGnQ7JjuugyOaENlaykp9Z2nvyFb|9|19-738-303-4653|4325.58|its. furiously ironic foxes above the i +2314|Supplier#000002314|iU5DZSv3 i04wiV,cTxg0joJLG5AJzVxMdtHW|0|10-848-716-8078|-914.25|ully above the final, even dependencies. furiously express ideas cajole along the requests. +2315|Supplier#000002315|A PIXxBK8AIEDcrDps7j0T9O7qhdBa,hiCSUyJM|18|28-512-328-3909|-916.84|ounts. slyly regular hockey player +2316|Supplier#000002316|M,3PTPas5L3YqD ozi|1|11-715-299-6551|-431.15|y final warthogs are: carefully regular ideas along the special theodoli +2317|Supplier#000002317|qsqEhtdmvi0qz1BJ00UxYJUM,KkpL94|13|23-583-194-5524|596.75|thely unusual warthogs doubt according to the furiously silent +2318|Supplier#000002318|gn2LmQi Xwjp9v129esqSry5E|24|34-734-544-9177|6473.95|ular theodolites. quickly express waters detect +2319|Supplier#000002319|3z3bTulBgv8Re30oDzKgGlZQT|6|16-531-572-9386|4627.81|ptotes. quickly pending foxes cajole carefully slyly silent accounts. quickly ironic instr +2320|Supplier#000002320|ruej4G6jZkXIWaP0c|1|11-550-946-1118|7565.72|ronic requests. special, final packages use fluffily alongside of t +2321|Supplier#000002321|0FWKkhdhrSNpN3ql|19|29-805-994-2628|6029.14|l ideas. furiously ironic accounts sleep. iron +2322|Supplier#000002322|ijvybmLZ9eaW7,u4TXUiW5AxLeEQj2GpnHC2Rj|20|30-373-325-3862|8797.73|slyly about the furiously regular accounts. blithely +2323|Supplier#000002323|WnaL 4ffPzkBUcGyzWE8T0HmWjAn|20|30-134-309-5304|-552.33|ideas wake carefully. quickly pending depths ac +2324|Supplier#000002324|y3OD9UywSTOk|7|17-779-299-1839|9923.77|ackages boost blithely. blithely regular deposits c +2325|Supplier#000002325|oTwOY8yP,yoCHXAcqnDBZdrrNEQJ80Zu|18|28-167-932-2440|4875.37|nt carefully after the quickly unusual pa +2326|Supplier#000002326|wTAOL2UlxXNMMWniK6djM0C mT|5|15-143-424-9736|-718.89| dolphins wake furiously across the fluffily ironic id +2327|Supplier#000002327|CgGSbx3j1aowZNq|17|27-147-109-3635|4509.64| around the theodolites. regular packages nag furiously +2328|Supplier#000002328|DT vaCygfFQ,I0dpbhKS,Y38uVA9angbZsFzF|6|16-820-102-4881|1592.91|l, ironic deposits cajole alongside +2329|Supplier#000002329|24L1GUVKPooZgk4TiDQAtVODBExVoCwznM,K |10|20-963-452-3436|7006.69|xes are across the ironic accounts. slyly regular packages among the pending re +2330|Supplier#000002330|voucODqKZgWtSlcG3fp2Pib|6|16-950-436-2659|4400.84|ns. final foxes haggle about the unusual Tiresias. carefully unusual deposits wake +2331|Supplier#000002331|WRh2w5WFvRg7Z0S1AvSvHCL|3|13-753-341-2617|9984.20| the final theodolites. quickly even de +2332|Supplier#000002332|aGeaonYAjBkJjlz58 JLrR2h|1|11-222-148-3911|8583.56|unts. blithely brave packages wake. deposits believe quickly. blith +2333|Supplier#000002333|mmAvPwgc9irA,MG,GWLlsjoaA1CnP|22|32-100-992-5815|7968.46|. special platelets haggle silently. express t +2334|Supplier#000002334|TRASs4 HRa7BJLxXRms327M|21|31-806-679-3391|9903.47|nts doze blithely ironic deposits. bold deposits boost. care +2335|Supplier#000002335|dtvcfOsHopS5Y0 7QcGRRFcRa q3UuJsMKodwEyE|24|34-732-988-5709|4669.72|ggle slyly furiously final deposi +2336|Supplier#000002336|AkGqxP1DB c3MprXQTvbzLo|22|32-837-824-1722|7933.68| instructions-- carefully pending foxes sleep-- quickly +2337|Supplier#000002337|IFXxxWgM2qN fndBu2WioI|17|27-378-149-6984|8437.09| packages boost never pinto beans. sly +2338|Supplier#000002338|Zr2hwHQYmqjAq95y |22|32-576-711-5780|-727.09|e. regular dugouts cajole blithely blithely brave packages. slyly pending platelets +2339|Supplier#000002339|npAyRCjBaPweChb26dkZuLDVGNEdAOHHk MdUy|8|18-919-726-3263|9649.23| foxes along the blithely express pinto beans integrate b +2340|Supplier#000002340|FGDdku5tR8Rk3gk|8|18-652-194-9734|-261.48|ending instructions about the slyly express theodolites wake furiously regular platelets. +2341|Supplier#000002341|iUiGXav0KdZjPDOrAC1 4sj2A8CIZ,yyWWIGE|9|19-816-319-3654|3285.36|l, regular ideas. furious dinos nag. furiously express ide +2342|Supplier#000002342|xw3qJ1mffXVou66hIqmedyx,|8|18-450-851-7270|6079.38|ccounts. slyly special packages around the quickly ironic packages cajo +2343|Supplier#000002343|Qy5YodXWVh6o|17|27-694-863-1826|6151.84|sits across the slyly regular instructions are beside the even requests. slyly pending +2344|Supplier#000002344|KYmxbGiFlAxzclrA|20|30-701-750-6443|2951.72| slyly pending pinto beans. +2345|Supplier#000002345|WoJC49Pg7s qEgSIeXGPePT|22|32-788-147-6555|8593.04|ests. carefully special theodolites among the slyly special instructions cajole about the unusual +2346|Supplier#000002346|tzwxlyYXGiMe1mk099ty6vn|5|15-308-126-9449|2278.04|e. slyly permanent hockey players along the ironic, regular deposits solve asymptotes: +2347|Supplier#000002347|iZjD85P4li3H4zs6,9 ZJYnV81afGp,Yd|5|15-381-519-4927|4903.11|nal requests according to the carefully express deposits are slyly along the ironic, specia +2348|Supplier#000002348|SSKnKqFQs5KphgoOLU9p|8|18-595-554-2536|3494.00| ironic pinto beans nod. quickly regular dependenci +2349|Supplier#000002349|VIwEmd2PeKy1|22|32-139-397-9180|3648.90|foxes x-ray quickly final dolphins. express dolphi +2350|Supplier#000002350|TWsO2iJGOl7v3vSwiscXp6X|21|31-777-388-3229|77.80|oubt fluffily. slyly pending so +2351|Supplier#000002351|OUheQZEw,TqaIoJJd7XpEpwaUZfI2R4b|12|22-806-713-2802|7581.65|sual dependencies? unusual, regular ideas wake carefully carefully ironic theo +2352|Supplier#000002352|4LfoHUZjgjEbAKw TgdKcgOc4D4uCYw|22|32-551-831-1437|9817.10|wake carefully alongside of the carefully final ex +2353|Supplier#000002353|tJfE2O3Jwt|5|15-175-686-8133|5015.22|to the slyly even platelets. bli +2354|Supplier#000002354|OifJATleNh13fks7KtW5QsVY26AB0hYBjjQW|14|24-240-811-2622|6802.11|e fluffily express packages. express, fina +2355|Supplier#000002355|cwFVZ6IFKvv0CSuW8K tlSenqBu|8|18-914-376-1742|9041.08|eas sleep. permanently final pin +2356|Supplier#000002356|x XVmR1oTh,E8JGOQ8zwgw BGVvgXtuNEgW q4e|19|29-275-759-6372|8471.66|ffy, ironic ideas integrate along the quickly special platelets. blithely +2357|Supplier#000002357|V8x3JyiEOtAo0w|20|30-139-515-7766|8199.77| dependencies. pending reques +2358|Supplier#000002358|4ucUvhCCU MESh|22|32-973-594-8385|3555.22|lent instructions cajole fluffily among the requests. fluffily ironic asymptotes nag fluffily +2359|Supplier#000002359|K99xilTpFY1enNa9TtKKQu,eq|1|11-732-769-1277|9493.58|regular ideas sleep slyly final accounts. slyly regular excuses af +2360|Supplier#000002360|jQ53qr,GLx7apLTqAIuHOBUkQc5Wz|4|14-732-582-8107|1627.18|ular, ironic packages. regular orbits about the carefully regular pains nag carefully +2361|Supplier#000002361|1p U8A z36oRmNfqN|17|27-333-451-9926|-47.26|old sauternes nag after the quickly silent reques +2362|Supplier#000002362|8DQrWe0VvbxI|14|24-932-114-6441|6553.84|pecial dependencies use carefully special requests. courts sleep doggedly according to the blithely +2363|Supplier#000002363|Ei9HRUUYW2uFGY|18|28-513-439-4183|5422.54|. fluffily pending deposits sleep. enticingly special accounts +2364|Supplier#000002364|CbuBnUrU,S1294IMIKvqWhqG6nCm|14|24-226-174-3239|-356.97|ding deposits are according +2365|Supplier#000002365|zXBLdYwy3ofxVs9XREb8Lg|15|25-956-226-6898|4247.65|uriously final packages are slyly fluffily regular theodolites. carefully daring +2366|Supplier#000002366|5GhVZxD5SUeygDMLE7|14|24-438-417-1009|9175.95|o beans. carefully unusual foxe +2367|Supplier#000002367|c42IIv2nUcLJ9hGrhhiAmfOQwvq1zgzxg8tv|0|10-321-635-8217|9194.06| sauternes boost carefully sly +2368|Supplier#000002368|HnDHRsdyt2xNN2qXBHCjxWzOtZgxwf0zG, jq|9|19-878-901-8050|5616.01|uriously final requests sle +2369|Supplier#000002369|oV0yc2bwtTMFLG|22|32-254-496-8786|6305.29|slyly pending courts use blithely. final, regular dolphins after the special packages wake at +2370|Supplier#000002370|xGQB9xSPqRtCuMZaJavOrFuTY7km|20|30-373-388-2352|8031.68|dependencies. carefully express deposits use slyly among the slyly unusual pearls. dogge +2371|Supplier#000002371|b72ZYzlEsIZLmGYJxjC4dYUTIu|23|33-571-287-6263|5491.15|ual accounts. furiously special deposits above the furiously even accounts integrate fl +2372|Supplier#000002372|BwftakIRj5NBO3x0lcAJZbs8E k8yCL7f|8|18-542-265-3537|8815.78|arefully. special courts haggl +2373|Supplier#000002373|RzHSxOTQmElCjxIBiVA52Z JB58rJhPRylR|3|13-845-687-2699|5871.14| special platelets cajole slyly: carefully pending packages +2374|Supplier#000002374|nJ9MEqHHLFiP0Eq2jul8JXj7,WeH7pUk,LN|10|20-878-577-1706|4285.12|g theodolites are fluffily special accounts. ironic, regular saut +2375|Supplier#000002375|pP4gyVgnxG22m 6pAIlNeHQQzujH|18|28-885-378-7547|985.97|ly final requests. quickly express ideas ha +2376|Supplier#000002376|dKY1qJIOWUBZF99EXp|4|14-803-406-2891|3074.05|integrate. thinly express packages above the furiously special pinto beans breach +2377|Supplier#000002377|pqWJYBmfPJCJOGoPKhtoh|16|26-210-849-7512|7455.66|osits. special deposits above the +2378|Supplier#000002378|15WlmRLudnfxG9lXex3xNGmcz5p0p1jHQQd|16|26-195-588-8079|4273.19|ithely slyly ironic accounts. blithely specia +2379|Supplier#000002379|i6Tq1AwS1Ug6copx4X|17|27-473-939-8868|9562.61|ickly unusual foxes haggle alongside of the regular packag +2380|Supplier#000002380|1Q00W4cwp0F|7|17-811-830-4132|6898.84| to affix silently according to the blithely regular requests. blithely unusual +2381|Supplier#000002381|ayAhKvaFnOHQ 1aN3bMzg2bf6g1VqV02sVK0I|15|25-248-483-6418|6795.42|he bold ideas! pending foxes dazzle sentiments. furi +2382|Supplier#000002382|otBPUYXGv LZifYiBq1XPTVKn7d|5|15-391-167-3113|6714.90|s boost after the quickly pendin +2383|Supplier#000002383|wiYnCHr2w4s2MuUHURvZKeCaw|16|26-524-349-1880|6357.95|uffily unusual requests. bold deposits affix slyly special packages. +2384|Supplier#000002384|LvwNYrJKOqS 4VajSC7ot9CQU2izWIa|5|15-204-993-2671|2090.98|the slyly final deposits-- pending deposits above the slyly pending packages +2385|Supplier#000002385|ZoMa,NXhLAlQ|17|27-818-153-6387|7109.01|pinto beans thrash blithely regular platelets. slyly final ac +2386|Supplier#000002386|8H,aRrhqr1N5hg5Yk3TNY0MMt|22|32-503-951-5981|8189.55|pending packages sleep fluffily unusual instructions. ironic +2387|Supplier#000002387|ZfC27DFeh,AEpy5CWOY|8|18-600-116-1124|7828.09|s may affix blithely. slyly express foxes haggle slyly according to the bold, f +2388|Supplier#000002388|niOYugipCSgj5xa2x0SQ8XvD0AjyrbpF,|7|17-309-898-9587|4824.69|ke furiously final deposits. quickly +2389|Supplier#000002389|R3IfcKKa2LBWVj9F4PoePbMim8KjNqyhWsh|2|12-990-545-4605|5486.14|deposits cajole slyly! ironic, silent accounts breach. carefu +2390|Supplier#000002390|HGW9n1scoMDHgF8z7O6iTm,|18|28-111-996-6030|1637.35|y ironic foxes are along the quickly final f +2391|Supplier#000002391|Ou5,Pc9oTyiKhc3Wvj6dejwFN|17|27-830-100-5792|5964.01|pendencies. unusual theodolites sleep fluffily. pending requests nag pendi +2392|Supplier#000002392|xsd6wFsVPZcnWwU1365R|22|32-766-757-1420|3118.95|fully even, even ideas. theodolites are. pending deposits affix furiously i +2393|Supplier#000002393|e2CEFKZR9lLgj5gE7g5wxu5c33HcNqv|18|28-728-930-3521|3122.78|ording to the final requests. reg +2394|Supplier#000002394|xGHwPzfnQR|14|24-503-463-6339|1524.09| bold packages-- theodolites are slyly at the carefully iron +2395|Supplier#000002395|2hSfzXAmVqbw13isaB,nHEWZYo|11|21-976-541-1583|5282.11|gular accounts are. regular Tiresias kindle slyly. carefully regular ideas boost always bold excuses +2396|Supplier#000002396|qm,ACugivE78yOmteX6S8cQM,r837Seoo|22|32-229-206-2429|4377.57|ages. platelets nag furiously ironic instructions. silent ideas sleep ironic, +2397|Supplier#000002397|E0b,zxlk yKgtoKg1jH,|6|16-439-996-6973|3778.48|bold pinto beans haggle slyly. silent, ironic requests +2398|Supplier#000002398|QYinUHrFQkhCbqo9gO|19|29-410-565-8198|-21.75|even pinto beans sleep! furiously express deposits nag alo +2399|Supplier#000002399|SrJEPXIjxUZm21|15|25-717-506-8761|6353.37|arefully ironic accounts run furiously! quickly unusual requests print +2400|Supplier#000002400|rvXmKk9vCN|24|34-521-714-2582|2162.45|dly express excuses. pinto beans wake furiously c +2401|Supplier#000002401|MZoiVRB8uOVheBJmKYKScBpZkLYSOe2wQIoDsIX|13|23-883-970-4369|9311.16|encies along the even deposits haggle regul +2402|Supplier#000002402|7jxiJXJwB1ze4uD4p7DKz44Ifg|18|28-116-664-4294|4724.36| packages. final, regular foxes poach quickly sp +2403|Supplier#000002403| h4Z,AoykWBrD|22|32-688-192-6743|2304.07|ng theodolites poach finally about the final, unusual dolphins. furiously unusual platele +2404|Supplier#000002404|g4LXlcHDN4z6DrdiBb0iCyLA|5|15-366-436-2694|2475.73| blithely regular requests integrate quickly +2405|Supplier#000002405|i0gXQmyUex,XBCf9xn|21|31-791-713-4691|5622.13|riously among the furiously even p +2406|Supplier#000002406|iaNlOIvCCtBoOPeZVWxcc6UZzR,64pKF 96F|24|34-491-823-5980|-807.35|o the slyly bold deposits. even dependencies lose beside the regular, unusual gifts. c +2407|Supplier#000002407|e2sFnvk7D93Bgq5lyOWNHqUq|7|17-717-277-2482|2758.30| even packages. ruthlessly bold requests x-ray carefully along the blithely bold fr +2408|Supplier#000002408|7EkHfS6Pw0hjOjEPB75flYvg9wAwTbi8k|15|25-832-499-5996|2668.82|g deposits wake blithely. evenly ironic platel +2409|Supplier#000002409|oy39SaSQ,FIP pzLqblhxj|21|31-943-688-4514|5902.03|, regular packages cajole blithely furiously regular depths. forges kindle furiously unusual pa +2410|Supplier#000002410|G6aFGbpr3c9|20|30-605-986-8185|6436.60|dolites cajole above the express asymptot +2411|Supplier#000002411|vhHQgR,vY3s,Ntb,yIUYN6XCZFMopFZslUL|16|26-906-744-9070|3955.67|lyly final deposits haggle furiously ironic theodolites. even, final theodolites wake. qui +2412|Supplier#000002412|V1fWqiqXCUUvpEDQ15Cf2yxpv2QHPCDJpD0o8x|6|16-352-776-3522|7942.56|nal foxes detect theodolites. pending, even requests sleep after the express, unusual +2413|Supplier#000002413|wu1NNpaLxJ64RJxh2FSkAyQ,zOA85hQjXirIPS|14|24-407-547-6036|3606.12|se furiously after the slyly final accounts. special ideas affi +2414|Supplier#000002414|W6uBF44GHsJFPwnG,VBH8a e|5|15-728-678-4300|7075.81|ep furiously slyly express theodolites. carefully ironic asymp +2415|Supplier#000002415|njNGJcmSBHy1zgiTEA|14|24-178-426-6863|213.61|accounts. even theodolites must boost furiously fu +2416|Supplier#000002416|,M sqna1,5Hz3|13|23-111-711-7500|2064.31|counts are. furiously final ideas affix regularly a +2417|Supplier#000002417|2zX,k0aiQ he,N qVT9DMhP0|9|19-580-615-8291|9744.22|unts cajole. regular platelets play furiously slyly ironic packages-- furiou +2418|Supplier#000002418|v5lKj6QhaNZ|24|34-948-564-1781|1160.00|slyly alongside of the even, unusual dolphins. regular, regular instructio +2419|Supplier#000002419|qydBQd14I5l5mVXa4fYY|3|13-362-261-8296|2080.92|alongside of the dependencies. theodolites affix deposits. boldly regular packages sleep against t +2420|Supplier#000002420|8N4qV745ulJKR3eiKEt|13|23-347-640-4875|3003.43|s cajole fluffily. fluffily special ideas wake always accounts. r +2421|Supplier#000002421|jIIalqYeEL3ktnsrjTikOYN,EEG,wLWTqOD4|8|18-486-744-9142|8549.71|to beans haggle. silent accounts lose slyly. slyly unusual requests dazzle carefully pending dept +2422|Supplier#000002422|,rCHMLszDW0QgQmfWamYIq|2|12-133-227-8771|1.40|solve. ironic, final packages hinder slyly. fluffily even +2423|Supplier#000002423|lBxqJrIlIS|16|26-565-505-4155|-583.85|. quick pinto beans along the furiously blith +2424|Supplier#000002424|XlAu,cWxwTYXYHBZ,e1GL70ahRFKAp7Xq97Dtt1W|4|14-653-437-1244|8142.92|latelets cajole doggedly? blithely ironic +2425|Supplier#000002425|fYrAOohGgHQ9tjfppZcZ LYl16|4|14-197-662-1723|5121.77| above the carefully final packages sleep against the instructions. regular orbi +2426|Supplier#000002426|85gpHFfMdr8j5,X|19|29-327-898-5197|4555.46| pending, regular packages are about the accounts. fluffily unusual instructions are care +2427|Supplier#000002427|THyH3qFPcBWsGpMID|4|14-421-805-1874|1599.81|uriously even ideas cajole furiously across the carefully re +2428|Supplier#000002428|HFrLq7UzM 7yC8je0I5XxkkIY|13|23-179-824-5941|2032.33| furiously even dependencies ought to cajole stealthily above the fur +2429|Supplier#000002429|mWOt6oozKZL0kPUhIe|1|11-874-703-1500|374.71|about the silently quiet deposits. quickly e +2430|Supplier#000002430|tITNSz6Bak6PYrFrCV5mvd5 GGqlkg5Ib4cwh|13|23-542-548-8369|3683.96|lithely fluffily pending accounts. quickly even instructi +2431|Supplier#000002431|WZx5fCjhvQVdNUB 3vNgNZsUUF|8|18-953-303-2407|6277.92|kages boost ironic, final warhorses. silent packages s +2432|Supplier#000002432|P89UhOIAeQHiriakK4YH8|15|25-789-798-8733|-916.52|as use slyly slyly express orbits. quiet, even braids besides the sly +2433|Supplier#000002433|t4zeqTazryuWWLLKwW93xmdG |22|32-399-660-9878|4958.26|ges maintain among the ironic platelets. busily ironic foxes cajole. qu +2434|Supplier#000002434|XIZDz5FeJuN|15|25-103-521-5496|6411.49|ly regular deposits cajole after the bl +2435|Supplier#000002435|2Wg6s 0Y5Mnfuois3iIoaq1wrVETh|21|31-379-702-6071|-77.87|refully above the blithely pending deposits. ideas are fluffil +2436|Supplier#000002436|1HM1aUbsvoohOyDx5n6biIAoONPjmZ6O9LkDg5|6|16-977-315-5631|9198.95| regular foxes are along the slyly ironic deposits. excuses wake. requests cajole quickly ironic +2437|Supplier#000002437|TKC61Nm9LMg7YDqOFS8U1E7j82gwWkS gimXhZ|15|25-990-673-7561|-193.72|y ironic deposits. quiet, regular accounts are furiously above the sentiments; regular, special +2438|Supplier#000002438|vVqUc3VNaWWvEZfs|16|26-280-148-5426|9141.95|deposits. carefully final theodolites wake furiou +2439|Supplier#000002439|rXxojWV0VefSK7A0dhF|4|14-410-168-5354|9755.43|p quickly packages. carefully pending pinto beans after the +2440|Supplier#000002440|57 Ig2hYAIuZ|23|33-425-942-3361|2626.73|fully bold accounts. quickly ironic requests are carefully pending foxes +2441|Supplier#000002441|bUvEhklrRDp4S3Ipq|15|25-465-742-8457|2373.36|equests. carefully ironic packages sleep furiously about the fluffily even +2442|Supplier#000002442|t73nwMfpO4rjUFkhWnJcS09dBCR|19|29-971-643-5013|966.76|even packages. unusual, regular deposits are about the regular dependencies. slyly sp +2443|Supplier#000002443|Lin y WUwWKx8dZZUiVFeqNVNpP0cHBDYF|16|26-957-263-5938|-751.29| regular deposits. accounts cajole asymptotes. ruthlessly regu +2444|Supplier#000002444|q8x5 AFA, 9zfXRVYkn67kb3o0AagF|3|13-459-140-4138|-949.38|final theodolites cajole fluffily bold reques +2445|Supplier#000002445|gVkPmZC9v7zjro|0|10-325-193-7475|2611.48|to beans. silent patterns must have to cajole ironic d +2446|Supplier#000002446|ZGcshZ3G74DgrYV3JF|10|20-465-546-1417|-877.41|riously final theodolites. quickly ironic deposits are quick +2447|Supplier#000002447|GTnNYGvzuSr0gWTE|0|10-865-807-3083|2258.89|lar, express warthogs according to the never regular deposits detect furiously si +2448|Supplier#000002448|WqxdtiMx3aj,ip YtG9Kko4Rmtk9T|0|10-445-749-9046|2190.74|es. regular ideas about the bold, special waters lose quickly deposits. furiously regular +2449|Supplier#000002449|HYuJQsln523IXNeA6mzAjDo|13|23-638-196-3565|814.73|refully final ideas. even, regular deposits wake. silent, pendin +2450|Supplier#000002450|MR GbefBrwM AmKJtpAkFZ72EvomYHyaQ0|7|17-703-335-3427|7108.84|pecial instructions sleep furiously. pending, pending deposits wake ab +2451|Supplier#000002451|V9fTPy7NjswDmefs7 |16|26-840-405-5119|5355.73| notornis. regular, regular foxes snooze slyly across the slyly final +2452|Supplier#000002452|aSYD1SvrdIGV8LxRL QDp5m9dV|20|30-505-249-4504|9705.00| about the blithely express foxes. bli +2453|Supplier#000002453|hANwgEJk3juMwTdumtBOz|9|19-426-129-7300|1960.39|eas are enticingly final platelets. carefully even foxes alo +2454|Supplier#000002454|cWf49koXMVcuiChRZM|2|12-743-793-2820|1235.83|n deposits? sly accounts cajole furiously. +2455|Supplier#000002455|KzU4h5DjCtdKZiY5Rrp3Jukvcn|16|26-257-769-5335|546.66|old pinto beans nag blithely. fluffy deposits cajole blithely. Tiresia +2456|Supplier#000002456|di5JVO5IIrl6as|15|25-580-375-8992|689.11|ets. ironic theodolites boost furiously unusual exc +2457|Supplier#000002457|iPOyBsGzl,I4 cRkMelt4PRpNzFx9 dkGC1Eu|10|20-752-453-1043|7874.99|ing foxes boost slyly finally express theodolites-- even, regular ideas kindle. e +2458|Supplier#000002458|c8eZIly7LJrl7buLs7OTvNeEStmEFHIJ4hOk|0|10-844-579-4095|1034.04|re against the thinly regula +2459|Supplier#000002459|qj5N6aoCeaNWR64Bcb4AVnnoGLi4ybkcklvTh|10|20-807-908-1295|8126.38|s-- even deposits haggle quickly. furious +2460|Supplier#000002460|wu8B2e YWGoKM fi e23UQn,YD79edUIIXX|3|13-420-979-3295|2135.50| ironic deposits. carefully express accounts boost furiously slyly silent requests. carefull +2461|Supplier#000002461|3AxwmIx6iSzuXKNRqSNcSMDS13zWQU|5|15-442-573-5761|7260.83|eas-- even attainments will cajole at the ironic pinto beans. regular, regular pinto b +2462|Supplier#000002462|HTn6CKSBFVq9P5b0|4|14-751-339-3389|7862.57|olites boost furiously whithout the quickly r +2463|Supplier#000002463|ZIZG tUmzpws6ikxVyqtSAM6|1|11-936-730-2756|1228.97|sits. carefully special dependencies sleep slyly platelets. slyly express instructions against +2464|Supplier#000002464|QEnR4EwAYxnMt5PARcd7|4|14-956-770-1030|6241.59|uses. final instructions are. final dolphins haggle furiously against t +2465|Supplier#000002465|bQqOEbJ0bZF|15|25-984-843-3694|4399.40|ithely. express packages use final +2466|Supplier#000002466|lhyLkN5MSl3SzXCc0a|8|18-121-162-5720|7757.26|g the regular requests nag furiously fluffily express deposi +2467|Supplier#000002467|wVcklXhlbHBSm1kX1wsLngqvc3rVC|16|26-210-933-9923|2257.33|y even packages between the blithely special accounts grow foxes-- bold, pending id +2468|Supplier#000002468|ppFYJjhM83qk|22|32-382-806-3956|8890.91| slyly ironic requests integrate fluffily. blithely even +2469|Supplier#000002469|5D30kvreTqsqgl|13|23-871-633-5677|3653.00|instructions haggle fluffily silent packages. fluffily special accounts wake quickly +2470|Supplier#000002470|o716ZrDQeuT17Ba GW,tcQGh0Avqd y|22|32-424-721-4536|2916.88|l excuses. fluffily pending instructions nag quickly above the quickly fin +2471|Supplier#000002471|xz QZKHFiBBN6bLHTf Pnj|4|14-572-795-7115|5948.61|. ironic deposits are carefully furiously even asymptotes. blithely +2472|Supplier#000002472|WkNfcNCiE5L3bpukI4C Izk5hTqDpcZ|14|24-919-741-1591|8287.19|ans nag furiously blithely silent deposits. blithely final instructions are slyly. furiously entic +2473|Supplier#000002473|VPqdi7aT,1MMyPIhiT|2|12-215-170-2967|9883.14| pinto beans. furiously regular pint +2474|Supplier#000002474|JnSkZjFOopokCB1kCjv33URnieAYfSBITP|0|10-508-846-5031|3942.97| quickly. fluffily bold packages haggle quickly ab +2475|Supplier#000002475|TNYNqO3V1Z32WQBWFjxqMrEPwBtSHx0DWodbL3|7|17-811-939-7444|3405.72|y even requests. accounts need to dazzle blithely. slowly final requests haggle across the quic +2476|Supplier#000002476|XmIgFZUxS9KpXDl|3|13-403-947-5166|4895.38| requests: express, ironic instructions haggle fur +2477|Supplier#000002477|FVLqfD4TzY5s4qoxgB3BUkVOVOR1qmtNptdH|14|24-950-623-5371|5651.16|nal foxes integrate slyly. bold, unusual deposits w +2478|Supplier#000002478|4oxBGBoQ6h|8|18-656-819-2163|3050.82|y express pinto beans detect slyly carefully regular accounts. fluffil +2479|Supplier#000002479|ljPjzEMXbEUjd51mpuFZGthk4Mc5RC|8|18-133-194-6180|2116.69|w dolphins. carefully final instructions use silently bold pinto beans. fur +2480|Supplier#000002480|1t BL2i,JwN7lHjhYFuuBRYy8p3a|15|25-567-152-1810|-785.16|ans believe carefully. slyly express dependencies sleep furiously slyly regular packa +2481|Supplier#000002481|nLKHUOn2Ml9TOA06Znq9GEMcIlMO2|3|13-343-133-8820|1725.76|te final, even ideas. final theodolites print express theodolites. closely bold acco +2482|Supplier#000002482|VgZca,08Okt4SYYxnN80OkUy3Lt0X0TB2ue|18|28-698-684-4533|6742.63|unusual deposits cajole quickly after the fluff +2483|Supplier#000002483|G w,HeMY2qo25|20|30-945-204-5443|-657.11|y deposits haggle above the slyly express packages: slyly +2484|Supplier#000002484|GNI43eYjlbcTIG9A VsSJrppaMrVAj|18|28-151-254-2873|8797.54|nts. quickly ironic deposits boost quickly alongside of the stealthy +2485|Supplier#000002485|k5ho3Qa8Jpuigzwu|5|15-819-144-9617|7322.08|hins. blithely final excuses boost carefully. slyly ironic accounts accordi +2486|Supplier#000002486|5209RQqxCQIW9Utatqvc|10|20-300-215-6442|3203.48|s instructions. ironic instructions against the carefully sly pack +2487|Supplier#000002487|ubg4I99MPSr6ll2Eq5oKAvkRtvCK,zs4EM|19|29-188-309-9763|4800.26|ironic deposits. quickly regular foxes cajole after the slowly special reques +2488|Supplier#000002488|OZAn,O1eScSbR4Uka MCwoff9S|23|33-523-780-1426|2012.52| orbits use. blithely even requests nag ironically enticing pinto beans. furiously ironic requests +2489|Supplier#000002489|kT1LxQaDYbzUM,|7|17-968-207-1176|8389.10|ffily against the carefully silent requests +2490|Supplier#000002490|uYTLvRbjZvdgawxn7 46K68ubbTwyVQV|20|30-672-535-3910|6738.50|kages wake about the fluffily regular excuses. quickly ir +2491|Supplier#000002491|CmQmwzn1lclZ|1|11-501-249-4467|6986.49|cording to the carefully express courts. blithely silent mul +2492|Supplier#000002492|lIrbzGFrpfwuxLv3wdC,3Oq|6|16-435-517-6389|5914.33|es nag slyly across the blithely regular packages. furiously silen +2493|Supplier#000002493|aEUtBq9 cRKRxDt9FGiZHq57sUbwv8 LeY8J9|1|11-286-640-4677|4095.80|carefully regular theodolites are quickl +2494|Supplier#000002494|8 j3qJvL6qjroE avAdKFooKo4IMH2oPHLXH0H9p|3|13-124-905-2996|9020.48| requests. furiously ironic packages boost fu +2495|Supplier#000002495|vrphcM,ZHHiQqjpiSqGVL9Erw|13|23-606-503-8431|3277.78|the requests! carefully even accounts snooze bl +2496|Supplier#000002496|ZuJsU3r4OBQUneIvzg|11|21-485-518-7874|-335.37|fully. accounts are blithely +2497|Supplier#000002497|5aW2,OxhYoH|11|21-246-666-5921|4363.30| final packages integrate furiously. even theodolites solve slyly final excuses; sl +2498|Supplier#000002498|VvpYr7G6OozCvtkIW6qdEZBi3918Vb|5|15-815-470-5030|6008.19|sits. slyly bold accounts are +2499|Supplier#000002499|oSExya3cUOhugbKzsUiQ4iQGQi3qQ9RXhcD4o0|11|21-284-560-2540|8677.75|the fluffily regular depo +2500|Supplier#000002500|3E6egPL3NCIN4Vawqt0pxUoH7ji Zp|18|28-937-418-6930|7025.49|ully unusual pinto beans. accounts across the even theodolites w +2501|Supplier#000002501| QzqpvJ3UdO w13Hcrh1QNsY6Z3 kONPdatHGsPA|7|17-880-517-1690|1404.32|. quickly regular dependencies use slyly quickly regular deposits! furiou +2502|Supplier#000002502|WwZj0jzQ5Znb6NJIxgPAqS,vZu6IX|1|11-806-724-3037|5344.68| are carefully. slyly final instructions detect. quickly unusual ideas cajole acc +2503|Supplier#000002503|DN in fCWeZe Z8ValtBEdwywt9tU48yplcAQc 7|2|12-403-106-5364|110.59|ully carefully pending requests. close, regular instruction +2504|Supplier#000002504|tfZRDUl9jXa j|19|29-420-164-3183|8562.35|sits detect blithely slyly si +2505|Supplier#000002505|QgRYtv8XPssUkrih7Pc6Nau9fgHvh01rrp|15|25-666-134-6990|3448.16|osits mold across the slyly +2506|Supplier#000002506|adoloJGA7m7CVTyzzOCg9rV|11|21-105-487-9563|3280.68|ts affix along the slowly special dolphi +2507|Supplier#000002507|6E0NMc6A64sZUSrEtTp|10|20-205-336-3196|4347.09|nic requests. asymptotes alongside of the idle, regular tithes wake blithely alongside of +2508|Supplier#000002508|rolzg7UZ3KxV6U|12|22-961-254-1433|8576.88|ly slyly quiet escapades. pending, unusual foxes across the +2509|Supplier#000002509|TV cQFJuOUUaJaZGSG|5|15-163-414-7176|5616.71|c packages. special instructions sleep carefully about the furiously final foxes. blithely special +2510|Supplier#000002510|J3xtZTxnkUrG0KjE 9ru76PJGpGINiciqU31|5|15-761-995-7298|7055.81|ly final accounts around the final a +2511|Supplier#000002511|Ys8TacFEOq1E41NDAHESYmaXBuR|17|27-852-513-4637|4998.34|ly regular instructions sleep blithely except the carefully special instru +2512|Supplier#000002512|Tl8qaTO 4k0tMuYRARcsaW8kO5ABOIrrgoatxf2D|6|16-137-555-4045|9077.50|ly unusual escapades according to the furiously regular asymptotes use around the even, unusual r +2513|Supplier#000002513|nsHTqAc8vuw0sx0Z|19|29-819-858-5280|4582.50| fluffily bold courts cajole furiously along the requests? carefully reg +2514|Supplier#000002514|Ur93sG8TyZehI3UjA4AXhmsmGIso|17|27-562-380-6184|5134.16| theodolites. blithely express requests according to the slyly silent warth +2515|Supplier#000002515|pFcun6TefWByFbtruvgys7IUoSUV|1|11-930-354-3487|3826.43| instructions. blithely ironic asymptot +2516|Supplier#000002516|4m wG67rDQpzzCaJcxEpnQhdBONg6E60b GKY|4|14-642-300-7682|2819.79|sly about the quickly regular pinto beans. blithely ironi +2517|Supplier#000002517|uUBLDC5YRPCF9 bdXiB2FTWSmW5ofWUWT,2BT|16|26-197-426-5039|2238.82|ct blithely about the furiously final ideas. unusual pat +2518|Supplier#000002518|aYEA SkVHs7KV8m5ZAT,PihWBgyyVmu3f3CO4lB|10|20-679-460-8455|5758.68|the final requests cajole above the carefully pending accounts. carefully unusual account +2519|Supplier#000002519|SkmiSEA, OwoQRSOF|1|11-817-659-6741|6121.85|dolites cajole carefully against the blithely express accounts! accounts +2520|Supplier#000002520|5y55UzYQKByZP3|21|31-826-707-8432|4965.26| deposits. final accounts use bli +2521|Supplier#000002521|UeoELXSGq2eKHSImwz5ZcF0K 5V3mQUt6sPJ8hB|11|21-779-583-9438|3159.28|ular packages. furiously regular accounts sleep carefully furiously close +2522|Supplier#000002522|n7Zcn1iWbfuwRX49I6e|16|26-228-116-5164|9997.04|ans are blithely alongside of the waters. express pinto beans above the theodolites affix blit +2523|Supplier#000002523|enQWFlNzV6oLMbXOBmSTt|4|14-690-302-5291|4316.22|nal accounts haggle blithely. ironic, regular accounts boost car +2524|Supplier#000002524|CXLC5IFTBdK iGu|24|34-991-307-3040|6710.27|hely fluffily ruthless packages. deposits wake brave ideas. +2525|Supplier#000002525|FP7jl0FjGgC1SRsGw6HaReHx3z7FB4bWvZ3 aFMh|7|17-384-868-8732|5490.95|ly silent accounts dazzle carefully furiously +2526|Supplier#000002526|8JWRGgabYBoXKYaQlmUt28W|22|32-171-767-7340|5465.42|lites wake carefully. theodolites breach. theodolites according to the slyly +2527|Supplier#000002527|HaYSaQW,vH0L|13|23-861-343-7143|5254.03| regular accounts boost pinto bea +2528|Supplier#000002528|sCNrdl2h9YEfToc7JhqpOtwz3|12|22-426-661-4377|5469.56|l instructions wake fluffily quickly special requests +2529|Supplier#000002529|isvgJhjULHVI|21|31-957-321-7048|8826.93|ld instructions are fluffily carefully un +2530|Supplier#000002530|3OKbiyS2QaEk34L,Ru4T8aU3xqWgn5zLuuwFR|2|12-634-424-9935|6345.68|ly requests. slyly bold packages sleep against the furiously ironic asymp +2531|Supplier#000002531|4aZc7CD0nvPmCCjmjkg8K0w3QcwwbFz2 4PSsokq|7|17-315-767-5536|4874.46|s sleep furiously above the furiously regular requests. regular, even epitaphs sleep. +2532|Supplier#000002532|y2ZpNRiaO10wmkyPvFweSC4TzxvX5qm5S|8|18-274-299-4827|6084.57|ending attainments poach blithely about the caref +2533|Supplier#000002533|3ZSYvP04QM|6|16-751-912-9737|5402.57|lar excuses. final packages haggle quickly about the depos +2534|Supplier#000002534|s94jok37BRByP5iZaCDRAcM6zZ|2|12-316-675-2618|4938.35| slyly fluffily unusual packages. furiously enticing orbits across the fu +2535|Supplier#000002535|A C1WmBRZA2BePs3f nB6D|24|34-445-685-5978|1917.68|r after the quickly special accounts. furiously pending instructions promise id +2536|Supplier#000002536|8mmGbyzaU 7ZS2wJumTibypncu9pNkDc4FYA|23|33-556-973-5522|9461.05|. slyly regular deposits wake slyly. furiously regular warthogs are. +2537|Supplier#000002537|IaF gQatYacP2Sgjnj4CsbaUqEmZvoEkDKPV3|11|21-277-778-4471|4961.48|al ideas. accounts dazzle carefully blithely regular de +2538|Supplier#000002538|fz2MZDQNk0sls|21|31-155-875-3030|5628.86|unts sleep slyly! foxes cajole carefully. quickly ironic +2539|Supplier#000002539|9aNuRZI46e6b01tUcwkNoImkBE1|19|29-615-339-9075|2237.94|yly along the slyly silent deposits. never +2540|Supplier#000002540|fpd9A3WsslyUkKypgdpVs69|20|30-301-816-1967|1958.59|ly blithely regular requests. pending, unusual accounts across the regular dependenc +2541|Supplier#000002541|K6QJwctTWE3dTBH3CEMld|6|16-619-383-9029|4061.43|y regular packages. packages above the bol +2542|Supplier#000002542|vPQq7x1BGpdKke797PC2 eYAAMpVMi|22|32-760-854-8942|3498.30| haggle blithely slyly special +2543|Supplier#000002543|WKxIzDMHU82botDW9TRVqqQkxWyDmB3XCM1TP5K|14|24-355-893-5866|9992.70|toward the special, final excuses wake ironic deposits. regular requests play. bold requests wak +2544|Supplier#000002544|zL6nFXaRKp64M|2|12-809-592-9231|8381.66|its wake slyly quickly even requests. carefully regular dugouts among the b +2545|Supplier#000002545|9pb B0jPGB4MDNXSaHKlQqmTung|10|20-490-946-1389|2671.08|sly fluffily regular gifts. blithely pending id +2546|Supplier#000002546|8MmTUlvZyZxYi8j49nGCf4aaHsmt|20|30-726-657-3533|-199.40|ironic packages. express, quiet requests sleep carefully after the carefully +2547|Supplier#000002547|ad64YMOBUZC6 Z9XX|1|11-535-325-9392|1652.66|yly after the furiously pend +2548|Supplier#000002548|UABiGgMCkyTzQnloHsNBCr6da6ITjR|6|16-886-477-8855|-720.02|ironic instructions after the quickly pend +2549|Supplier#000002549|Cw7DOp84cHjMPgkznVhel2XR1kp5g2xkTTd0Xfek|9|19-496-148-4391|7514.39|eas. bold, ironic requests affix special req +2550|Supplier#000002550|2DZFnvi1 nXSreP|21|31-677-917-5353|1331.91|usly. blithely regular packages boost furiously instructions. permanently final ideas cajole +2551|Supplier#000002551|VXntNh4UX,Z8dJ|5|15-457-583-6333|9418.87|excuses cajole requests. ironic, ironic waters use +2552|Supplier#000002552|n8pviFNw0NBjkz z yERRcE|23|33-183-384-8624|4.06|lly pending platelets haggle according to the carefully unusual accounts. unusua +2553|Supplier#000002553|2OABGoDWfzEWBPHE SrGOwRuzARtYVVwAFzeA|16|26-176-559-2109|146.56|e about the quickly pending pinto beans. unusual excuses above the deposits sublate +2554|Supplier#000002554|tFdJFJR46eF3X tG0xnfsTQ|5|15-739-167-4541|8245.77| the ironically unusual patterns. blithely express foxes poach slyly carefully iro +2555|Supplier#000002555|GxjCAMEHTP4H|22|32-447-186-2875|6733.16|s. unusual, regular deposits haggle furiously regular sheaves. slyly quick asymptotes cajole +2556|Supplier#000002556|285GS9P,eiB9kowjpyjhvEtIxtaR4FplLGuUj0y|21|31-134-422-5382|5211.00|unts wake along the unusual deposits: furiously even packa +2557|Supplier#000002557|LBspOqI3Cv616,9Rv75o1c|19|29-960-502-5687|8231.85|ithely unusual waters eat slyly according to the quickly final reques +2558|Supplier#000002558|DHTaILleK5JTKuPqwnMRCQXjuFfxd9cJklY|24|34-688-489-8462|3761.03|cross the carefully even ideas sleep sl +2559|Supplier#000002559|JVO7bgmYiLfwrH2o5HFx6wPuheD|12|22-936-317-1641|3680.85| bold requests haggle carefully after the quickly even pinto beans. +2560|Supplier#000002560|gC4t9RFtBMoItUG5dPD|6|16-519-901-3498|9106.91|mise furiously permanently ironic foxes. quick +2561|Supplier#000002561|D6LnupIqpKTq3QWhs4lES5q1sq|22|32-684-702-3167|3953.13|ngside of the carefully bold excuses. slyly sile +2562|Supplier#000002562|8wONfGJQNBwFNWLjnPQAC5|18|28-247-123-2874|9322.21|cajole quickly. fluffily regular excuses haggle furiously across the ironic accounts. +2563|Supplier#000002563|sT7R4q4Nfxt5wAS7JQcp|8|18-387-293-5801|3406.33|e slyly ironic foxes. fluffily even deposits sleep slyly. fu +2564|Supplier#000002564|lZUJcDjtVMbdaBJ0w82CN|0|10-921-690-4190|4215.34|t furiously according to the blithely final instructions. flu +2565|Supplier#000002565|VtH0Ip3xthMV,iur|16|26-415-528-1475|5220.13|fily regular requests believe slyly even pinto bean +2566|Supplier#000002566|Tkmgiiw2n3 qEinrgYUxJ y5WFumVG5W9|10|20-612-454-8060|7807.14|carefully regular dolphins was never. furiously silent asymptotes affix quickly furiously p +2567|Supplier#000002567|HOXMdN0DnTQV c4kRgO2lYyd OsmnxW|2|12-783-245-5346|4601.75|ckages across the furiously ironic ideas doze above the quickly unusual +2568|Supplier#000002568|SDcPOeCWygcHRk5E7aBCDFpGlgk5xT6|8|18-578-440-4077|9552.82|thely regular foxes wake slyly. multipliers against the sly +2569|Supplier#000002569|1ZoetZI2,IycOy6|8|18-369-276-8483|7374.27|arefully after the pending, ironic pinto beans. instructions cajo +2570|Supplier#000002570|X3PL1AZMSUGm|15|25-442-175-1791|432.20|use furiously. deposits dete +2571|Supplier#000002571|JZUugz04c iJFLrlGsz9O N,W 1rVHNIReyq|3|13-281-349-4224|1862.65|e of the regular packages. requests are after the enticingly unusu +2572|Supplier#000002572|UjDzDmFXJ bXgX UMB|22|32-734-670-3159|8548.43|ake blithely ironic accounts. quickly dogged in +2573|Supplier#000002573|9fNFFKoAJ90HZrDn0iGbGHq5j4P7BOHER|13|23-558-420-3222|460.37|excuses are express foxes. cour +2574|Supplier#000002574|HBrF59QaS97s5YRJvovz|17|27-303-560-2471|2411.88|sly. carefully special platelets lose across the ironic packages. unusual i +2575|Supplier#000002575|LYl93qb9CakGgrI6GtThLi|21|31-192-614-9741|-499.69|lly doggedly final instructions. regular packages wake boldly slyly ironic dependencies. qu +2576|Supplier#000002576|FzLhY1AzB2FGb9rHjNq8vich|13|23-123-416-6792|3551.42|ffily special theodolites. furiously special packages aft +2577|Supplier#000002577|l Iv oeuS3h5fysjC|11|21-771-305-7160|658.06|s above the carefully special pinto beans x-ray carefully spec +2578|Supplier#000002578|2gqjxXV iLm2MtuknrTq|17|27-838-408-3899|2998.93|uriously bold packages. pending excuses nag furiously by the carefully final packages +2579|Supplier#000002579|BH80ivdOGxI9plQ|2|12-310-374-4856|-922.77|as haggle final braids. slyly brave depo +2580|Supplier#000002580|qrs2ub3wSjbzv1kc0a4k|5|15-284-495-3109|9888.26|sias. carefully even orbits wake blithe +2581|Supplier#000002581|JNhWdkCsY913kACx9zOSkDnj2kqnyH|14|24-129-434-1961|2031.91|ng to the slyly regular pinto beans wake slyly final, bold theodolites. even packages sleep +2582|Supplier#000002582|R1auTuXErSql4 ,FqfJoJStindVfyNjgY|24|34-817-929-1695|4265.39| even pinto beans haggle slyly ironic theodolites. bold theodolites cajole bli +2583|Supplier#000002583|Gr6WrUG9mY0Yy9|13|23-890-461-6671|-612.03|dolites wake furiously furiously unusual deposits. ironic packages affix blithel +2584|Supplier#000002584|,tsRk K2JA2xniuLbwtE1XHJ|19|29-531-347-2796|7515.35|uests boost among the quickly pending packages. furiously +2585|Supplier#000002585|CsPoKpw2QuTY4AV1NkWuttneIa4SN|3|13-500-958-8671|6476.64|c requests sleep blithely about t +2586|Supplier#000002586|UiAjevd0CTsQiDahrbFrP2 uxMzrbxEpcN0,ot|19|29-390-542-9908|5855.13|ptotes haggle carefully final instructions. fluffily ironic packages are according to the carefull +2587|Supplier#000002587|hcsr Obo8yP0176DtTqs5CDxn1,HK1Uvmem|3|13-518-417-7741|-734.18|nusual realms according to the careful +2588|Supplier#000002588|QBDXWHsFbL43hT6gqU4ynyr0PrIr,sl2|19|29-974-158-2888|533.65|lessly special deposits cajole. carefully regular deposits would nag furiously +2589|Supplier#000002589|RXJleJvDsIZ2xeVXri8Vy|12|22-956-336-8515|2067.45|ss requests. fluffily unusual deposits serve ironic, final packages. +2590|Supplier#000002590|IO,ItO vfqkXPHqGr93Nip,xo0Q6f6ws6a|2|12-549-169-2005|7537.14|kages: thin platelets sleep about the furiously regular instructions. carefully regular instru +2591|Supplier#000002591|RCENZsjugC0D,5n54w7C0aeQybazo6dl0UopQ|18|28-878-723-2991|9809.13| above the slowly bold pack +2592|Supplier#000002592|2 ZDHmKO7y9Io 57xTjfk6|5|15-482-126-8688|3138.29|ffily final theodolites. pendi +2593|Supplier#000002593|oRkHFWHinuEMfLR1OPZSbW0rjYBK|14|24-370-287-8574|9182.66|ffily. fluffily regular platelets sleep carefully. even asymptotes boost qu +2594|Supplier#000002594|BJdtygY30B7X6fqO4iCQukCWuseKiUkdTKC9aIhe|9|19-468-860-7708|1072.26|sly slow depths cajole fluffily. final acco +2595|Supplier#000002595|k5XWWSZmvUScjcqYMk6k7L|5|15-858-185-9462|1353.00|arefully even deposits are quickly above the blithely regular deposits. regular, +2596|Supplier#000002596|aaRdHdsy9D0ErOwIWDvyBBjRy36Ulz33y4aUma|20|30-945-983-3277|768.67|ecial theodolites. special, regular theodolites eat furiously +2597|Supplier#000002597|8yyFE4QJiycII7AbOPOcH DJYSmTwxG3TK|24|34-630-617-7051|2988.99|haggle along the regular deposits. furiously ironi +2598|Supplier#000002598|tunUZrfZ3bzb0e0vhmhtmp6LR9aC|13|23-511-169-9974|7933.97|nstructions! regular asymptotes wake above the sometimes special accounts. slyly special asympt +2599|Supplier#000002599|JiOiuAAQrryculGlaHHjVQa1Yfci|14|24-146-433-8368|2244.19|jole furiously furious foxes. ironic, final deposits above the blithely even ideas +2600|Supplier#000002600|x6q9Kzx AfI7yqjYujRs4t RTvO9v2pgG73morL|8|18-611-478-3201|8150.06|furiously ironic, fluffy requests. carefully special excu +2601|Supplier#000002601|O,xUQnlTenpSKG4f9SBXcWpr1Yb|0|10-898-717-9719|4039.03|latelets haggle. blithely exp +2602|Supplier#000002602|So6U9dxBXM60pCxlEBmKm HZo|24|34-266-788-7504|905.25|atelets haggle fluffily against the fur +2603|Supplier#000002603|gDexQYDE8EZTZ|17|27-102-973-4543|-487.60|furiously. pending platelets sleep. deposits among the carefully final r +2604|Supplier#000002604|7Uvm3Rwch9Dk3I2Dmr35|15|25-539-302-5578|8752.88| pending ideas integrate carefully fin +2605|Supplier#000002605|xbIrP5HDQnF2eg68Ipnd0DnvXwd7QhNK|15|25-328-917-4525|4801.92|egular accounts sleep ironic packages. unusual foxes use bold, regul +2606|Supplier#000002606|ggBYvJkN8E9du|19|29-829-782-8023|7880.38|carefully ironic packages according to the fluffily +2607|Supplier#000002607|SmzOOdE5aCFu4galUc82SVwNSGL0zNCgIs8O,yMF|13|23-660-465-4876|3672.84|among the even instructions. special accounts boost carefully above the ironic accounts. +2608|Supplier#000002608|k5SMIxuhT0k|24|34-464-161-5429|6479.19|ely against the fluffily ironic escapades. carefully +2609|Supplier#000002609|BYITGBp7JW5Zfpb8CQLj8|7|17-307-925-9609|4782.14| regular, ironic requests wake +2610|Supplier#000002610|L5I4XUPZsSdjORrN|10|20-468-124-1460|5379.22| packages about the slyly special requests are along the regular, regular pinto beans. ironic +2611|Supplier#000002611|Hu,yHVcjcpK3x6IA1pja5FEgFtqDGVuCT1o,x|10|20-556-126-9439|8521.86|ains. deposits haggle slyly. furio +2612|Supplier#000002612|QGS9JH6mZxefQuWZ|1|11-841-636-6934|4890.38|ular asymptotes haggle slyly final requests. furiously ironic theodolites wake slyly sly +2613|Supplier#000002613|pwXlcYnq4nd07F|15|25-972-254-3158|9680.20|onic instructions haggle furiously above the quiet packages. ironic foxes dete +2614|Supplier#000002614|If3TvxLfsTCqdJ1|18|28-238-580-1808|3210.45| special, pending platelets detect quickly +2615|Supplier#000002615|MiGmjZy93D|20|30-285-851-2800|1047.61|. pending packages haggle furiously atop the regular +2616|Supplier#000002616|f3Q7DaCzXvmMZGnqNcMuo26laeWl|10|20-145-750-2687|5166.65|nstructions. daringly final requests boost according to the +2617|Supplier#000002617|rgdjJRVgZQ2zadNW5QcvG5AayIuMh8|11|21-947-364-5238|-164.74| blithely final foxes cajole. express, ironic excuse +2618|Supplier#000002618|3UtbE1kKm29kKyx09hSEBMhRLM|21|31-842-369-2440|1177.11|efully express excuses along the accounts play excuses. blithely unusual foxes nag blithely fina +2619|Supplier#000002619|5VZUuaO4Sccukdj820WeUqF5Do|4|14-369-772-3067|3600.03|ronic requests sleep along the deposits. pending requests are against the boldl +2620|Supplier#000002620|CLsgGeCwLHLyo4SXod9wVKcr|4|14-270-393-5410|3717.06|s sleep quickly blithely regular asymptotes. slyly ironic requests cajole fluffi +2621|Supplier#000002621|wemDWzHU w nuGVJ|7|17-201-263-3636|1549.21|y regular deposits boost carefully slyly final packages. even acc +2622|Supplier#000002622|miUraxxZwzGBi3E8SDavxizLy8si|12|22-611-122-4693|9470.09|gle furiously never final packages? furiously regular accounts among the carefully even deposits wa +2623|Supplier#000002623| 4Fu1G9iVp4ID|23|33-479-498-3860|2887.89|ly. blithely final requests integra +2624|Supplier#000002624|Ehmo9DNJ1prmKBlvQCs|8|18-632-900-4407|2693.76|nos are slowly silent accounts. furiously unusual dolphins nag blithely express foxes. carefully fin +2625|Supplier#000002625|a4V0rWemgbsT ZMj w7DB8rUbZ4F4lqqW5VKljQF|11|21-136-564-3910|6913.81|. asymptotes among the express requests cajole furiously after the ca +2626|Supplier#000002626|GY8qLW3UCwiSMUb6IFlNlzA1oOdvH8knsUq5|11|21-367-503-5748|5440.21|ely ironic asymptotes haggle fluffily along the express platelets. doggedly pending pinto beans +2627|Supplier#000002627|VtzcXgv0R46BaV21MlPTTJV,TM|11|21-375-787-2926|-459.38|have to snooze slyly after the +2628|Supplier#000002628|0qI4Lll6L V5af|9|19-518-345-8832|-939.51|lowly alongside of the final, special waters. furiously pending foxes after the final instructio +2629|Supplier#000002629|0Bw,q5Zp8su9XrzoCngZ3cAEXZwZ|3|13-192-161-4976|3633.93|s. even, express excuses after the deposits are carefu +2630|Supplier#000002630|ZIQAvjNUY9KH5ive zm7k VlPiDl7CCo21|3|13-489-929-1350|1371.55|oss the deposits; carefully silent ideas above the slyly regular theodolites eat about the final +2631|Supplier#000002631|jbO, cOqoJTHQAetYnUNOKa3U|18|28-153-887-4167|4658.15|nic requests. furiously final foxes cajole c +2632|Supplier#000002632|xb34n8O3vDE9|23|33-982-421-4429|590.37|lets. furiously bold instructions sublate inside the deposi +2633|Supplier#000002633|RJbm8rMRyUesH|1|11-157-698-5285|9234.14|sts integrate stealthily +2634|Supplier#000002634|6ktaVHC7og7N5UI8CEtGsTuqStKRzb6Y|12|22-961-867-1818|8185.69|usly special pinto beans snooze about the ironi +2635|Supplier#000002635|QWtNr1h9jyotgBhsFgHlZ4QbqV|5|15-141-516-7435|8867.03|cording to the quickly unusual accounts. ironic theodolites use blithely. regular deposits +2636|Supplier#000002636|aGyJICENjppMbqsejm9wc|14|24-527-297-5348|9220.97|gle. even deposits are quic +2637|Supplier#000002637|cpaViP1r,C9aRi0vWscl5N71UrIIylnRMhfMFRU5|17|27-152-237-5856|6986.83|uriously unusual requests. requests despite the slyly regular ideas are across t +2638|Supplier#000002638|AVxWH8wr755sf7ys jCr7C24KyXG9earxO|11|21-265-852-9504|661.70|odolites nag blithely. req +2639|Supplier#000002639|DDS86GEba3puFUBs0g1Mlqyi,m,KatYyy0Nb|8|18-981-674-2068|9185.89|kages haggle fluffily at the ironic requests. carefully pending asymptotes +2640|Supplier#000002640|8y3orskO7aymVQ6t|12|22-416-966-4904|344.90|uickly ironic foxes according to the quickly thin dependencies prom +2641|Supplier#000002641|nPjjEtlqUdUbRNvnwEOx5VKo2ETVcMDTKWSQ,Z|24|34-634-432-7359|8588.66|long the quickly ironic instructions. express accounts +2642|Supplier#000002642|hI,Q Fj7Iv3QSp04x7ka0LS1EAtrnFSKMHJ2MeXL|2|12-502-238-3518|5613.09|osits are. fluffily express requests believ +2643|Supplier#000002643|eDN6YjGtp2dcj0IF,BKEEYjElO,sUjjcNI|21|31-979-945-6354|1231.38|slyly bold accounts. regular accounts boost above the carefully express dependencies +2644|Supplier#000002644|aOrGHuvmh2mYtEquclV,7qYjKK|18|28-289-594-6509|2692.56|onic dolphins haggle: blithely bold packages sle +2645|Supplier#000002645|DhTKpVgrzHa4NcbuJtK9ZokCfxuCrugCY|12|22-288-789-8442|8771.23| blithely permanently pending asymptotes. furiously regular requests +2646|Supplier#000002646| ZGaGVMSy31SeBVfpf8ey8D2mm,kUbdX2SWsM|19|29-314-944-1425|5134.08|d deposits. furiously unusual accounts are along the silent cour +2647|Supplier#000002647|32IqHFUOCoPODJZNmXu9DP0XstHVWjZl|20|30-418-603-4562|2479.38|silently above the carefully even attainments. unusual foxes nag sly +2648|Supplier#000002648|JW2itPAf0JuigZ125wC3mk|4|14-997-274-9526|930.31|sely even packages. quick +2649|Supplier#000002649|agDQi9iCt1cUaS|21|31-891-427-6188|2717.58|l requests haggle fluffily. blithely bold dependenc +2650|Supplier#000002650|51iJSMOmYqTXNLB|18|28-193-450-1096|273.93|ngly regular requests haggle carefully among the ironic accounts. pack +2651|Supplier#000002651|1RyML5ukQyVOziwj3jVEZ14xcSjtEZs|18|28-377-752-2756|3823.25| above the slyly special forges. pending requests over the packages nag slyly slyly special f +2652|Supplier#000002652|fAzRSsQ6VnxRSosL7aO7YqSBbHt5 |4|14-882-858-2723|4366.99| requests. carefully even ideas a +2653|Supplier#000002653|QhF5puAJxt yLg4px2|23|33-387-651-1873|1981.07|c requests detect accounts. carefully even requests sleep about +2654|Supplier#000002654|XVHK7aKCY,JCeefZXuxxYWp7wMeeHT j5dAZrjHe|17|27-644-402-4455|7838.57|ending packages. furiously ironic instructions engage fluffily. carefully regular packages w +2655|Supplier#000002655|i6v8dkQBuK0NSCeqQCE8|21|31-276-987-1638|4856.74|ccounts. ironic instructions use carefully. even packages across t +2656|Supplier#000002656|bBENRxX 9Kfod5X73nXWEnK8D2vDffdyGgiNy|11|21-659-641-9807|5173.10|use across the pinto beans. blithely ironic requests are within the blithely express waters. +2657|Supplier#000002657|yPbr8IqeYqsj5l4mwKzX|24|34-270-666-7655|9263.88|ts. dolphins above the regular pack +2658|Supplier#000002658|zr3IT4NPv,7P4uZQje2cHoSWrQ0Od6LC|2|12-362-581-3985|2077.71|unts nag blithely across the final dependencies +2659|Supplier#000002659|nHaiofS3Ra6nsLVCqNC7O0BD|17|27-639-321-9156|3991.55|o beans use slyly. carefully final ideas are on the b +2660|Supplier#000002660|yqHOcpFCkr00HW3u1LvK88fXVdvsDwh|11|21-841-314-3330|6021.61|he carefully final excuses. express requests use above the instructions. quickly pending orbits ar +2661|Supplier#000002661|vSX6VPxeqSM9ft8VLbRQxVn|4|14-798-808-1495|3170.70|d the final, ironic asymptotes. quickly fluffy instructions use. special, regular requests print +2662|Supplier#000002662|bYPKwBjGWKUWzEwrQ3s2a70Om Zly|2|12-230-709-7086|3937.51|sleep along the final waters. hockey players use. slyly ironic platelets mainta +2663|Supplier#000002663|eVFAioltCy3il8D tOc Cslu|17|27-626-994-4952|-344.07|ress excuses! ironic packages are blithely fluffily final packages. caref +2664|Supplier#000002664|Yubrd7VE sB|18|28-949-545-1481|1095.86|ites. final accounts nag. blithely ironic requests boost. regularly regular sauternes nag acr +2665|Supplier#000002665|1syLhsDCzxCiGXgfvycJDSSPu1yIW|17|27-345-362-4570|2051.01|l packages alongside of the regular accounts are slyly +2666|Supplier#000002666|phm2stNKBHUQEfOhJ|18|28-724-415-5092|6252.96|egular packages boost carefully carefully even theodolites. unusual, regular deposits nod +2667|Supplier#000002667|4pEWtzzONq0LV0kMQh4YzdXDJrW|18|28-775-155-4297|8484.21|ously even theodolites nag carefully iron +2668|Supplier#000002668|vcRr6OF6QVbEeFfy sGqagXr7|21|31-869-135-5603|-797.87| carefully unusual dolphins sleep quickly! furiously re +2669|Supplier#000002669|GG1GW2tM6wvOW iIPblxxmXj2g|3|13-960-967-7081|8068.64| instead of the requests. ideas use ironic deposits. blithely regular package +2670|Supplier#000002670|o8LD7HvzSuRnlafJ1CNyPnMBo 4xKIv Pl|18|28-245-214-7690|-309.30|lyly final pinto beans. ironic requests integrate slyly. furiously even sheaves b +2671|Supplier#000002671|QNc4DU7QWBkmFW|19|29-422-456-6151|2531.24|pinto beans nag blithely. quickly even packages unwind abo +2672|Supplier#000002672|ltw3PjtQ05 KumuVhrzxUnVgueMkhG1E8Ai8A70|7|17-545-775-1990|3558.18|le carefully. carefully pending reque +2673|Supplier#000002673|A5 AJQjPCaOl1quSW,l35HZ86mu|16|26-227-566-5779|4340.33|unusual pinto beans wake afte +2674|Supplier#000002674|MsTnXr0aEziTO2dsm|2|12-183-939-6643|4928.27|uriously pending foxes nag carefully carefully final somas; furious +2675|Supplier#000002675|HJFStOu9R5NGPOegKhgbzBdyvrG2yh8w|19|29-474-643-1443|8431.40|ithely express pinto beans. blithely even foxes haggle. furiously regular theodol +2676|Supplier#000002676|Xl4TnYEpX4JlkQh11gL8hXTYRQ1|6|16-262-321-9209|2560.12|ickly excuses. final packages detect blithely regular ideas. never even acco +2677|Supplier#000002677|QYcr2ZFZEQ F59zKoGJCSHG3j10e9HAqzW|21|31-596-125-8311|4980.70|s nag slyly ironic pinto beans. blithely silent ideas besides the instructions serve quickly bl +2678|Supplier#000002678|nnZNHDMCnZKTm|11|21-411-717-7148|9560.60|y according to the ironic, regular excuses. blithely slow foxes about the iron +2679|Supplier#000002679|MaiB0O77I5cCcE9w|4|14-228-859-6923|6030.75|s are slyly. furiously regular mu +2680|Supplier#000002680|M70pZheQJntXMqbwPRMAq6|13|23-161-390-4381|2830.27|lphins. slyly express accounts affix around the carefully express +2681|Supplier#000002681|3LkJhU4kbsgbf|18|28-791-158-2651|2308.66|gular pinto beans haggle according to +2682|Supplier#000002682|bmgSMI5JAAg|12|22-309-556-8720|2641.61|eodolites play thin deposits. furiously regular grouches cajole carefully according to th +2683|Supplier#000002683|WMUccHizpga,Z2BSAV|7|17-754-432-1167|-433.51| foxes are about the fluffily final deposits. perm +2684|Supplier#000002684|KPsOyszpmQeTDgdsX1vcoO SDbPJxqQC5dgg|20|30-893-703-5309|4900.44| slyly along the bold, regular +2685|Supplier#000002685|OEDQNNMQ6zU2xkppSXM|4|14-996-206-9950|2764.88|in theodolites shall are carefully furiously regular ideas. carefully regular i +2686|Supplier#000002686|HjZjaomwf3GTfTsasFjSZL5J|1|11-817-316-3479|3249.79|mold. blithely unusual instructions impress special deposits; realms +2687|Supplier#000002687|bfA0yTgLN6MHtwVjEknzyZA|18|28-619-694-1343|2141.29|ies nag after the furiously regular packages. deposits boost according to the ironic, ironic excus +2688|Supplier#000002688| 4zQf8C1dSQj9yYgZQVhVUFOOm|15|25-926-753-9861|5603.38|o haggle blithely along the silent, even foxes. special, pending asymptotes along the carefully flu +2689|Supplier#000002689|91eykk9cfBmJwZuncC3N7csEifAJE0|17|27-115-222-6568|3091.65|ly final accounts wake carefully. carefully pending requests sleep. daringly special p +2690|Supplier#000002690|qN7Mezx2P5FOO2A0w,N4fzD1G7d|11|21-547-904-8382|6414.78|ix around the blithely quiet theodolites. quickly pending theodolites cajole furi +2691|Supplier#000002691|WkXzAronS1Gn|5|15-641-370-4866|258.33|s. deposits solve quickly across the unusual acco +2692|Supplier#000002692|1B3q56lLAYJlOR5LGa V|6|16-399-574-8135|5630.80|counts integrate slyly about the furiously unusual packages. final idea +2693|Supplier#000002693|,zt0AJVhjsw|10|20-447-921-8308|1880.36|the carefully unusual packages sleep furiously pend +2694|Supplier#000002694|MA,zH,ZlgwhheYzQdnJyf|15|25-668-762-7939|9185.36|leep. fluffily express instructions against the special instructions use ideas. unusual accounts ac +2695|Supplier#000002695|ildy7QNbrDD8eRVPQVUPD|8|18-490-712-8829|2384.98|around the slyly regular dolphins. blithely +2696|Supplier#000002696|jcKZe1I4rY2p7Lg,SV b|21|31-168-339-2310|9393.43|nic courts. regular requests wak +2697|Supplier#000002697|qiE2VlquDc7kgapBF|18|28-183-549-4660|1438.45|permanent asymptotes haggle around the accounts. furiously t +2698|Supplier#000002698|CMlIXJJkMBZVQ|9|19-477-654-8549|7048.94|packages affix carefully. carefully bold dolphins detect furiously across the carefully +2699|Supplier#000002699|,AhvhZzc5HW5UA8in0PmuzwbO,02i|10|20-815-307-6571|-420.45| instructions try to nag blithely against the blithely bold dep +2700|Supplier#000002700|MNao9wJlVEU zvnluESvUqqHei68eMm,BL|15|25-474-598-5263|4412.20|rmanently ironic requests. fluffily even dolphins sleep blithely reg +2701|Supplier#000002701|ls4s5FmEieD9WE2eBoHf|4|14-423-824-9030|2763.83|ts. even theodolites boost fu +2702|Supplier#000002702|9bFADaQgBUF4QdKbfpHo|22|32-484-731-3344|7712.38|nts nag blithely around the furiously +2703|Supplier#000002703|IiqJmsfyVQo|6|16-340-123-9930|6950.24|s. packages maintain furiously final, regular requests. carefully special excuses +2704|Supplier#000002704|WhpxJOQJQ3FDKvjjismQsy6uoGi|2|12-227-559-6658|1641.36|heodolites use furiously. final, final requests are quickly. furiously regular theodolites unwind at +2705|Supplier#000002705|Rh6DZOQQ8kO6,NtCMeqTUA3|9|19-514-252-8017|7289.54| ideas are quickly courts. slyly even +2706|Supplier#000002706|Yio1FNwmjgD8UX9ce40marDkGoML3S,,OX|10|20-887-223-4341|6261.52|kages are bravely. instructions haggle express, final theodolites. furiously special requests snooze +2707|Supplier#000002707|xMXO5KpIsdXt,mJ5|20|30-127-723-4248|9324.00|tect. carefully regular deposit +2708|Supplier#000002708|RN5tqUQEsdJw3LvTd4F4|12|22-378-957-9914|584.35|r deposits cajole slyly inside the express, ironic instructi +2709|Supplier#000002709|TrDJk3OHXxcrgob4Yf1fF|2|12-872-274-5011|7034.15|oost blithely after the slyly special requests. furiously i +2710|Supplier#000002710|n0B8gQQBP0oHY3Sj0|0|10-749-448-5794|4006.20|ully regular requests. sl +2711|Supplier#000002711|Gs8YnLauQRaIm5kI,DdaNgI8|10|20-404-994-1317|-803.17|ent requests along the carefully sly instr +2712|Supplier#000002712|wfZXyCtlYU5tg|15|25-883-422-7510|7103.17|never regular ideas integrate alongside of +2713|Supplier#000002713|yAt5WmRDsnVPjGx9IEzPIWatZhz4yuqY5p|7|17-661-207-7790|9037.75|ular requests are quickly about the requests. pending foxes nod. express packages alongsid +2714|Supplier#000002714|YaFQe1MulI1kn2 gp8TD16s Ht5IdlBCKV1|12|22-662-530-2565|7591.72|l packages. blithely final frets maintain care +2715|Supplier#000002715|Z5oAPo57q31ygVY2IibeIZGsSA6txe3UDxOXAq|18|28-782-769-2846|3048.56| to the gifts affix fluffily alongside of the silent, ironic foxes. final dinos w +2716|Supplier#000002716|oyXKHQx 2zUEJ6Tf T5KtXN9Nh7xxFklVZ7Z0|12|22-917-110-5332|8045.77|l deposits eat furiously dependencies. packages snooze slyly pending packages. quickly regular pin +2717|Supplier#000002717|Dv1LGyklog2b17|20|30-642-603-1134|273.61|pinto beans. quickly unusual deposits cajole furiously slyly regular reque +2718|Supplier#000002718|qAIEvc9S13WI9TtYqU,48gj0|13|23-700-162-8801|9461.04| even deposits use along the deposits. furi +2719|Supplier#000002719|4nnzQI2CbqREQUuIsXTBVUkaP4mNS3|3|13-241-286-9786|4777.59|eas boost carefully ideas. furiously even fo +2720|Supplier#000002720|XxB7GKDpnp5Xkdg0zGQrPaOZIKTrK6T5c4|17|27-233-812-2662|6016.99|kages! slyly regular theodolites about the ironic foxes run across the unusual, express d +2721|Supplier#000002721|HVdFAN2JHMQSpKm|3|13-957-330-7254|2535.29|ronic foxes. requests integrate. even, e +2722|Supplier#000002722|YlPsKWwJQTZj5aAq4XTVnYDBZaw9|19|29-868-338-3436|5515.09| silent accounts above the slyly express deposits doze blithely blithely ironic foxes. slyly specia +2723|Supplier#000002723|,M8ShKTXyBramz 90ahZXSFpbF16a2JYqPDv|23|33-174-542-2072|4162.42|ress foxes detect along the express packages. accounts sleep blithely alongside of +2724|Supplier#000002724|cMpJhoC2MLoLMAu6FlIisIti0nh5|14|24-215-299-8776|6833.26|cuses are above the express, regular deposits. +2725|Supplier#000002725|LoFZWIp7I4NZHNzj,n2o|7|17-827-742-4461|4632.40| affix. express foxes nag carefully along the unusual deposits. slyly regular pinto beans +2726|Supplier#000002726|NGStDcC9aXsThy|8|18-120-566-6473|7776.28|ptotes. quickly quiet courts nag after the blit +2727|Supplier#000002727|kLi RyYaUPXk IhCUPulfm1mivvPWkfFPlEFq|8|18-299-626-4348|2922.73|s are furiously quickly special requests +2728|Supplier#000002728|mI3cc72SxNEwaWpAtZ407p5KqxI1|24|34-856-434-7213|5256.31|es. slyly special pinto beans are slyly ironic +2729|Supplier#000002729|fC4Yumgq6TdMUorP1CFDQecKdTl6QzmGp|19|29-888-561-4661|-112.15|nusual packages. thinly ironic accounts after the fluffily final requests are furiously unusual acco +2730|Supplier#000002730|lIFxR4fzm31C6,muzJwl84z|3|13-102-220-1137|5072.89|posits. pending instructions s +2731|Supplier#000002731| nluXJCuY1tu|19|29-805-463-2030|8152.61| special requests. even, regular warhorses affix among the final gr +2732|Supplier#000002732|IAizS6WFBkDb3iSwzx1ASIJiO9EaF|5|15-270-948-2546|3025.60|blithely ironic patterns wake. slowly express accounts haggle slyly special acc +2733|Supplier#000002733|WkFhvxbp2n5OtyQide,I1d1HQt|1|11-998-190-8682|7284.29|gle blithely express deposits. final foxes thrash. pinto beans may are u +2734|Supplier#000002734|srX7fvJ5eKuHM1tyN|1|11-251-919-8885|6168.31|se, final theodolites boost furiously pending packages. slyly regular r +2735|Supplier#000002735|mnAZDTvi6bBZJzwFHR58Um7vUo5c5qRjm|16|26-217-989-9553|4820.63|eas! pending, final requests after the platelets haggle above the +2736|Supplier#000002736|bSUiiXb6pyQXCCHf|18|28-961-417-2503|3369.19|st the silent foxes. slyly bold packages wake furio +2737|Supplier#000002737|jCTp6rTlySOAhwrVApGT2gKddKhnOiCIJAJ3Ua|7|17-317-725-6046|6999.03|thes. final requests sleep above the regular dependencies. unu +2738|Supplier#000002738|uc,zYilzDQIkEmH8dWYJE9M681TmlEPsdFCtB6NY|11|21-215-533-3433|7685.78|y even dinos sleep carefully final, final deposits. regular accounts wake sl +2739|Supplier#000002739|kyj8yCMk2OTqItCvNE2GV,7H5rtrD|22|32-226-606-2427|-18.60|ites wake blithely. carefully regular theodolites cajole alongside of the quickly regu +2740|Supplier#000002740|DkfOP6VJXWnQp2a0U7eYkE9V7PcXVSPn6n|4|14-728-555-1976|4349.38|structions sleep furiously. quickly final h +2741|Supplier#000002741|4uly 6P8PPJXCRCcvSPUFWunFVy1XX4C|12|22-600-779-8361|3085.65|y. carefully special accounts be +2742|Supplier#000002742|MOOEut4u QZ2xkXfwiNKTxep,USxRUQ9|18|28-884-467-8595|4571.04|efully final deposits hinder slyly blithely regula +2743|Supplier#000002743|Ogj4Ndxtu7FARpgYmpeQMjoAnYlIi|10|20-176-647-8475|-584.12|y after the regular, final requests. packa +2744|Supplier#000002744|elDzXb5qPLhE1Ii8|2|12-942-618-4195|3603.65|eans. blithely special accounts cajole blithely daring requests. unusual accounts are. fin +2745|Supplier#000002745|AiY,iWimTpLQ1CsF79|1|11-409-317-7941|9599.25| the carefully even warhorses haggle furiously according to the +2746|Supplier#000002746|XUG IMlW5YBIeyv jB5mFUneoHKy|9|19-294-716-7685|6735.40|gainst the silent foxes. blithely special packages are against the pendi +2747|Supplier#000002747|Tg3 99K0lqdnqP3tr|8|18-107-661-2022|9899.24| requests dazzle slyly above the deposits. ironically even accounts hagg +2748|Supplier#000002748|67Y1bH8enJlO81k QtJRWIst|14|24-325-183-6599|-367.06|ar pinto beans. final accounts +2749|Supplier#000002749|Gt43BFe99L4nCmMpYoVEW2xL 5s5R0bwceWV|21|31-627-908-2351|-296.48| sleep furiously above the unusual instructions. regular deposits are carefully blithely pending pla +2750|Supplier#000002750|m0rNQaUPhc,6cptq3YV1cu5|16|26-780-842-9248|8947.68|ess foxes integrate slyly special dep +2751|Supplier#000002751|eeqAMZwb9nFyWJG1q9mMzue0KumSp |13|23-612-617-7847|778.74|lithely silent theodolites haggle af +2752|Supplier#000002752|gr0ct2XL3XRAyD|10|20-886-567-5538|7205.98|o beans. boldly even frays use slyly. regular ideas ha +2753|Supplier#000002753|Q6D89sklsThNCyo7VY|13|23-732-338-4999|9882.68|fily. pinto beans affix furiously across the blithely regular theodol +2754|Supplier#000002754|zw8Vl,Lpglqlbu6xAuNu,T|15|25-682-858-7129|7357.05|symptotes. regular deposits nag. slyly regular deposits above the +2755|Supplier#000002755|96QubsRr9dKaOLDZc7dC|23|33-325-718-6444|7984.57|iously quickly express theodolites. final requests +2756|Supplier#000002756|VBK81o8gevpj6JnjP|17|27-449-182-9217|5791.10|ggle slyly about the carefully even depos +2757|Supplier#000002757|CaERkgKvPpr4LVD7voAskre1t8mzOHd 47mHT|9|19-959-961-5223|1085.89|ggle blithely. regular accounts +2758|Supplier#000002758|2MJl7GcaHrJA4FEUxarCld9Feht7otlkleD2Bc|14|24-105-707-1145|-445.99|nt theodolites. express, regular deposits wake slyly. un +2759|Supplier#000002759|UMMOYgG lDV2niUiXUx4Ft1SzT6ix6Vy7qej4sO|19|29-818-946-1920|4232.40|gle quickly furiously express requests. slyly regular ideas haggle. requests sleep. slyly p +2760|Supplier#000002760|JwaOuD8jdeWmVr6SEnVbF,KpdO6|5|15-554-965-5719|5997.41|aggle carefully final deposits. furiously ironic pinto be +2761|Supplier#000002761|HJP4T kgOVZG53FL|18|28-534-311-2595|3448.45| gifts detect after the blithely final dolphins: unusual theodolites promise across the regular, ev +2762|Supplier#000002762|hFPPebJ3MhSGg8Fm8Glvq6rM|1|11-404-694-4902|8039.56|y ironic ideas cajole slyly according to the ironic, furious deposits. express requests haggle sl +2763|Supplier#000002763|yhMKsqKMP49okN,Lvd5SFwarU|13|23-410-220-4620|5951.47|posits. furiously unusual theodolites cajole. pending orbits print carefully carefully +2764|Supplier#000002764|lHWP7g8NCterjtVbJgoAOg|9|19-902-749-1334|1437.89| accounts detect among the +2765|Supplier#000002765|7bX3YXh7KlScRw7EbEU2lW|0|10-265-692-9778|8498.62| deposits. bold requests maintain fluffily amo +2766|Supplier#000002766|CPJjKybUHBxm0snUwnwWxfZZLk4sbE4JISVWhr|6|16-684-563-6309|-563.34|ounts. carefully final instructions detect across the furiously ironic pinto beans. foxes haggle qui +2767|Supplier#000002767|TH8T96SZPPsJOC5|0|10-387-614-6454|883.26|atelets affix blithely blithely ironic accounts. furiously express deposits +2768|Supplier#000002768|Z5XIwCs200u2A2aA|15|25-753-877-2336|3932.75|ts integrate after the blith +2769|Supplier#000002769|7QXYQHHuMvIkDPVwCi9jvD1TH|13|23-312-807-5269|7704.51|ackages cajole carefully. pending instructions se +2770|Supplier#000002770|Y4O7cs1mRjv5ih 0nt RFI2qwNK|5|15-931-941-9457|6748.10|ts are. blithely final theodolites across the furiously special packages haggle outside the blit +2771|Supplier#000002771|VdZoiNsPVr91KzOZ8kg|13|23-957-362-2541|3319.35| the final deposits. fluffily silent instructions along the fu +2772|Supplier#000002772|fiBJLxZW7th1rnfiLATfh D2m,i8GdROIf|15|25-484-519-3754|5326.03|e ironic dinos. slyly final packages above the enticingly pending dependencies impress fluffily pe +2773|Supplier#000002773|jaz81C2z9HKz,g,9a1z7Gdc5YWdoa|14|24-935-655-7184|5519.18|platelets. carefully regular excuses haggle carefully ironic instructions. slyly pendi +2774|Supplier#000002774|Fcc4cLWbvwjRPRtyMtLD3DU|10|20-223-712-8809|6885.10|have to wake ironic dependencies. furiously special foxes detect. furiously bold asymptotes thrash +2775|Supplier#000002775|yDclaDaBD4ihH|3|13-580-308-4425|6970.23|ly unusual pinto beans haggle fluffily beyond the carefully idle deposits. +2776|Supplier#000002776|sny7GWbRh8E0OH6spG7u|9|19-487-327-8393|7740.14|nstructions. ironic accounts sleep at the final instructions. ne +2777|Supplier#000002777|DC 0LAoxgBs VJX4|13|23-800-817-4822|599.44|s. finally express deposits cajole across the ironic packages. furi +2778|Supplier#000002778|hvxx8zSnxrwYV2pibE9eIYfs6mZ|14|24-704-773-8124|7749.22|latelets haggle furiously. fl +2779|Supplier#000002779|9OPa5O7wKtwossAJ1zxgEQmQppgt0beAS2AX4c|11|21-789-270-2631|-740.06|lithely quiet packages affix blithely across the fluffily even accounts. idle requests integ +2780|Supplier#000002780|I6AgeoI10 c0N03didb2NhzddiUg|7|17-482-626-4174|664.04|sual foxes. bold dolphins cajole fluffily even, silent foxes. ironic foxes sleep +2781|Supplier#000002781|M6OMG51Ozym8tFBslU7nVkExqaWIGU9N9E7TAc8|10|20-531-230-8533|4448.22|deposits. bold, regular requests haggle furiously unusua +2782|Supplier#000002782|5bA3KjG,mlOK,G9FavM0UpAK3P8J75Uma|2|12-360-905-4584|3152.07|s are carefully about the express, even theodolites. special instructio +2783|Supplier#000002783|rc rAYfUYEGG95YWvE|11|21-174-613-1836|9895.96|ronic foxes cajole slyly against the furiously regular platelets. slyly regular platele +2784|Supplier#000002784|GZxLX 7hGH0kmsmsOqCJqqUKMk13eploaWSXuT|2|12-698-910-4462|-570.94|deposits thrash. blithely regular deposits shall lose; slyly ironic platelets +2785|Supplier#000002785|ud,J9YhDpiDc1x8zF|14|24-251-430-2783|5181.05|s should play blithely about the express deposits. even requests among the quickly special exc +2786|Supplier#000002786| 72f,5VdQ4nVanOO|24|34-864-688-6054|-107.21|ffix furiously ironic excuses +2787|Supplier#000002787|LmMoJPKa9GVlhAPDndICfkPhhjfH1uAYx8e|24|34-670-823-3358|133.11|refully even, final excuses. fluffily even dependencies haggle requests. quickly ironic +2788|Supplier#000002788|r6oyjf1YBDaqeyd gQQmw|17|27-994-918-7883|2072.12|ove the special packages. carefully ironic packages at the final, regular requests run fu +2789|Supplier#000002789|7EkIldjP7M6psSWcJ11tf65GT7ZC7UaiCh|10|20-842-716-4307|9058.94|s. platelets use carefully. busily regular accounts cajole fluffily above the slowly final pinto be +2790|Supplier#000002790|EpAhauHHPf9ke5|8|18-337-445-1530|675.29|luffy requests wake blithely. slyly unusual fox +2791|Supplier#000002791|qN7ZlkL2KHGHsX|10|20-951-419-1791|6468.16|across the deposits. blithely +2792|Supplier#000002792|si64pUzgf3NdY|10|20-155-471-4831|6324.60|usly final requests against the carefully ev +2793|Supplier#000002793|aJBVO0DwezZsCpS9cD7DwubbGvSON48MGiWW|18|28-445-136-7777|2577.10|lly ironic theodolites wake furiously regular dependencies. even, regular depos +2794|Supplier#000002794|fURps89KuaNy1Jd1ehnMi7QFl8M9yyrK2O|14|24-975-523-8091|4271.28|fluffily pending requests. slyly express theodolites among the speci +2795|Supplier#000002795|1Pj8j m,aGVpW|8|18-452-280-8239|-584.79|xpress, final instructions. carefully final packages +2796|Supplier#000002796|8KznWokHECGTp|20|30-504-790-4841|3437.17| special ideas. carefully ironic theodolites after the fluffily +2797|Supplier#000002797|O2ul2htH4kg1TRsn5d|9|19-530-852-4505|5508.50|theodolites. carefully even platelets haggle blithely above the unusu +2798|Supplier#000002798|OYEhScazzY0d6B22YYByDoznCzVkdqwH|2|12-945-745-9514|3440.62|t the blithely even accoun +2799|Supplier#000002799|lwr, 6L3gdfc79PQut,4XO6nQsTJY63cAyYO|3|13-757-175-8800|8439.97|en, quick ideas detect quietly against +2800|Supplier#000002800|ZwUjfUl, NdgdRf9zoi lkMHn84gbJ1EDC|4|14-699-352-9705|3592.24|te furiously silently silent deposits. ironic, regular deposits affi +2801|Supplier#000002801|AeOzk8VQx5WiMjV,P|9|19-733-439-2229|4772.52|nticingly special excuses among the final packages boost furio +2802|Supplier#000002802|ioeBm2a9ag5MCFn026MUWYP|10|20-649-761-9820|8771.29|l foxes. ironic ideas are after the slyly ironic +2803|Supplier#000002803|6pUdfifEqaKeD3rScPUlSApW1TbxYlA5NleGhl|1|11-452-177-8866|6214.59|ording to the idly even packages do +2804|Supplier#000002804|H44ZbMgCvnMwy,JGY70L53iGLCRAg|16|26-504-934-9733|1717.95|deposits sleep quietly against the furiously silent fo +2805|Supplier#000002805|2OCTb0 6r2GYlor5s|7|17-713-737-2095|7556.69|to beans among the silent, final +2806|Supplier#000002806|MmYsVviAPaOtnHRHAf6Y XKCNcO|23|33-176-897-6738|8413.43|nag carefully special accounts. quickly ironic instruc +2807|Supplier#000002807|2nGNpyXEobNolKSeuspKvc343Aw,ys|2|12-830-799-6122|8651.48|usly alongside of the daringly +2808|Supplier#000002808|Xb39Au2OzG3CaL5ldiNGlsEbyDhd|1|11-279-836-2483|5547.47|ed theodolites sleep. unusual pinto beans wake along the pinto beans. carefully regular d +2809|Supplier#000002809|4sJNoIa81MpdL7vvp95IN|17|27-115-710-3700|9393.74|lar theodolites run furiously. sly +2810|Supplier#000002810|q17doVYZJqBV|9|19-891-319-1849|6722.74|itaphs use beside the regular theodolites. blithely final deposits use bli +2811|Supplier#000002811|5pvhhUcKa8UmuMWM6brYi|19|29-740-466-9013|6179.66|ly unusual requests haggle a +2812|Supplier#000002812|Q9sO3wZkBU5QBe0VITRWShv|21|31-791-182-6447|8492.64|tructions doze slyly. slyly unusual de +2813|Supplier#000002813|9 FfkJtSNtSks3|14|24-407-542-6533|8853.41|bravely even deposits. express theodolites was blith +2814|Supplier#000002814|VNdgEHJNmxEy,zD|20|30-761-456-1588|407.63|ckly ironic platelets. furiously unusual deposits promise. slyly quick deposi +2815|Supplier#000002815|Tu BAZdCROTW4QaKZ3zTnpfF,1zaMIeRL8a|7|17-733-125-7787|6984.35|s. furiously regular epitaphs nag express account +2816|Supplier#000002816|YUghXeSwJp1K3T5oTIs4aEGkh2u1hecU|16|26-892-838-6204|3051.71|e stealthily. unusual realms use slyly carefully ironic ideas. fluffily even +2817|Supplier#000002817|bm7o45AbMo|10|20-416-876-6667|5988.40| even accounts affix evenly at the accounts. quickly +2818|Supplier#000002818|kzzNb5Jcm9WNmB LGlHk7JgN7|6|16-847-601-2480|6109.09|lites haggle carefully among the unusual foxes. accounts are furiously +2819|Supplier#000002819|bx78XM7oJSA0t0PYr22zsMSdHv7|13|23-526-415-7857|-507.37|quests would cajole. thinly iro +2820|Supplier#000002820|Fmb9hufE41cerFFf2DiGF6yfDSe157Koz6S|7|17-358-908-6872|6475.21|Customer Complaintshe slyly +2821|Supplier#000002821|LnRvBX9m,QKj9jeoNpM5b5V|1|11-556-875-5083|3844.02|ve the furiously special platelets. furiously unusual epitaphs might boost against the +2822|Supplier#000002822|,aZy8lMNPn5HP0|5|15-438-209-6013|1357.16| pending packages haggle after the regular, bold packages. regular, even accounts wake +2823|Supplier#000002823|abnp,SINkto,Ci,bRO3PeFm k8BAVmBFYBOs|7|17-976-349-8153|4784.08| after the special packages haggle against the instructions. blith +2824|Supplier#000002824|thiY8LAz9mpadxC1Uy71CPDr VMcYl6w4CVjU5fF|14|24-489-868-4574|5011.66|. ideas cajole final foxes. final requests doubt regula +2825|Supplier#000002825|mzbJR5gHrpDZJ1SG YnDs BFH6aDRC8X|24|34-988-146-3830|1906.41|furiously special, silent accounts. blithely slow sentiments wake: blithely regular +2826|Supplier#000002826|iSDDxGpc3SRoEIC6rVf4|15|25-339-572-5765|7061.54|ual pinto beans are carefully +2827|Supplier#000002827|yaziW4tyg4TaX7j|0|10-570-512-2484|2337.77| instructions-- ironic, pending dolphins sleep carefully ironic ideas. regular deposits boost slyly. +2828|Supplier#000002828|amuisQibvSRaYesfuCxr1CM4pt U,POx|2|12-897-934-3069|7845.20|ly ironic deposits at the +2829|Supplier#000002829|MpOduzdNQVLn3l2K,fDBjoC0|20|30-172-382-6256|6322.96| was carefully regular theo +2830|Supplier#000002830|,tuLnQTihW|14|24-143-923-5463|8046.18|e express requests doze furiously about the special sentiments +2831|Supplier#000002831|8DGtt26QGtxI,3xEQ8gwSwY0JkzYpZWl4OjiunU|6|16-936-287-2633|7215.47| blithely regular packages above the fluffily ironic accounts boost +2832|Supplier#000002832|VZ4nLhhztXOX8pP|13|23-980-196-5345|4506.06| furiously ironic packages nod +2833|Supplier#000002833|TaAJUhGazS|15|25-391-375-6124|7364.30|es. finally regular accounts sleep carefully above the blithely ironic deposits. sly deposits m +2834|Supplier#000002834| cbmx43fm6SlE9UbdE7koBZlzRIeFwkXA|12|22-261-543-1063|8804.25|counts boost. slyly regular ideas use fluffily furiously unusu +2835|Supplier#000002835|rCYshZ3VTcDy|17|27-999-990-2918|-916.64|equests are about the slyly even asymptotes. bold, even platel +2836|Supplier#000002836|L1V j89w21qfT6Tnxjrmr6qzsKj|18|28-519-616-9647|3028.98|deposits alongside of the fluffily bold dependencies t +2837|Supplier#000002837|SrAe60iQ8wlU6wKYnTNoL3pNp w i0X7|2|12-746-794-6452|9031.33|nts are furiously pending packages. bold, unusual r +2838|Supplier#000002838|,IDsROC39qrP6m0 oHo|17|27-123-795-6158|9654.15|uffily regular ideas nag regular packages? blithely ironic requests cajole furiously +2839|Supplier#000002839|AbFEkk5348s|4|14-913-264-5815|5354.93|ag slyly ironic deposits. fluffily regular packages hag +2840|Supplier#000002840|nXVY3S0,1VZrTavsOSFyBZx DOUU|3|13-232-174-9777|7354.58|ronic excuses promise. accounts boost. asymptotes affix alongside of the unusual, express req +2841|Supplier#000002841|Qum176TQROLmtZ6hy,eX|23|33-688-720-2476|-487.07|lets are furiously. blithely final pinto beans detect quickly excuses: sile +2842|Supplier#000002842| yi51SqL7kw7sX0|0|10-908-979-4738|6787.09|the attainments use slyly according to the ironic, permanent accounts. fluffily unusual accou +2843|Supplier#000002843|FPBWGrodgNgZ7FZ0pHjDquoJbjflt1h6yP|1|11-544-776-8269|9685.37| furiously final deposits! bold instructions above the even p +2844|Supplier#000002844|hylGNK,m i9KSia0OTFxN6oE5LXqUP|15|25-459-219-9071|2016.35|sts grow blithely pinto beans. ironic dependencie +2845|Supplier#000002845|ZOlKEPI,8ftemk3cAGokylKstRcZiBT0sc|23|33-639-575-6452|6820.97| furiously ironic requests. carefully final pinto beans after the blithely ironic orbi +2846|Supplier#000002846|q2Q8fji7wnwDkpNjclU|20|30-336-286-7353|7151.58|the even, pending theodolites. regular, ironic platelets wake quickly silent requests. furiously reg +2847|Supplier#000002847|kKqzzQNY2FhfJMLQmDAh5p|22|32-867-760-4458|8720.73| final ideas haggle carefully about the carefully final excuses. carefully express theodoli +2848|Supplier#000002848|yxgyfj4kxIoLMs7kSiemccTW3N 4xxX7mT|22|32-141-986-9817|3344.01|yly ironic courts sleep. unusual, ironic theodolites integrate above the blithely special accounts. +2849|Supplier#000002849|9RiwUS 5QfH4hbZ5WAL0ksWRoBP4P0eybuGrw6C|17|27-507-769-5407|1751.62|ctions across the slyly unusual accounts hang carefully at the slyly ironic instructions. slyly iro +2850|Supplier#000002850|DRJkjJB9sELbmuvyCBjq61k2i4ICmt|23|33-423-379-1746|1431.24|ost slyly about the pinto beans! blithely rut +2851|Supplier#000002851|l6etLWkvx9|17|27-958-530-5167|6796.08|accounts. carefully unusual theodolites integrate slyly. carefull +2852|Supplier#000002852|SimT75OJBu|19|29-847-935-4880|6780.06|hely bold deposits thrash furiously. quiet sauternes lose slyly. furiously regular packages wa +2853|Supplier#000002853|rTNAOItXka|3|13-572-233-2928|1428.96|. quickly ironic platelets wake carefully ironic p +2854|Supplier#000002854|mSmLzuTu4LjFX9EJP6pOTZWe6kFTEX2wIu|0|10-208-594-1218|1563.66|ckly slyly ironic accounts. carefully express +2855|Supplier#000002855|nm5KMthLA3KqQCK3b,X|10|20-641-231-9291|-574.96|its are slyly against the fluffy, quiet +2856|Supplier#000002856|Gu,arFye zY5HBZkjf5yPGnQdMYLuVz3HcA3y|11|21-193-297-9577|3492.83| regular instructions. carefully +2857|Supplier#000002857|7 Hhv9CYdw,CJJNT|21|31-185-371-1494|5894.50|riously. blithely silent requests affix against the blithely final theodolites. regular dep +2858|Supplier#000002858|O kR3oe7lJ5pdsFx0N0mZ3 cCVrdikn,n6|5|15-284-893-1446|914.43|ully regular excuses cajole slyly regular packages. even theodolites affix furiously packages. b +2859|Supplier#000002859|McqoKuNb8OA6LfFDJcxe x lLw|9|19-741-641-8464|-211.20|lar platelets. slyly unusual foxes maintain. even dolphins unwind quietly. +2860|Supplier#000002860|fgNgVo75YdY8iaz1gHxxOPuGm0ASGP|15|25-117-806-3906|1274.82|the carefully final foxes are quickly ironic pinto beans. ironic pinto beans nod. final +2861|Supplier#000002861|0q6yjuni0wE5g|18|28-718-572-8605|6821.08|iously regular accounts haggle carefully around the quic +2862|Supplier#000002862|4lulEErFf6wQZfWtphZx5hNaDS3wmDIHiYH4|9|19-472-558-7064|8861.16|arefully express asymptotes nag fluffily +2863|Supplier#000002863|CKTfjYetZLnm1KxDLjb3Br4Nec|8|18-744-168-9042|9572.09|furiously bold requests boost along the accounts. car +2864|Supplier#000002864|iaWbXRM6tv9p|17|27-562-141-7557|1259.38|ely at the regular platelets. +2865|Supplier#000002865|OZrNw337it|22|32-783-627-3731|369.14|eep blithely. thin pinto beans boost pending ideas. express, ironic d +2866|Supplier#000002866|x 4Iy1qgbFyct1N3o5qsPNHU5wZhyOD|7|17-311-244-5890|9041.04|s wake quickly ironic, regular dependencies. slyly pending plate +2867|Supplier#000002867|SHa2rHj98mCJd4Q2F8MAqYkoaR88|17|27-100-247-1087|7802.82| regular requests. blithely bold platelets boost. final accounts boost furi +2868|Supplier#000002868|Y08OgmNLYxvJMmy5iAsbfOH2,Ty03u|16|26-133-738-8771|9111.93| use ironic deposits. blithely special pinto beans above the quickly pending platelets are slyl +2869|Supplier#000002869|JKSU80DiulM05|20|30-207-987-3948|279.99|lly regular packages-- specia +2870|Supplier#000002870|HJKP5aF9yuszsmqQ0t XZj2,8Ca6TxZ9|17|27-460-726-6100|6711.96|unts haggle fluffily. special, even deposits nag slyly. slyly regular requests integrate slyly furi +2871|Supplier#000002871|9ixHrIGvo8l4cDoJi1UxhLp|2|12-618-729-1384|871.84|ites wake blithely. blithely pending requ +2872|Supplier#000002872|MJ9iMo lUlY00f4glR|24|34-995-365-6753|9919.62|as wake furiously regular accounts. slyly close pinto beans haggle furiousl +2873|Supplier#000002873|QB784uIz5Lm|4|14-863-282-3688|1167.99|final ideas. ironic, ironic requests haggle furiously around the quick +2874|Supplier#000002874|LqNZRLweFy4xU8,osnTUNiqk1|23|33-345-155-4868|4322.70|d instructions. fluffily silent accounts about the final platelets haggle slyly quiet pa +2875|Supplier#000002875|6JgMi 9Qt6VmwL3Ltt1SRlKww0keLQ,RAZa|3|13-735-738-9581|5651.47|sits haggle quickly blithely even foxes. care +2876|Supplier#000002876|fJCG0fzsc,wRDttm3YiiEDSuK75h jPih1p|24|34-971-270-8904|8327.45|al requests boost blithely according to the final ide +2877|Supplier#000002877|fpT4ugECQjRaZYAwQghQwghs8ee0|16|26-102-519-6038|4471.91|ide of the slyly ironic p +2878|Supplier#000002878|jGbfUqUk32LrI|5|15-888-304-8960|5390.32|cial waters across the pending, enticing waters are carefully according to the quickly regula +2879|Supplier#000002879|EWfqEmqPdHxrKZwuSxpXVvIckc|2|12-669-954-3436|8148.49|ly. final packages are furiously. furiously ironic asymptotes +2880|Supplier#000002880|rq8DvKyfUz,FEwdPulv|1|11-518-907-4132|-192.70|fully final requests. carefully special requests cajole blithely. furiously ironic ideas sleep care +2881|Supplier#000002881|G4SolNwFBO3h6BJQ3 SmCWAup|15|25-744-839-5916|4253.81|eat quickly-- ironic dependencies affix slyly accordi +2882|Supplier#000002882|nU5GrwXophBkPAyx79fUftk|6|16-904-937-5856|2842.33|s. final instructions are. +2883|Supplier#000002883|UjClh1Zm63nTOUV llCzIBgkJafzkaGptMqyrY|1|11-895-494-6558|7060.57|s the furious, unusual packages are blithely regular packages. excuses w +2884|Supplier#000002884|FeHLSpzVJjrvyi51|4|14-454-798-2278|7991.84|rts haggle blithely carefully bold instructions. blithely regular requests are carefully aro +2885|Supplier#000002885|7wgc,Ccw1HNHeJZSWhmubwW,HK1Yy8ADe|12|22-492-408-3593|6967.02|posits cajole. furiously i +2886|Supplier#000002886|3HHp2t9xBFNN|1|11-653-573-9355|8715.18| regular packages. always even foxe +2887|Supplier#000002887|murbc6UWrdWJc|14|24-336-728-7434|9140.15|. furiously blithe excuses b +2888|Supplier#000002888|3AtRoxBFh6HIBa9kdBX,6,Ml2SZGUA|21|31-571-702-9106|1546.44| accounts above the slyly express requests detec +2889|Supplier#000002889|wq2VxMbXfgVeP5asKNnYGs7eO06V50BzUqfc|8|18-708-785-8483|6918.27|e. regular, special requests boost furiously fluffily ironic request +2890|Supplier#000002890|MA4ikT9mOlgSSZu17d0hdSGNc aFsivmV4 CIAS|15|25-877-235-8087|3390.21|sly final packages. even depo +2891|Supplier#000002891|w9uvS3xgBETHlWYAuU,E7mafb1sNAVx,qwp9 s|20|30-211-619-5853|8322.54|gainst the final deposits. quickly reg +2892|Supplier#000002892|g RYnxGX5D 192b2QWfOL9 DII|12|22-714-719-2067|9989.02|. regular requests across the pend +2893|Supplier#000002893|zfqhXpAB20J1xyGLE7fQbB y|7|17-104-541-9859|1487.31|ongside of the regular, close instruc +2894|Supplier#000002894|UVVMOnNXCV0Gcc|8|18-901-471-4979|3097.35|t. carefully regular dependencies sl +2895|Supplier#000002895|1KLa2tI64oL89Zt9HlhFC,SmKlQuawcp|9|19-160-315-1977|3201.94|ly pending requests wake slyly above the carefully +2896|Supplier#000002896|GRl9xCj5asXptSr9e3b1ipNVXF,c8|2|12-800-846-2689|1062.01|s sleep slyly. slyly ironic deposits sleep slyly +2897|Supplier#000002897|qRXRzPDLAOXiG71h6RV6Yxzk7UACZ|11|21-735-665-4680|5081.70|fully fluffy packages. blithely regular requests are fur +2898|Supplier#000002898|PibSzZrTRJkgRueCePU6lG 2Ue6|13|23-848-103-6610|2145.83|ounts. blithely even foxes grow slyly about the s +2899|Supplier#000002899|xHNGjUK4ehM1W07YQ|1|11-995-163-2776|4854.30|ep furiously special foxes. silent deposits along the regular +2900|Supplier#000002900|G0 wchKDgWmciVdmWz,4nlOOfR FIHmUL7c,|14|24-231-768-4259|8216.26|quickly bold theodolites detect requests. slyly final pinto beans impress. slyly blithe braids h +2901|Supplier#000002901|,vXeXneMMTdGMQlsc78te2gpYUxP1NMvI9XZPLa|19|29-639-586-6460|5815.58|blithely regular accounts sleep bravely even ideas. carefully unusual deposits according +2902|Supplier#000002902|lepogO7Z7RQdMaC3zz2YMy85F|23|33-108-310-3488|5395.28|y blithely ironic requests: carefully bold deposits nag furiously about the foxes. packages +2903|Supplier#000002903|KDCfcaxRMpHV|18|28-339-904-2431|3459.57|s use quickly furiously pending platelets. bravely final packag +2904|Supplier#000002904|3AVLyzSjCrHYuiLian1626bEfJgpw5J M|18|28-729-639-9863|8039.06|riously final deposits against the even, regular foxes sleep busily silent requests. final, bold +2905|Supplier#000002905|8MZyNigmh0f bGbvLkFTRd8MNu1E 31JXgb8pk|12|22-591-903-9061|8460.15|lar accounts. bold deposits detect carefully excuses. carefully express excuses thrash ca +2906|Supplier#000002906|498dqBD0lISHzpDOGmJf3W57mBSh woorgn|6|16-629-297-7385|1804.65|fily bold asymptotes. ironic dolphins nag quickly against the platelets: carefully ironic asym +2907|Supplier#000002907|GlC3UFYEf FCY|9|19-259-848-4746|1681.45|ld foxes about the even requests sleep fluffily acros +2908|Supplier#000002908|BTPIcYpTqm 9 iI|17|27-220-405-2567|-986.42|the blithely ironic pinto beans. quickly regular asymptotes cajole. f +2909|Supplier#000002909|3uyzaTPXLVpjxQchxEXV0IuWcfNUWnEGO3|17|27-475-364-4414|7143.05|. final requests detect furiously regular instructions. blith +2910|Supplier#000002910|nlH1gjApxHkQe5SU4iVZwi2xWk88wwhTWRkSvOBB|21|31-244-874-6754|8317.07|s. busily regular dependencies doze. asymptotes a +2911|Supplier#000002911|uWmxKbuocxwrqJVHAGuNNBO8nmY9D9 E8fg|10|20-393-818-1396|6036.57|eposits. pending theodolites dazzle blithely ironic, ironic warthogs. ide +2912|Supplier#000002912|Mool9qTbSd,c8hnQw|12|22-328-833-9401|1648.30|fully around the carefully even platelets. even accounts cajole furiously spe +2913|Supplier#000002913|822WFlkUiPOZ|14|24-929-759-3671|2852.99|l packages detect slyly pendin +2914|Supplier#000002914|fUC4IkGB8pt1S|21|31-392-590-6900|-882.43|ronic, even deposits. furiously ironic asymptotes along the slyly final pinto beans cajole blit +2915|Supplier#000002915|Eqgaj0E2tFwfqwLrUpb|20|30-367-332-1079|5954.68|c deposits haggle among the accounts. furiously even p +2916|Supplier#000002916|9kNV5lg5OgZp1EmDg7LJ0lu2tZCpFPyOeKFO|22|32-673-876-3048|339.82|s. slyly ironic requests haggle slowl +2917|Supplier#000002917|OLvrZSt7NBNyM6O1u XS686d3XyQ3 XY|20|30-746-268-6000|315.42|pinto beans. final accounts detect fluff +2918|Supplier#000002918|4w rdIiue3sCYPd93ZwfJ9C6r20BjAtmhwXwiv|16|26-661-662-2802|8279.39|nding dugouts unwind furiously. regular, ironic packages i +2919|Supplier#000002919|wDUxk1fSAitN6clR5Fau|17|27-836-653-5155|-296.50|its according to the express, e +2920|Supplier#000002920|Je2a8bszf3L|22|32-122-621-7549|8638.36|ly quickly ironic requests. even requests whithout t +2921|Supplier#000002921|S1UO1Dfj5y0kG0ui6gJ30yt5CUVaebtyIlRSp|0|10-392-215-8665|5010.73|ind carefully above the escapades. slyly even requests +2922|Supplier#000002922|EVl5qdnpZLOP|23|33-289-294-9337|8416.83|omise quickly ironic requests. blithely daring ideas haggle carefully +2923|Supplier#000002923|CYxD6MjRJYe|15|25-230-259-8699|-346.47|ar requests. furiously final foxes wake furiously alongside of the regular, even +2924|Supplier#000002924|6 nxmhb4Okr1CdJZPA2TaNRrLSXFfzy|6|16-624-993-1488|4797.53|lithely blithely ironic theodolites-- furiously final requests boost quickly ironic ideas. slyl +2925|Supplier#000002925|kUwc39OBibnkZa3WzgORN472f12SfOaSKX9VLU|5|15-479-367-5494|60.66|lites are furiously carefully express foxes. furiously even packages boost +2926|Supplier#000002926|f2WG37N3s3JcoAYNWMQym87cpNUu WpL|20|30-133-257-7524|5446.74|s the blithely pending deposits. slyly express accounts use. slyly r +2927|Supplier#000002927|La5H5k861WZ|21|31-331-903-5366|-684.31|ts: blithely final deposits affix quickl +2928|Supplier#000002928|NHwjBt6dLdB2o0JXweE tPFy6vNLqVtvp2|1|11-292-804-3769|2722.16|gular theodolites sleep unusual, ironi +2929|Supplier#000002929|CaN0aIlXQjQ1BTVPVEa |13|23-811-269-5779|365.08|sublate regular dependencies. regular, even packages at the ev +2930|Supplier#000002930|wMqzw,9XbQeExyiZVbMD|11|21-991-359-6441|6844.80| of the deposits. even, even requests doze careful +2931|Supplier#000002931|aUivhoesqMqv0FmJcPBMxBSl8DJvXBGj|7|17-905-318-3455|555.18|t the fluffily ironic packages wake furiously +2932|Supplier#000002932|gAIrgSCdvtJltNKuZKRGYeYLRf|4|14-722-108-2914|796.19|sits across the theodolites cajole along the accounts. warthogs impress slyly a +2933|Supplier#000002933|McM32vlbIGYiPW FpsyolgigYbQYE,|2|12-428-138-4594|3557.95|ic foxes. blithely careful instructions play against the furiously express excuses. furiously sil +2934|Supplier#000002934|m,trBENywSArwg3DhB|3|13-200-945-6095|473.68| ironic packages about the slyly even accounts cajole above the quickly expres +2935|Supplier#000002935|FjT8qWHLVhrrzbuM6|16|26-846-324-1432|6157.89|l platelets wake; blithely idle requests use. regular pinto +2936|Supplier#000002936|,Sa69LgNhOajcB3bGTCyspgkoFR6uKpFTpb|10|20-247-703-2293|5599.55|y among the slyly silent requests. bli +2937|Supplier#000002937|U0aanY JbHezCPWFAfZR5BEKjhgdk6JBB uZ|7|17-997-911-6062|4627.85|aring, dogged dolphins de +2938|Supplier#000002938|tBhqA8j84nU5lvYBRYrJQd9g|2|12-480-248-1944|8195.98| about the even ideas. quickly pending asymptotes are slyly! regular foxe +2939|Supplier#000002939|20abidKcKK t,Ujv|12|22-901-588-7686|5898.76| slyly express excuses are according to the bold packages. final instructions sleep. pend +2940|Supplier#000002940|kCNPx,OJnJWYi6qd32vGytk|7|17-935-264-7724|6449.93|ross the accounts. final requests use. regular, bold instruc +2941|Supplier#000002941|Naddba 8YTEKekZyP0|3|13-722-750-8709|8001.44|l asymptotes. express plate +2942|Supplier#000002942|LbLB8jQpSTY1Wwt8LVILeyMvFrug9ZmYy5|13|23-973-651-8097|3202.17|nto beans. requests nag beside th +2943|Supplier#000002943|WjiPe46HP4K7DC|13|23-357-202-6680|4877.60|hlessly ironic foxes. fluffily express frets are. slyly unusual packages wake blithel +2944|Supplier#000002944|IpInhMoSjE1V5Rvmn39R|24|34-430-611-7260|3831.09|nto beans wake furiously according to the final, pending packages. furious +2945|Supplier#000002945|H9 ZXkTE7i4J17oCa|8|18-946-203-4742|4164.16| haggle slyly. fluffily final platelets are furiously. special packages about the +2946|Supplier#000002946|0lwGexaCR2GyTBjQGla8L|0|10-805-470-4559|3011.77|permanently requests. theodolites cajole carefu +2947|Supplier#000002947|d,WhU9xiwlkGWuiwLC9UtlMCdlbSc7Z,WI24|0|10-779-667-8328|5760.29|its. furiously bold pinto +2948|Supplier#000002948|0p41jJN,CYS5uxiehjAjqod|16|26-160-755-2098|206.65|n packages sleep. pending packages nag slyly quickly special packages. express, regul +2949|Supplier#000002949|QIodj7qClw1wRvcFUdx04,r9j8|20|30-223-342-7753|6104.07| deposits according to the ideas cajole finally furiously regular packages. blithely bold acco +2950|Supplier#000002950|df M,arizMt1pWEpUbDGtFbc6MGQ|23|33-359-326-8404|3053.85|thely bold deposits alongside of the furiously even accounts sleep express, s +2951|Supplier#000002951|z79jAgZp,5L DYdjf7FP1H,ivePXiQiKNw3|10|20-478-840-2588|9095.09|ackages. slyly regular deposits are furiously furiously regular accounts. furiously bold depos +2952|Supplier#000002952|dIKGgSf6YfEeGOQ5shtlMdzeV2O|9|19-961-224-9429|3222.41|ackages. blithely express warthogs boost ruthlessly above the +2953|Supplier#000002953|TepT5BpdFsnZT grAXI7,x51O|16|26-118-226-8835|3955.99|usly final requests integrate carefully regular accounts. carefully pending ideas haggle aft +2954|Supplier#000002954|fR,RDoFTT2Y3XCDW|11|21-448-918-9473|2264.74|os wake bravely. slyly quiet dependenc +2955|Supplier#000002955|0QYFiINAihBC 2E4|19|29-411-552-9157|1379.73|latelets. ironic asymptotes above the quickly even excuses are quickly across the even reque +2956|Supplier#000002956|fEm5FTMzYzCKYZVbtv8I0dAXguqSd|13|23-515-852-9931|-881.32|ng to the furiously express theodolites. furiously pending requests cajole slyly among the fu +2957|Supplier#000002957|9LSKFBs0q1sWgGk|1|11-471-704-2831|2521.47|final platelets cajole slyly regular packages. furiously special packages +2958|Supplier#000002958|XNcxw,EjCJrTOqk3|5|15-451-727-3089|4301.53|slyly ironic packages haggle ideas. silent theodolites integrate blithely. theodolites against the e +2959|Supplier#000002959|0doqdWfV9actCvLdnvNZ9h4aYn9KBud|2|12-522-834-3875|1778.66| even requests sleep quickly furiously even hockey playe +2960|Supplier#000002960|KCPCEsRGGo6vx8TygHh60nAYf9rStQT2T|3|13-958-873-7133|4957.41|packages. slyly final theodolites detect quickly depo +2961|Supplier#000002961|WiNswf36sCtyTT2jLt7DBsQm0iUzIHqZmIL|17|27-242-303-7314|3133.20|usual accounts believe blithely quickly unusual pac +2962|Supplier#000002962|gRQXWtrUwB6Al99PmX1O|0|10-441-106-4025|617.59|tions boost slyly regular, express dolphins. ironic ideas haggle slyl +2963|Supplier#000002963|iTGObUj78Ni1TSLEWloxS9uC,K9Kvmw0 n|13|23-827-941-3544|4694.43|n asymptotes sleep blithely along the final, final packages. boldly dogged r +2964|Supplier#000002964|yJHDv1tHUT3zS8S|5|15-812-632-8371|5302.65|ep final, regular accounts. slyly bold a +2965|Supplier#000002965|MmAjwypVRRZs6y|19|29-966-692-5169|8651.05|ckly for the furiously ironic requests. express accounts haggle carefully blithely final +2966|Supplier#000002966|7YrfyeXbR385MUaYaS78fzpE8A6NPK|2|12-499-529-1414|9338.20|efully unusual requests wake slyly quickly express requests. +2967|Supplier#000002967|lIgtycEJeN4N1WUZG1hX8v1irzyj4qx7kCwrCZj|18|28-248-238-8501|8228.66|in the regular pinto beans. regular, bold instructions sleep slyly. carefully iro +2968|Supplier#000002968|Ck1lYj8berQBHcC|19|29-280-937-5732|5124.60|thely unusual pinto beans according to the f +2969|Supplier#000002969|fP16EawW6J6VROtePeotd|23|33-548-309-7067|8151.98|xes sleep blithely pending deposits. +2970|Supplier#000002970|5GLkZCF0VYvzQ4d7RQbMT9kEsB8HGAbY3jP|10|20-135-362-4484|3419.13|symptotes. furiously pending packages lose furiously. accounts integrate carefull +2971|Supplier#000002971|kzp PsZ5N2uId,Lv2vED,F4mBMhO JPOWl2d|11|21-238-891-2196|-620.02|eodolites boost furiously abo +2972|Supplier#000002972|UV6ajsKfv3WALu2LFPFrrl3IaPPF6YtVgoyClz1i|6|16-493-546-8467|5212.36|re quickly quick, ironic accounts? carefully unusual requests whithout the even requests use blit +2973|Supplier#000002973|x95AAV25GHIAYU yhQsM|5|15-295-459-7347|9071.54|o kindle regular pinto beans. ironic packages nag quickly. ironic realms sleep carefully foxes. +2974|Supplier#000002974|get47Fj5mzwXEbXGKePoPP7cXTWY|10|20-160-632-3353|3488.17|slyly final patterns according to the requests affix furiously at the quickly regular platelets. +2975|Supplier#000002975|5w03,6,CTJztE|4|14-822-991-8478|4692.62| fluffily. ironic requests are after the quickly +2976|Supplier#000002976|LD4J,ohC3sUG2ZT1PhOH|21|31-616-221-9262|7786.91|us pinto beans. slyly bold sautern +2977|Supplier#000002977|9NZKyvtwgBN,rXTIh8s759t1H4nVbN7,YamaJ4s|1|11-983-724-5780|5582.07| accounts. regular deposits dazzle fluffily alongsi +2978|Supplier#000002978|mY6Qc8LjRe3w1Ynou3lq3wq3|20|30-157-522-5634|7915.67|ly quickly bold requests. carefully ironic courts engage never against the carefully f +2979|Supplier#000002979|l YrwL2c5fBzytrU1|5|15-910-777-6663|2709.83|its. regular, silent instructions boost carefully special pinto beans. carefully fi +2980|Supplier#000002980|B9k9yVsyaXvWktOSHezqHiAEp9id0SKzkw|3|13-121-795-3494|2118.58|slyly carefully final requests. final deposits sleep blithely. regular, ironic deposits +2981|Supplier#000002981|5HRuYAcKGMcGfSkx0jQS46Fi|19|29-591-615-9173|8921.28|: express, special requests breach quickly according to t +2982|Supplier#000002982|PzCwmAnHPfqfwKmdwHWkAtWz8xFgkj9x9 In|1|11-661-855-9797|8085.36|boost furiously even deposits. blithely bold acc +2983|Supplier#000002983|0BPM3teAFg |4|14-331-253-4156|5714.48|iously express deposits use according to the fluffy requests. even accounts are across th +2984|Supplier#000002984|WvzfKAsjOtpNPSbjor7|4|14-829-961-8448|351.44|ully ironic accounts detect bold packages. fluffily final platelet +2985|Supplier#000002985|XcGrON6iJDJ4Nki|5|15-653-433-7716|8543.83|coys alongside of the asymptotes haggle +2986|Supplier#000002986|ANRiao qNt9h3vyEi|10|20-957-807-3635|222.35|bold forges. fluffily thin grouches boost blithely against the bli +2987|Supplier#000002987|aVQPPTYTTw0VYSWfa8y|22|32-632-644-8964|5952.20|theodolites; ideas according to the multipliers nag foxes. accounts nag quickly. +2988|Supplier#000002988|Ck B7MmscxnrrFaB7H8cLxXUYY,26Xb DYnh|14|24-952-723-3371|2622.64|pecial instructions after the regular, final req +2989|Supplier#000002989|mHyjXyVDWKtiwQmeEcNXXksmyKrYDOjVYs|7|17-282-348-7573|-292.07| requests cajole quickly even depe +2990|Supplier#000002990|s 3hZdJj0MbU9W7n11z,|20|30-201-740-7249|6028.70| the carefully regular instructions haggle against the final dependencies: courts nag +2991|Supplier#000002991|NC0h3TZcVC|14|24-607-395-4134|1393.90|ackages integrate carefully. furiously +2992|Supplier#000002992|3KA5HGgzTK9GJOG4v2BtScFmL8sk e|23|33-297-879-8463|6253.17|regular requests! slyly regular gifts sleep quickl +2993|Supplier#000002993|CDRN7azuEWTawl9G|20|30-541-514-5637|977.32| are regular requests. reg +2994|Supplier#000002994|gMO5J1X9uSPlI3glDUSYsud,yzfOG5HTQ|18|28-200-203-6249|805.42|unts are about the closely final deposits. special asymptotes about the regular, bold ideas cajole +2995|Supplier#000002995|RWgWCeIgsXXHgCxWZ5jd|21|31-224-848-8404|4614.07|uests detect furiously! blithely bold excuses detect furiously carefull +2996|Supplier#000002996|Qn0ukH2JOjam20O8Z5DeB1XS2t|7|17-253-104-7616|7710.82|ies snooze quickly. final, pending deposits sleep according to the furiously ironic soma +2997|Supplier#000002997|79e1hAERDnLwvCE2H|15|25-888-696-8121|2868.48|ys final ideas haggle after the furiously express requ +2998|Supplier#000002998|D,VPM01oL0gAHNvGaEpVJmM9 Vg402aLMvv7cq2|17|27-657-379-1540|6517.02|refully ironic notornis cajole along the carefully final dependencies-- warhors +2999|Supplier#000002999|Gr5PrGGyZa8U|9|19-105-965-8570|2528.66|hockey players detect across the sly +3000|Supplier#000003000|JtDvRf4iWHJkj54PYxl|21|31-910-209-2002|4223.03|slyly silent asymptotes against the carefully regular asymptotes cajole sly +3001|Supplier#000003001|p,vtdzmZWY7YiO|24|34-171-876-8885|2399.74|latelets about the fluffily regular accounts us +3002|Supplier#000003002|c0hweuB4hWi3gf0SA90AQjNmaglOQANz|23|33-348-775-8804|4366.90|longside of the regular requests. bold, regular accounts print furi +3003|Supplier#000003003|8P71Pj89GiBowF,bvouOI vLK74vOCllIy1GYUJ1|4|14-926-417-9940|545.44| foxes. instructions x-ray +3004|Supplier#000003004|O4nA6,ODeYqsE,ekbxsHM5vc|5|15-931-357-9763|2072.87|ounts impress doggedly pending accounts. dogged, silen +3005|Supplier#000003005|7xWPGUbOUoeiPLNo,r|1|11-352-166-3995|-361.14|s; unusual, bold dependencies affix q +3006|Supplier#000003006|y,NWe4l8cUbXgvwLDBpNnUmMxI5JODzUocB5|1|11-335-383-5953|227.50|deposits. unusual requests +3007|Supplier#000003007|99oAgeU6C7zko3KhdUC|15|25-894-347-8240|4572.92|ss requests wake furiously ironic, unusual depo +3008|Supplier#000003008|TSt8rc4Usc 1UdSD,6VwiuG2XS0tBzY|9|19-634-851-5012|9080.40|are. blithely dogged pinto beans about the carefully even pinto beans sleep furio +3009|Supplier#000003009|oTbwfQ,Yfdcf3ysgc60NKtTpSIc|10|20-306-556-2450|9315.33|the carefully special requests. express instructions wake +3010|Supplier#000003010|vOeYAusMVN9HDeDJZuvr9nC0E1a|1|11-900-702-4200|8793.40|e slyly final instructions. daringly ironic ideas kindle carefully abov +3011|Supplier#000003011|vfL mV0MTdyozfRIPZkJbM1Z7Lcm2NCPIj6qSgBz|21|31-773-703-1358|3755.20|nic ideas. special theodolites wake slyly across the thin +3012|Supplier#000003012|5rgJwAVOqj6KqSAKn2eX|9|19-119-699-4000|4611.45|ual packages-- regular pinto beans x-ray quickly above t +3013|Supplier#000003013|LPjXyEI0Vb7,|4|14-108-315-4034|7568.83|gside of the slyly special braids. quickly even accounts cajole amo +3014|Supplier#000003014|HfrcgK9O6K,N0z|20|30-522-592-6417|3434.23|sleep after the final asymptotes. regular, regular accounts hinder against the f +3015|Supplier#000003015|sMfAdl5XgLeFoiL|21|31-863-281-6684|91.78|the braids. special accounts cajole: bravely final theodolites use slyly alongsid +3016|Supplier#000003016|VwqWdkQOKILMPkBiXMKBSza d3pdhk79GjOfO|17|27-929-687-3620|483.30| cajole carefully. furiously unusual platel +3017|Supplier#000003017|45FE 2ykESNBNe4Sp b|4|14-285-994-8319|2695.73|ravely according to the slyly regular requests. sl +3018|Supplier#000003018|1vxIseSmPRpCXvKpPbCJhFDgPZVLAXODdix|8|18-268-925-6512|7117.46|ts. furiously even requests are across the pending, pending deposits: requests +3019|Supplier#000003019|ycKfX2I3O8GOyhoqc|11|21-532-192-1317|7110.22|final platelets. furiously final theodolites sleep carefully blithe +3020|Supplier#000003020|wAwQ5b,l0CsJ,dbv|19|29-163-423-9151|6566.21| bold deposits use. excuses sleep among the carefully re +3021|Supplier#000003021|95BmhvrcF4aL3B Amoz6pl|7|17-444-213-8631|4297.93|osits above the ironic, final braids are express request +3022|Supplier#000003022|NXlEQWla3XSL2hyeWEHEvDWQdHV0,dd|18|28-399-458-2092|7285.50|ep blithely against the foxes. regular courts haggle. blithely silent pinto beans are theodolites. i +3023|Supplier#000003023|a2vIsI7JLHFfBvJoSmPGfm|22|32-598-929-2593|4383.12|arefully unusual instructions are slyly regular accounts. busy courts sleep furiously. +3024|Supplier#000003024|qL7O1fXRUnUP2brTlcjMZ|19|29-178-209-1949|9089.00|ngside of the requests. permanently regular pinto beans sleep bold requests. +3025|Supplier#000003025|HtCJxqcyYoMKBu7BQ5hRdVXj24Fkw5s8LgGe3|18|28-337-459-6916|-88.05|ests. special patterns are slyly blithely ironic +3026|Supplier#000003026|La4dO0Ey1H2mXK8 N|19|29-673-539-2157|3011.28|usly unusual deposits. car +3027|Supplier#000003027|s5EizSYfzP2vYQkHI|0|10-170-124-1414|8510.97|sts among the even hockey players wake carefully toward the quickly +3028|Supplier#000003028|jouzgX0WZjhNMWLaH4fy|3|13-671-694-4022|8853.35|. even deposits x-ray quickly. +3029|Supplier#000003029|aWkIsIRUh3zz8LiwvImuv|6|16-692-588-9167|-640.95| will have to sleep furiously? pending, express deposits against the final +3030|Supplier#000003030|VzSGqv4ObRMkugkPNrbNmpXKoAcwtmlO5N|21|31-179-866-4329|5551.50|xpress accounts nag furiously silent dependencies-- pending, ironic pearls use +3031|Supplier#000003031|IxvC,mH98sxToEwvuu5lS6m6h0Nvnpsb0rEL|11|21-564-970-5808|50.64| unusual braids. pinto beans among the slyly express platelets are alongside of the ca +3032|Supplier#000003032|gyIGDWCX2KTglZJUW,1|16|26-323-974-9307|2043.73| packages haggle slyly ironic, special dependencies. slyly regular requests wake permanent +3033|Supplier#000003033|4F nwz,wreEqeQrvSNsR0n6N3u50 p8SoiY1dN|13|23-117-168-6842|4954.62|leep about the special courts. blithely special instructions try to slee +3034|Supplier#000003034|MSKa6VFpxLB5isjLyUh|11|21-694-316-9987|340.84|sly against the regular ideas. platelets along the furiously f +3035|Supplier#000003035|j12TLg2wqaP9Ii|10|20-919-155-3978|6437.53|s according to the fluffily regular escapades wake blithely alongside of the furiously unusual accou +3036|Supplier#000003036|SifDb5Zv8PqS|5|15-950-158-1297|8549.25|ely special requests cajole thin, regular platelets. regular, express deposits across the c +3037|Supplier#000003037|aBPbT4XfxDheA|22|32-150-300-6644|3224.71|pending dependencies. carefully silent deposits hang slyly brave platelets! pac +3038|Supplier#000003038|F5Tz7P juuCbABDuW8JGomRFxqVHBWyQrsLwg4i|21|31-160-454-6537|9629.88|odolites. bold, even packages across the blithely ironic deposits sleep +3039|Supplier#000003039|al8BwGPFlYt,Gw5otw0bVs0c|21|31-564-294-2527|4403.31|accounts. slyly final foxes against the carefully regular platelets solve against the regular foxes. +3040|Supplier#000003040|xv4jkaRH0sYJZ0X0dsqAu4p 2Yw|8|18-683-528-4003|7500.88|wake slowly above the ironic deposits. blith +3041|Supplier#000003041|V0H ouJwyajn70|4|14-437-897-9946|5324.47|y ironic deposits wake blithely. furiously regular pinto beans impress furiously across the depos +3042|Supplier#000003042|vpaZSrbFsA,Ds89Q7|21|31-607-632-6095|1366.40| pinto beans. final packages boost carefully. fur +3043|Supplier#000003043|VFXltguBKJ|16|26-979-145-5297|6055.34|s. express accounts nag slyly alon +3044|Supplier#000003044|2zldvYuxlCiK4|15|25-761-914-1310|9091.06|ronic courts. quickly final dep +3045|Supplier#000003045|kem8cuT44 Zk8uFDUUser|19|29-141-178-9154|1451.71|lyly express theodolites. ironic requests across the slyly final instructions wake blithely alon +3046|Supplier#000003046|ZsTNXVjY6mqeQeYT4oW1vb KfBj0|17|27-323-628-1319|9812.48|ut the regular packages. quickly even foxes slee +3047|Supplier#000003047|FuYidA,WeJOANHkgrypZb0wv Zcadcy3iN|8|18-526-898-1547|4477.17|e slyly pending packages. furiously express deposits nag furiously silent theod +3048|Supplier#000003048|Cu nnq0xHJm8MfHUw5KwjlYgI|20|30-282-606-3047|5841.32|s sleep carefully unusual deposits. blithely express reques +3049|Supplier#000003049|l,TKJ7YWuyqUEiFkJxT87l7wPH9T tUSPl,|8|18-532-916-5559|9033.39|lyly final theodolites? carefully ironic instructions haggle. quickly expre +3050|Supplier#000003050|do1y5Z5UnEoJdi1Fs9im143KSnehd IYe|15|25-641-183-2596|275.36|l platelets are blithely careful +3051|Supplier#000003051|W7oijeW6YZpnEAxRCXhA6v27cdR|24|34-233-112-6252|5938.37| are above the special, sly dependencies. slyly furious packages detect carefully around the +3052|Supplier#000003052|gDTZv40h9WvgCfZ,QneRmvxnTtSG|17|27-214-211-8159|1149.77|he instructions affix furiously special, regular frets. bl +3053|Supplier#000003053|EVzC xEU8hGQ1rTnsO|23|33-265-220-3273|6214.16|rint blithely carefully regular accounts. slowly bold asymptotes are above the sl +3054|Supplier#000003054|28NgA4borqn|8|18-791-898-4292|5437.31|ally even pinto beans haggle slyly always express dependencies. furio +3055|Supplier#000003055|hIK4aKsmc jZgrCSUk6EyGArXqBa4|18|28-422-347-4948|5898.52|ets. bold dolphins haggle regular dolphins. furiously express reques +3056|Supplier#000003056|I8fnZ5uR787YgM5P1urcBhFF1ZeJ3FMesySwkE|11|21-313-342-4216|2450.36|uickly silent gifts can nag closely alongside of the blithely silent accounts. pinto bea +3057|Supplier#000003057|nI,owTJt24PQXdzm|10|20-976-269-2335|8193.26|aids sleep. carefully regular depths are of the blithely express depo +3058|Supplier#000003058|Aw XxUmBnEk |5|15-915-896-1162|4169.53|ully ironic theodolites sleep according to the carefully dogged deposits. carefully ironi +3059|Supplier#000003059|4L5nNuegCdCY3m,88BFVEut28TFlbZ6hHCj|14|24-184-213-4553|5231.48| deposits cajole furiously according to +3060|Supplier#000003060|IkfB9x222pSkvA i,t5M|2|12-218-639-2134|527.62|unusual deposits haggle slyly ironic platelets. bold decoys cajole sometimes eve +3061|Supplier#000003061|GX3lvTUJ,jS6vR5WqZyCv6dRhdowFMxwOshCquq|13|23-468-814-1685|-300.50|e the fluffily even theodolites haggl +3062|Supplier#000003062|LSQNgqY1xnOzz9zBCapy7HwOZQ|3|13-366-211-5375|7425.18|unts sleep carefully foxes. pending, silent accounts according to the +3063|Supplier#000003063|kBUdB0mUgc HX2EzKnh|20|30-532-979-8127|9093.11| the slyly unusual accounts affix slyly bet +3064|Supplier#000003064|ilyfT4q21TMX7vowjzwl2uinnz|22|32-591-459-2474|2930.00|fluffily quickly special pinto beans. regular foxes cajole frets. blithely +3065|Supplier#000003065|CqIGkVuAzyEhk8K5|1|11-110-407-1470|6515.91|slyly after the even realms. pending, final pinto +3066|Supplier#000003066|rbLg0pp3sKYeZ,Knk8L8oKmon,LKWX|11|21-190-153-7754|5919.17|ove the quickly special warhorses. slyly regular requests according to th +3067|Supplier#000003067|9EPagnou6ashdkFA|6|16-785-224-5371|8591.14|ld pinto beans. slyly final +3068|Supplier#000003068|aVgMmx3nfzPgpKm|18|28-145-548-2517|3353.86|e blithely pending asymptotes nag iron +3069|Supplier#000003069|3jzeLuounYUDGGv9bXg8RNd|2|12-329-596-8136|2293.77|ly pending requests. fluffily regula +3070|Supplier#000003070|F0sjr17IwdKH9B7DOZXOnM hjhOsfHy9okdsk|22|32-581-887-7880|5337.68|nic packages among the deposits cajole according to the carefully special foxes. furiously ironic ac +3071|Supplier#000003071|7VZ5G5mbbLingg7z3gUY1ojC5QR5hkAZ,ok5d|9|19-948-142-5082|3452.33|al sentiments cajole carefully after the furiously regular instruct +3072|Supplier#000003072|IFvxRHT43mth,S1WjX8mNhgm6UCEcZKyD6H|8|18-664-412-1510|7279.73| the carefully express accounts. final accounts boost slyly. regular, final asympt +3073|Supplier#000003073|HCXoX79onWt0|24|34-892-575-9544|6455.78|luffily after the bold deposits. +3074|Supplier#000003074|EjHhAbYoa9Vv v771OEOXWCkARiH3b,|4|14-361-407-4649|7334.19|the carefully final accounts. unusual accounts integrat +3075|Supplier#000003075|4r1HsHGBEH88qIxsG8Dr9|20|30-700-119-4521|8855.28|packages. carefully unusual requests wak +3076|Supplier#000003076|T8pR6AMA3RRjXPDoqZ9CnuY HTZHbutrLh|4|14-395-910-3054|9759.99| special warthogs cajole quickly a +3077|Supplier#000003077|RXoE5UdenTMwvmqDmkI8XL,WPT|12|22-313-342-7068|2179.82|wind along the special accounts. final, final pinto beans wake blithely. carefu +3078|Supplier#000003078|WM2MwAU0hKWYd3tY5Y8|11|21-254-239-1333|4171.97|uests grow enticingly carefully ironic dependencies. furiously special +3079|Supplier#000003079|mxx x,pVuZ,DWWCJ35GGlWiY|19|29-267-241-5865|2289.04|quests? unusual platelets print slyly among the slyly silent excuses. s +3080|Supplier#000003080|V3DF4DVnZlS4xVJbx049Hr3 WJRr1UroXUCJAJv|22|32-704-119-9879|2867.26|ackages haggle above the fluffi +3081|Supplier#000003081|vKnj7hZN,Ky1YyqUVr7h|6|16-656-337-9927|8022.42|special accounts beside the pending foxes wake against the furiously fluffy packages. accounts ca +3082|Supplier#000003082|vLhIfPUhiW1Y rYmcj|23|33-680-262-1683|3784.36|y after the unusual asymptot +3083|Supplier#000003083|1LKbnA,BDdWyZnzW|18|28-403-241-5657|-209.99|blithely about the ironic, final +3084|Supplier#000003084|i4UjQ3YiKVKgrqqyp6IhLgpGqotVRU2Y|11|21-899-139-1798|-354.72|refully ironic deposits. finally pending requests according to th +3085|Supplier#000003085|aM6i8v3jKaD|0|10-428-575-1918|9112.98|l sauternes wake. theodolites haggle furiously. regular, bold req +3086|Supplier#000003086|EdiLbOuVZPvcIKQ 8C53GAQCRGDQEn|6|16-932-836-2080|6882.56|e slyly regular accounts. carefull +3087|Supplier#000003087|ANwe8QsZ4rgj1HSqVz991eWQ|3|13-939-850-7799|8218.17|pinto beans. fluffily regular deposits against th +3088|Supplier#000003088|DNwMHXZdqZ0T8enmPDX17Z73rah4yhMUO|24|34-331-890-4202|4759.38|ending ideas-- unusual deposits integrate slyly final, pe +3089|Supplier#000003089|s5b VCIZqMSZVa r g7LTdcg29GbTE7rI1x|3|13-279-334-8025|7901.39|lithely unusual requests. bold instructions haggle. regular deposits +3090|Supplier#000003090|bIUASui5l2t2MvJMEV6o2MHk0iZWSWdtuBuq|21|31-945-900-7711|4674.04| dependencies affix blithe +3091|Supplier#000003091|JNPFRBSLdD4DksRyryH7eaF51aut2QCYp9p4|14|24-781-138-5146|3530.16|. platelets sleep carefully ironic asymptotes. blithely express depths cajole +3092|Supplier#000003092|K HgdzaxErLyn4PNiB2CZhbmriyoHszsh|7|17-338-677-7980|6438.45| regular dependencies. bold, ironic warthogs sleep according to the fluffily regular +3093|Supplier#000003093|76Xzqa8AVjFv,GXw4gK4WQbAkT,W8MEH2|2|12-877-231-5288|2080.16| carefully theodolites. even pinto beans nag against the pending foxes. furiously regular ideas aga +3094|Supplier#000003094|hO4pweMKWo50zYMg4ENpoSpZ15YoVFvYcK|21|31-968-969-2992|1152.25|le stealthily. special, even theodolites cajole blithely across the r +3095|Supplier#000003095|HxON3jJhUi3zjt,r mTD|3|13-277-998-5722|3766.22|uests sleep slyly across the even accounts; furiously unusual packages nag against the evenly f +3096|Supplier#000003096|,ZraWb5SVMxGHmoNlmKLSqKjyC Q9JSlujpQbW49|0|10-478-465-8845|2860.56| deposits above the ironic, final requests +3097|Supplier#000003097|EiTFz f hHnMEp9l2 DqZXvfmVB5g|4|14-516-370-9680|5388.12|uests sleep fluffily along the thinly regular requests. q +3098|Supplier#000003098|z7pX6i3bDtay7JwDHnXJ5m5OKPpD0gf4h|10|20-944-694-6470|4159.47|apades. slyly even packages according +3099|Supplier#000003099|TbcJ00qZ1y|10|20-251-481-5942|1255.29|ithely final theodolites! slyly ironic theodolites +3100|Supplier#000003100|F6e6Cd9OxiEnPYEvvE3Exj85Y8UX7e YW7a|14|24-282-786-1518|8602.02| requests. furiously final depo +3101|Supplier#000003101|5mMVwiWrPEu6Hh9|20|30-504-674-8139|-704.41|ts affix quickly ironic requests. pending pinto beans wake. slyly ironic pinto b +3102|Supplier#000003102|m8,BcGs cif2 SyoQ5oIrb3 |19|29-467-229-9822|5879.27|against the thinly regular requests. blithely regular platelets use quickly among the quickly expre +3103|Supplier#000003103|r7vLesukWuV32ZGgbffKoPiqaopP3,IG|5|15-501-459-3317|8990.62|unts. dolphins after the furiously regular request +3104|Supplier#000003104|j7PAWf2Bo70YiHbpsdAtlvnIm|9|19-957-831-5492|7466.64|sly silent requests are. +3105|Supplier#000003105|oY9vcEXgL9I3zNx9tukxzCQ98MAyvf30DPzJmxP|19|29-831-521-6983|1896.00| unusual accounts. slyly even dependencies unwind. ironic requests past the carefully pe +3106|Supplier#000003106|74NWCBlio4ROyS5ENqmY CDlazIEtSxIQPb5|15|25-234-303-6527|8070.40|requests! even accounts haggle furiously after the blithely unusual packages; slyly bold pint +3107|Supplier#000003107|Vt4uiXc8CmFbXqY0VmNIELot3Nybh|2|12-322-314-7950|7308.17|eans. regular, pending pinto beans against th +3108|Supplier#000003108|XEBuwVaeU4N8eTeIty|7|17-465-157-6466|482.13| detect blithely blithely bold packages; furiously even packages around +3109|Supplier#000003109|2WfTvl8mtHqnyueg7Ki|3|13-589-994-3136|5135.46|ainst the special packages. blithely express deposits solve furiously. furiou +3110|Supplier#000003110|TX6,eAMqVun ed,piJHFzSuK6 f9jvzLiwsxJ9u |5|15-619-843-4600|3748.92|boost. unusual, ironic accounts nag carefully about the fluffily permanent w +3111|Supplier#000003111|4Zdm57kzJd8jPWW|24|34-291-865-3385|9107.38|rious pinto beans maintain blithely against the fluffily regular instructions. fluffily +3112|Supplier#000003112|9b0LhC9HYkb5clk8Nwn8st7ym0f2lpnRIYytm|23|33-739-982-3555|840.45|cajole around the quickly unusual requests. carefully even ideas a +3113|Supplier#000003113|HjX8M2Bjlz7pAcLzpyKT9 wNb|7|17-164-471-2650|-604.88|he ruthlessly final requests. express requests cajole quic +3114|Supplier#000003114|sbxYGYX1kDA6Ic xApo4O29CzR|22|32-129-694-7833|6627.17|pinto beans: quickly ironic deposits at the blithely bold pinto beans cajole across the bra +3115|Supplier#000003115|2o95Lhh3xn|7|17-905-443-7764|5855.84| the ideas? blithely express ideas wake blithely ironic pinto beans. dependencies are ideas. theod +3116|Supplier#000003116|yWpyR17u75wr0PH99QsdNNM7e92dAsQ0H1J8|4|14-461-145-6919|116.03|ar deposits. quick platelets are furi +3117|Supplier#000003117|awKOkc6y,vmP3jAD6BfL4i|8|18-191-586-4745|1028.66|ke furiously according to the unusual, special accounts. deposits ac +3118|Supplier#000003118|ryyoRKcYlBIioVfM4|6|16-943-521-7478|5554.39|: even requests maintain slyly about the busily reg +3119|Supplier#000003119|BesKnh7UtGXjw1SRrhcHipB|1|11-940-295-5414|4589.25|of the even accounts haggle blithely special excuses. regula +3120|Supplier#000003120|DBhl1PhP1V9vK7OMmo7Ri60dcI|9|19-634-281-8564|8461.22|ironic sheaves at the quickly regular packages integrate furiously furiously pending packages. qu +3121|Supplier#000003121|wNqTogx238ZYCamFb,50v,bj 4IbNFW9Bvw1xP|19|29-707-291-5144|8781.71|s wake quickly ironic ideas +3122|Supplier#000003122|oKI3ONEMwkYHffGipz L1z1bfsSg3|7|17-478-332-6727|5270.82|gular accounts sleep quickly pending packages. regular instru +3123|Supplier#000003123|aX,0xfzbndYceq2PDE7Plns75HfJ|10|20-561-221-1424|1615.79|ndencies sleep carefully regular pinto beans. furiously bold deposits after the even accoun +3124|Supplier#000003124|YYHQhbXYOI642fFIjZBhgKBPh1P0Edj|15|25-992-468-6188|6533.63|as alongside of the even requests +3125|Supplier#000003125|0A 825 JYoZc3fVFu0i6cMLWyo724X Cu77XB|13|23-561-829-8025|6735.43|l accounts according to the +3126|Supplier#000003126|RHm9TO8qDwK8WCjjgKqpruvKXNYIN|24|34-979-246-3014|-558.90|iet accounts. slyly special instructions according to the pend +3127|Supplier#000003127|e5zpOo2IbVIwawn2OeD,uhW5urkAgBm6jTcyiA|24|34-358-351-8680|9520.15|press along the quickly even packages. deposits +3128|Supplier#000003128|4ElO3EqeTuKYNh,SXKuqz2m21aPP|24|34-291-222-2494|8282.39|posits sleep furiously furiously regular +3129|Supplier#000003129|KUMv eIKElFwaK54Z3M Aku9g|4|14-479-809-1267|6130.92|ironic requests haggle. ironic, regular instructions integrate. fluf +3130|Supplier#000003130|HZnyZxed7QuysySWyvPR8xb8jQvB89szH|23|33-330-263-8188|3385.65|ar instructions cajole carefull +3131|Supplier#000003131|D2geDBAwmRoaL7ha1MTaQTyg1AAS|7|17-745-613-5999|8626.28|ly. requests doubt slyly. blithe +3132|Supplier#000003132|pfNmYxAAzFPdszCzw|0|10-724-107-1782|-30.35|uests integrate fluffily after the carefully special account +3133|Supplier#000003133|ctd9ax8DHT93kvfF91|6|16-259-195-1994|4876.86|ly regular accounts. furiously final pinto beans are quickly according to the unusual instruction +3134|Supplier#000003134|Tf2g02QiZNw5 25ztui bmuYKO1gHT|17|27-331-187-8404|2391.94|nic ideas haggle quickly. final deposits sleep carefully al +3135|Supplier#000003135|Gi51QG3FkK0KELpgnNvS0mQyXWi Bvc|2|12-360-560-7225|5319.06|xpress theodolites sleep fluffily around the even multipliers. unusual accounts above the expr +3136|Supplier#000003136|fz9yPkt1VOpmTka|7|17-835-332-2709|9468.72|blithely instructions. carefully final fox +3137|Supplier#000003137|dlR1SeOSy6FG|0|10-770-759-1402|1827.12|ly daringly bold platelets. regular ex +3138|Supplier#000003138|gIv3ntIwWm73YaBiJVA6d3ewf8 C9m1|14|24-345-619-5405|5287.18|ckages. final requests boost fluffily. accounts cajole stea +3139|Supplier#000003139|lyCs,nPAiP0Ho9ZVH8Niij5b|24|34-464-687-8035|1562.91|le slyly. slyly regular deposits +3140|Supplier#000003140|ZUyGRGQ rApQ7CHuglVlEsk 91w|21|31-146-104-9066|8811.28| quickly regular platelets wake furiously final requests. furiou +3141|Supplier#000003141|NY9pUW7,7YLdShXDLWVRt2sz|15|25-619-837-1643|7302.75|al foxes across the regular, pending theodolites use blithely about the slyly regular platelets +3142|Supplier#000003142|dqblvV8dCNAorGlJ|7|17-595-447-6026|8307.93|olites wake furiously regular decoys. final requests nod +3143|Supplier#000003143|hdolgh608uTkHh7t6qfSqkifKaiFjnCH|3|13-556-217-9388|6414.11|to the slyly unusual foxes. quickly close pack +3144|Supplier#000003144|aFox92QyiuMnhTngKPpVZ3yDjMOZ8UvS9BA|0|10-552-712-2179|-102.20|lites. regular theodolites haggle final foxes. fluffily +3145|Supplier#000003145|UzjUeHAN,FKq6Zl7jj7CewZsJflXxWG9F|2|12-143-538-1885|7192.13|deposits nag in place of the carefully ironic requests; bold deposits sleep blithely according to th +3146|Supplier#000003146|p,rZD0fzD SgSXkd3GkW2ZjQMSu4fCq|7|17-451-509-3636|8107.21| regular pinto beans-- carefully special acc +3147|Supplier#000003147|jIhW4pvcCF4K41rhP|23|33-399-127-2845|9954.71|sleep fluffily after the furi +3148|Supplier#000003148|CpCJWI4PHeiwYuq0|13|23-767-770-9172|7913.40|ove the quickly final packages boost fluffily among the furiously final platelets. carefully s +3149|Supplier#000003149|axCWHGuM6s HGRQmQ5dSfXZRJuB,aj|14|24-145-488-1805|8588.37|y regular packages. express accounts are! furiously ironic accounts use a +3150|Supplier#000003150|XwSjsmzEnANK,wAQUp4Xf5xJDqR |21|31-682-234-5255|837.85|y ironic packages solve quickly. bold, silent deposits sleep furiousl +3151|Supplier#000003151|I8hALO14Sa|17|27-536-600-8384|-286.98|thes cajole above the ironic, regular accounts. regular, regular instructions use +3152|Supplier#000003152|d9ZAEF3wyNHGtg2RISyMYuA|2|12-108-684-2764|3616.43|p blithely according to the even deposits-- ironic deposits wake +3153|Supplier#000003153|zZjHS,4cNlNAK1KFaFTNpYh9Y5Ceb|6|16-690-660-2374|4310.54|final dependencies are carefully final, final dinos +3154|Supplier#000003154|SHkGms7d9bD|0|10-209-166-3429|-730.29|are carefully along the slyly ironic excuses. slyly regular deposits boost. pending, +3155|Supplier#000003155|7PMPnR7F5AIZLTL8Uv6b3PexnNH4UncHa1dJ|5|15-598-914-5008|-30.24|to the blithely regular accounts sleep blithely unus +3156|Supplier#000003156|alRWaW4FTFERMM4vf2rHKIKE|4|14-843-946-7775|6751.81| are furiously. final theodolites affix slyly bold deposits. even packages haggle idly slyly specia +3157|Supplier#000003157|Iu8Z1STN zdULK|5|15-865-233-6191|-231.69| blithely. carefully silent braids are carefully requests. +3158|Supplier#000003158|TZHzRtdMUpxH2Fh2KRE|19|29-586-416-4420|8930.49|c accounts. dolphins snooze c +3159|Supplier#000003159|T2Uh9SfwrU3C MAJb75Y3nmx94AXRkX CuE|13|23-991-368-6362|8822.17|esias. requests wake sometimes silent deposits. ideas above the requests sleep slyly slyl +3160|Supplier#000003160|cscNqZ6YwOTBJfj1tvo|15|25-481-965-2571|3341.41|s theodolites affix furiously regular accounts. slyly e +3161|Supplier#000003161|cQScZDuLpBSAU9RlV4OxyvJeHbMpjfwoX|3|13-199-349-7949|2120.49|slyly regular instructions nag doggedly carefully even d +3162|Supplier#000003162|ELupM21SsqcCJOgwvOl0V9j5uulbE13R|0|10-309-213-4474|7994.57|fluffily final pinto beans. regular requests haggle carefully around the regularly si +3163|Supplier#000003163|1a12bXvGYZs3gFSYsRtTJ7I6CZd5OhHNjvY|8|18-380-830-5088|8752.64|ily blithely pending pearls. furiously ironic asymptotes detec +3164|Supplier#000003164|R3AZwhqjRhH|15|25-326-242-8592|663.40|ithely above the final requests. slyly final deposits nag slyly up the carefully final pinto beans +3165|Supplier#000003165|9yUOZLzN9KpH7XOcYn8MPW4beuJQXZQ2X0rrw9oj|6|16-489-897-9202|4750.32|final packages wake furiously among the bo +3166|Supplier#000003166|xdb6MfO7FcPZtB9|10|20-417-589-8306|9623.30|uests cajole slyly around the final, express foxes. finall +3167|Supplier#000003167|G6SS7LfQCuN1VfJ|22|32-707-517-3049|3958.63|ar, fluffy theodolites. quiet packages sleep carefully express pla +3168|Supplier#000003168|SG04rsEVfVZ|20|30-164-190-7475|2729.13|ckly alongside of the blithely unusual foxes. slyly unusual requests alongside of the slyly u +3169|Supplier#000003169|ODz6ABJGTs8Qnz|3|13-103-649-1606|7513.16|cies haggle. furiously iron +3170|Supplier#000003170|JmDdE,nzwVQiIEV2eEQuUCwrpkn8k|16|26-316-403-9859|2705.16|olites boost quietly along the slyly regular accounts. slyly special pi +3171|Supplier#000003171|hSq1EeyBQOBMzq1tJLDABbRjOQasUwcxsk|4|14-477-430-4257|659.03|pending packages maintain abov +3172|Supplier#000003172|8ngbGS7BQoTDmJyMa5WV9XbaM31u5FAayd2vT3|10|20-834-374-7746|8175.17|ss deposits use furiously after the quickly final sentiments. fluffily ruthless ideas believe regu +3173|Supplier#000003173|ydl44utgudl6CP46TF7kliIcF5sC8K9,WH,Tj|15|25-995-176-6622|8084.62| about the furiously pending ideas. quickly final accounts wake according to the furiously ironic +3174|Supplier#000003174|d,vGX8Det3nd7HLCO4pg|1|11-951-431-9601|4217.05|requests nag furiously across the quickly unusual pinto beans. slyly regular +3175|Supplier#000003175|vVyQZdtFzvoZpyeRC7jIzKKRSlXHO9kXivDYSrCN|8|18-179-171-6669|944.43|packages-- furiously even packages are slyly alongside of the special, fin +3176|Supplier#000003176|YUCh3B5y69ydExisZud1X|9|19-755-792-6352|9007.95|n pinto beans. furiously final requests sleep according to the regular deposits? slyly ironic +3177|Supplier#000003177|rLL4NUAWv0uppq1AhG3W7Zip2UxN|2|12-139-350-4726|1721.64|accounts. even, regular packages against the fl +3178|Supplier#000003178|Ts7owgQJctnQwuVpsq0ztFiw1uA|15|25-491-424-2023|3515.11|ions. ironic foxes haggle slyly along the furi +3179|Supplier#000003179|SsOOjSkuc957RKFt|22|32-902-657-1927|6492.42|ackages sleep slyly among the ironic excuses. deposits a +3180|Supplier#000003180|g2Iz,zOC7UWqfB0ARyrVQvDtnQ0WQ|0|10-269-505-6788|7124.10|equests haggle quickly ironic dolphins. final theod +3181|Supplier#000003181|BH8A77TNeEg4FiHlt1A|14|24-449-268-8264|8730.81|s deposits. special ideas nag blithely across the packages. ideas sleep spe +3182|Supplier#000003182|UCmH8hT3ESeCHU,2aIN8YxzLWvhE2c|22|32-232-125-5764|7759.09|nal, silent asymptotes. pinto beans across the final, final decoys mold +3183|Supplier#000003183|2b9GLvnIQhUGJsKvw5DxiGiZAibxq9H|22|32-959-611-8531|53.30|y at the slyly even accounts. express frays according +3184|Supplier#000003184|QePnLXsvlZR9BRYx,ytc56igzNcS,8lFUT|8|18-163-694-8341|2670.41|its. final foxes nag carefully above the regular, bold theodolites. regu +3185|Supplier#000003185|hMa535Cbf2mj1Nw4OWOKWVrsK0VdDkJURrdjSIJe|3|13-979-274-8468|-475.90|quickly final pinto beans wake carefu +3186|Supplier#000003186|psT4cqWz2mMi4EV OpBRFv|4|14-984-821-6231|7403.68| even packages. pending deposits sleep slyly final dependencies. express accounts boost carefully +3187|Supplier#000003187|letUiZcqEJT61wXYkP2XMjDW7yNqmXXO9l|15|25-418-344-7176|-314.43|ress requests breach slyly pending deposits. regular theodolites +3188|Supplier#000003188|JPdkZildVJC3d3TazH|8|18-993-415-7718|4351.56| use furiously packages. slyly ironic platelets wake furiously. express +3189|Supplier#000003189|DWdPxt7 RnkZv6VOByR0em|3|13-609-731-5577|6649.39|gle express instructions. regular deposits b +3190|Supplier#000003190|uKuj uR2JS0HGJsmDHiCrOj6vFLxVMq|5|15-538-280-5008|5359.64| regular dolphins above the accounts p +3191|Supplier#000003191|7A3COSS5tlXMcMa3jajZmQfQp|15|25-886-362-9931|9474.45|sly final requests nag above the fluffily even sauternes. deposits +3192|Supplier#000003192|jcnowqdrNtFws7AtZ2g6USqYv|15|25-932-338-8494|-897.76|lphins. evenly regular pinto beans boost blithely. carefully even ideas wake blithely furiously +3193|Supplier#000003193|XCBIUnI,URnww9TWChS8y27QGm5Eb8UxT|6|16-105-465-1817|1208.87|lyly final pinto beans. unusual requests boost pending deposits. slyly unusual pint +3194|Supplier#000003194|3q0FFKqQgY2C,|1|11-240-988-6083|9494.43|eep after the slyly final packages. quickly unusual requ +3195|Supplier#000003195|y4TyUvKINQFteapCIasc|8|18-418-715-2116|4995.18|ckly. quickly express instructions wake slyly deposits. fluffily unusual acc +3196|Supplier#000003196|u4T6CeKsJZD,MXn|2|12-420-873-2251|-56.59|ng accounts boost carefully unusual instr +3197|Supplier#000003197|xN7urTEvf3bME1qOMJ,CRmm|5|15-450-466-2595|3806.57|ent packages are accounts. unusual, final instructions cajole carefully. sheaves sleep +3198|Supplier#000003198|Y9NXI9djnQGwvPUfHVZ7Ww|2|12-121-806-4870|9973.98|nt asymptotes. blithely bold dependencies lose blithely. regular dependencies wake furio +3199|Supplier#000003199|dCLmojN3T0Nm9F|18|28-745-416-7332|1711.99|final packages. carefully pending theodolites haggle accounts. carefully bold foxe +3200|Supplier#000003200|cjSOWJGf,LfuxKLRla,fqQHPje |6|16-996-577-4033|6375.07|y unusual pinto beans try to cajole carefully along the busily ironic excuses. carefully unusual exc +3201|Supplier#000003201|E87yws6I,t0qNs4QW7UzExKiJnJDZWue|3|13-600-413-7165|3799.41|ely final asymptotes. express realms sleep. packages cajole; carefully sil +3202|Supplier#000003202|pK duW8oGZXScgMkf|11|21-751-618-8676|-409.79|y unusual theodolites. blithely final depos +3203|Supplier#000003203|rEvaJTcRX5KbGYImWjnva82NySVDc1oYPdu1xu|10|20-727-365-6152|7526.18|s cajole carefully slyly final platelets. fluffily bold dependencies sleep fluffily over the slyl +3204|Supplier#000003204|yLDdPfluaEjCpzYhBsrr2H1O4|5|15-939-676-7293|1584.84|unts boost. requests wake bold foxes. blithely ironic dependencies affix ironic requests. final +3205|Supplier#000003205|viUNFpv yrsjJsjG2M3sbt|20|30-793-761-8003|3403.13|unts cajole even packages. slyly regular pinto beans boost boldly regular accoun +3206|Supplier#000003206|wkE52vw4Bo3sIE|4|14-690-834-9683|5440.73|wake instructions. quickly special deposits across the foxes unwind special foxes. accounts slee +3207|Supplier#000003207|leSBkuggLEE8kYZS4EN|24|34-820-589-9927|8391.61|nst the packages. ideas integrate steal +3208|Supplier#000003208|tgLoFm3LDaDlVyjbw36QmiQ19lrwg02ABOJE|8|18-530-199-2138|4907.39|ke blithely against the slyly silent accounts. regular pinto beans boost fluffily carefully +3209|Supplier#000003209|qQYwsLNgelc0XrkX|1|11-553-898-5474|9560.50| regular foxes. even instructions haggle after the never final deposits. ideas cajole carefully +3210|Supplier#000003210|xG001SAxNEYwsjj|1|11-430-492-3540|4437.40|he regular requests. special, regular accounts sleep fluffily about the bold, ironic requests. +3211|Supplier#000003211|4TnN3VB1sxFafF6FyOPYQWfbRa7D54Tk|24|34-152-188-4052|8315.05|ets impress fluffily. quickly final requests sleep above the carefully +3212|Supplier#000003212|18WL8,9NRCSdrIzvHj6JSo4uV|20|30-148-178-8673|5132.32| are blithely silent deposits. slyly final requests at the blithely regular packages are carefully +3213|Supplier#000003213|pxrRP4irQ1VoyfQ,dTf3|3|13-119-676-7364|5978.55|k accounts. carefully ironic packages affix blithely quickly bold accounts. blithely regular sheave +3214|Supplier#000003214|WuxgYli,8Z,4BzFqgB EyFrV9MnX|24|34-702-523-7567|5550.15|fully pending requests promise enticin +3215|Supplier#000003215|pQWi1BS,ujQ0vmMg0iF7bRYtLzvqsdkQ|2|12-165-251-2610|-562.38|tructions nag blithely alongside of the blithely even ideas: sly +3216|Supplier#000003216|AsauQYym94o6O,O0Le4SGYuYSJmleOxHZ9U50|17|27-210-133-9765|6045.55|ess theodolites dazzle. blithely bold deposits haggle rut +3217|Supplier#000003217|B UeP1vTLP Do1h A2H2ajkrWrq|8|18-769-187-1802|9539.80|er quiet requests. ironic ideas sleep blithely. sl +3218|Supplier#000003218| VGNS7OvFvDIz4h73xZ3Q OomD19gce|5|15-395-281-9120|8544.61|deas dazzle fluffily slow pinto beans. blithely special packages play near th +3219|Supplier#000003219|x3aarf0um8GIbx4bY4bBJur2CQevZJ5qHWr|24|34-455-317-8245|2346.09|slyly even accounts above the +3220|Supplier#000003220|w,xfoEnCCmNSIjYYAjO4JXhuYSewWqbZqd|20|30-493-872-2931|4670.70|ounts. unusual, stealthy packages cajole deposits. slyly regular pinto beans affix final a +3221|Supplier#000003221|GihgWugZHILgmhYn 6Im0XNnuIr|5|15-847-639-4403|2474.43|deposits sleep into the realms! carefully unusual theodolites across the carefully ir +3222|Supplier#000003222|MdTV1w, 0dU3uNi,geP3Wkce7ut87wSKMZLIQeVQ|12|22-574-453-1097|5685.44|yly unusual requests hinder furiously fluffy requests. quick +3223|Supplier#000003223|j07Sdrjrg3yn5NeLX v24zbwNLj TwryY6C6F|2|12-573-248-8700|7163.96|. slyly ironic foxes boost slyly bold requests. sly +3224|Supplier#000003224|uehMf22Gyj10xiRpK8oT|8|18-540-759-6437|7733.00|uses use. carefully regular accounts nag blithely slyly special gifts. +3225|Supplier#000003225|WgQVQZtHUHCtMcz|23|33-734-505-3559|1582.40|to the carefully pending deposits. ideas use slyly +3226|Supplier#000003226|LU3K lPME2EQwpEkLzZF4Ezvi|15|25-339-545-4349|6424.17|onic, even requests haggle above the fu +3227|Supplier#000003227|hipxZ2D9SO1|15|25-454-472-2507|4135.10|. close foxes play pending, dogged +3228|Supplier#000003228|KDdpNKN3cWu7ZSrbdqp7AfSLxx,qWB|19|29-325-784-8187|9612.94|warhorses. quickly even deposits sublate daringly ironic instructions. slyly blithe t +3229|Supplier#000003229|lhFPfZZF8ec|17|27-868-150-6529|5771.49|l ideas. slow, pending deposits sleep blithely. fluffily final frays cajol +3230|Supplier#000003230|2KpgVsFdmEBL2u8fpAB9z24BOAoRJgAWJb,AHth|9|19-472-365-6787|1444.98|oze idly across the slyly even theodolites. final, final instructions hag +3231|Supplier#000003231|sOT0c7g85Y,kXUdk2ChGGCfytw6VObsi,uH|6|16-297-460-6360|6729.23|ording to the instructions. carefully express foxes sleep slyly throughout +3232|Supplier#000003232|yw4Qp7KSN,zjNa8OpEw8XFMoQvp0sYE|22|32-582-798-7274|5187.06| instructions are among the thin deposits. busily ruthless theodolites haggle blithely. furiously st +3233|Supplier#000003233| aVFqnCFgVXU3CJUnzNj3vlC|11|21-915-960-5343|2007.29|ilent excuses are above the regular, ironic packages +3234|Supplier#000003234|1916uLszlZz|20|30-583-104-8386|8525.33|eodolites? furiously regular foxes accordin +3235|Supplier#000003235|j,YQNiTREV1PQa6GTFU8SK4IwkkOUsc8qj|17|27-106-868-3294|8265.38|odolites about the regular, ironic deposits lose furiously furiously bold +3236|Supplier#000003236|MjHtjcaSuqm8jC|16|26-307-329-5100|6341.36|cross the furiously unusual requests. asymptotes wake bl +3237|Supplier#000003237|c7Bh3swAK12mAySTlXXq|2|12-714-462-8120|9207.82| at the evenly final ideas wake furiously even requests-- regular deposit +3238|Supplier#000003238|EZAif4tzOrTSxz,|5|15-394-335-2341|5846.94|efully busy platelets are carefully. furiously special +3239|Supplier#000003239|RRSmqHhyBHJegOW867GgvVlkE4MJ6tz6jzi6PqIO|23|33-288-804-3846|4593.60| slyly. unusual, ironic theodolites sleep slyly across +3240|Supplier#000003240|zj dgjF7UqakMae w|24|34-632-162-6473|4636.91|aves. furiously pending escapades use +3241|Supplier#000003241|j06SU,LS9O3mwjAMOViANeIhb|3|13-852-490-4891|6631.69|sits at the attainments wake slyly blithely expr +3242|Supplier#000003242|wywbFKk3ePvF|11|21-915-898-8965|4916.75|e. slyly ironic requests haggle quickly along the slyly iron +3243|Supplier#000003243|2HMGu Uu5VPguQQyK|7|17-224-347-1366|1897.76|instructions wake above the furiously regular depths. blithely ironic deposits sl +3244|Supplier#000003244|F83AVb,jqnseMRZrl4UwlyixFKcPac|8|18-830-871-9838|4599.04|cial, even dependencies use carefully final, express somas. reques +3245|Supplier#000003245|Dh8Ikg39onrbOL4DyTfGw8a9oKUX3d9Y|22|32-836-132-8872|8042.09|osits. packages cajole slyly. furiously regular deposits cajole slyly. q +3246|Supplier#000003246|3YNoAfDXHi|14|24-150-403-2751|4427.79|en ideas was furiously aga +3247|Supplier#000003247|3,W9b7YACuNPhwKEkWD5Yv|4|14-946-768-1500|947.76|ounts are blithely against the quickly regular a +3248|Supplier#000003248|MgrlLKjfFaVF3qqeoSUL ATYGRCf|20|30-474-806-7433|9954.35|ironic packages. quickly regular pearls are quickly slyly ironic theodolites. foxes a +3249|Supplier#000003249|gCvoob6Ul8x025IjOki458EkDbA5X4yNyt,D|8|18-931-461-8124|1891.75|usual deposits. carefully bold deposits use above the furiously bold packages. final excuses accord +3250|Supplier#000003250|6 DzosfrMQNELD6NjF0VczLB3qDB|24|34-460-919-7161|2624.95|ake blithely blithe, even de +3251|Supplier#000003251|8cOTBnlPaeN jQVTWvC4jQyOwBgp0|4|14-770-135-4689|5569.32|l ideas. furiously bold deposits integrate slyly among the always special dolphins. busily re +3252|Supplier#000003252|KdsnOrkoNgBtpguXwEe|17|27-403-707-5612|2549.61|. quickly regular theodolites use blithely ironic, regular ideas. foxes affi +3253|Supplier#000003253|hTuJ0zR7KJWY4SonYtUChJpF iV39W|2|12-874-818-9325|4307.36|en sentiments integrate closely along the unusual pa +3254|Supplier#000003254|4cyxoaqape1fnXMNNCUyuE51|6|16-339-926-9564|798.83|endencies alongside of the carefully final instructions dazzle against the carefully ironic reques +3255|Supplier#000003255|4TB2q kbKyVY3|22|32-693-382-9279|3859.72|nstructions along the carefully final deposits doubt qui +3256|Supplier#000003256|0G6XBim3XM8Pfjh510|22|32-121-427-6574|1383.42|fully express deposits detect slyly special instruct +3257|Supplier#000003257|1eohH908K748ruDTuvA|18|28-754-780-3793|6021.39|counts nag furiously of the special, ironic account +3258|Supplier#000003258|dbAoQREuaauQXS5KuU8DpkO|19|29-283-164-4334|-553.98|y final courts detect furiously requests. packages kindle close +3259|Supplier#000003259|AIEQ9swZsDtAQyho3YAjeQY3kplxytz|5|15-416-906-4462|4131.06|usly silent platelets. final requests solv +3260|Supplier#000003260|31DM3ZWB3t6fdBu9HQt30LmZgeHj8Cxt0G|15|25-119-892-8214|7749.01| ironic packages about the +3261|Supplier#000003261|KBzIrdRfxoDWv83w8xUHk ZGP,DMW8r7|24|34-823-103-9370|6618.32|ld ideas nag slyly requests. final deposits use daringly against the even asymptotes. slyly +3262|Supplier#000003262|qsGKC6ZKUpd8o c7xeX tVl84RqC|20|30-578-993-5633|131.22|egrate enticingly fluffil +3263|Supplier#000003263|K,WJzmirdWB5JT2uYVeYCb tpIvz8zdCpx|14|24-174-166-6188|3291.80|tructions boost by the carefully final requests. furiously +3264|Supplier#000003264|vn,zd7zHlBDeSAbSZva4MyN|14|24-396-825-8756|4245.69|ackages. carefully bold deposits wake among the +3265|Supplier#000003265|qypLU3dq2lpOkKGoY00pkmNWiK10zip06sAL|15|25-314-489-3975|9300.39|se furiously ironic waters. slowly f +3266|Supplier#000003266|fLRCV7DPrTLRgpiHpopD|20|30-718-666-3194|-280.34|ajole express, bold deposits. silent deposits wake +3267|Supplier#000003267|NTu9QeXged4l|20|30-456-842-5259|5321.12|y fluffily even deposits. regular, ironic requests are about the pending asymptotes. careful +3268|Supplier#000003268|wcRKYUyc1wQxr5q|18|28-271-882-9544|140.04|fluffily. bold instructions thrash +3269|Supplier#000003269|W1y8yHKAwJD boL|22|32-572-597-9900|8515.46|. final hockey players x-ray permanent packages. blithely even instru +3270|Supplier#000003270|7j4n5FnNEHVJxFhiyz|10|20-450-599-9591|7467.63|regular, even instructions boost deposits +3271|Supplier#000003271|Is8nCASBGG DaH8ukws6dstGC2Do06gJR2|14|24-212-898-2608|7542.29|l asymptotes. thin Tiresias under the express asymptotes integra +3272|Supplier#000003272|7umyW7SXVUO2H4D0JhaCKb4OW2rAfvJzqH|12|22-100-298-3405|8237.36|regular packages haggle ironic, express epitaphs. final accounts +3273|Supplier#000003273|pwb5G1n gtm0nXESwoY|23|33-902-451-5890|8292.61|le carefully final excuses. carefully ex +3274|Supplier#000003274|SH5h0up3L7Auhcs6YO1r|24|34-651-499-7523|1921.58|leep along the enticingly regular accounts. carefully regular foxes are +3275|Supplier#000003275|9xO4nyJ2QJcX6vGf|3|13-124-425-5571|8878.31|latelets are carefully quickly regular foxes. final instruc +3276|Supplier#000003276|dIPRD9Z7blDleqsnNfGF|19|29-181-839-9372|771.33|sly final pinto beans nag furiously pinto be +3277|Supplier#000003277|zW97hQ0M0JqUf7b3GDvsHkYMS|21|31-894-441-8862|3606.80|er the blithely express packages affix slyly special deposits. quickly regular deposits u +3278|Supplier#000003278|e2IO3KGtSZl18kn2rh6BNMe9U7LL7CW7CjCj|0|10-260-130-9749|7471.12|sly final requests. silent, fluffy ideas haggle carefully. furio +3279|Supplier#000003279|ewVQdElxm9 |18|28-760-651-8330|206.21|ncies. fluffily ironic excuses haggle furi +3280|Supplier#000003280|TtNwejP, 4GKXNfky9Jc,8gaGEI|6|16-794-822-3737|9747.49|ests. express, final requests haggle furiously among the blithely regular r +3281|Supplier#000003281|2TIoq48kSgsJd6J2jrZx|13|23-724-623-9673|1216.06| the ironic, regular theodolites. blithely final instructions to the even id +3282|Supplier#000003282|ZYRZsEp8l,kp3h6LsFiDk|12|22-476-692-7508|9249.17|lar pinto beans. furiously regular packages use quickly. packages +3283|Supplier#000003283|EZhC w5my7GyXlcW|20|30-524-916-1991|7940.60| beans boost furiously. even, bol +3284|Supplier#000003284|d8 h32eyBU6yi Rk2SZOfRYB06rG|12|22-125-894-4783|3670.30|s. pending, ironic deposits wake along the fu +3285|Supplier#000003285|rDe8Y 4pVW 1YdzBmyk|14|24-723-784-4153|7760.44|ages. deposits among the quickly even ideas detect against the carefully bold asymptotes. regu +3286|Supplier#000003286|9buhio7XaFz,MLuRTzosywx|14|24-567-682-2692|757.74|its affix blithely about the carefully brave excuses. special accounts +3287|Supplier#000003287|zTGbtB bMCt5Rl D6fJ rVj0F2ogI|9|19-416-117-4411|6293.03|ts are. carefully pending requests wake blithely si +3288|Supplier#000003288|EDdfNt7E5Uc,xLTupoIgYL4yY7ujh,|3|13-442-617-3554|-79.72|nt deposits. special courts serve f +3289|Supplier#000003289|ukH4T22Oiot6nnLC9vO68DgUV7,m|7|17-641-363-9867|150.75|final requests. special pinto +3290|Supplier#000003290|0oqc9hPgyqdcaJrcs7RRy0WrYur8A|4|14-168-862-2247|1563.88|as are. instructions wake slyly blithely pending ideas. bold, bold courts alongside +3291|Supplier#000003291|Gn1M6NCIuOOxEu6MuBGqj9o6LZWLJq17hQJpP8|22|32-105-285-2618|3111.32|lithely. carefully special accounts cajole above the furiously final +3292|Supplier#000003292|rdZxuvAOnGA5TGTgAZlaor Ah7pd1xqVUz4V7nYL|0|10-298-132-2236|6974.88|ng the blithely even packages. fluffily final foxes wake slyly despite the ir +3293|Supplier#000003293|filufiVzCBVot7vAwPRvCimnQ|0|10-984-560-5066|8801.70|rding to the blithely ironic packages. slyly even instructions wake carefully about the i +3294|Supplier#000003294|7T1AkctPf8|20|30-946-993-6591|254.42|ly final deposits. fluffily blithe foxes among the express ide +3295|Supplier#000003295|GDirhnhOVlplgJ4lrt3f50L l1a|15|25-313-798-3150|5952.52|le after the blithely special instructions. s +3296|Supplier#000003296|0c318ax2Hbuqd9qDJwxDVmMpTM9RLSFeXsXlLHck|0|10-367-244-8150|8022.21| blithely even foxes sleep according to the furiously even ideas. express requests above the bl +3297|Supplier#000003297|hVHrxF7TJM26wjdVbchXI an6TGDpIGlnbC,|17|27-763-965-5848|258.72|s use slyly. special requests are furious +3298|Supplier#000003298|gW0OcqV3TD7|23|33-364-207-9726|1252.59|ogs. ironic accounts cajole a +3299|Supplier#000003299|ZH jMEKsbbDbxEGl8M8 l6,tVtEmA|24|34-125-967-8542|7292.91|nic dependencies are furiously. carefully silent theodolites are quickly regula +3300|Supplier#000003300|ZbcbYHYoI5fRIRieYU1vUzVq|13|23-941-280-4033|8025.60|ly express asymptotes cajole quickly among the slyly even theodolites. accounts aff +3301|Supplier#000003301|avceT9G9MnYQuzH,VCT|4|14-412-668-5330|2299.15|e. thin foxes cajole furiously. special, unusual courts after the furiously fin +3302|Supplier#000003302|6XW,uY7xlDDOJIFJxRuf|11|21-966-112-8188|7827.03| fluffily bold, express pinto beans. express packages a +3303|Supplier#000003303|0E3ihSr0VseYg85LJVsY2Va3Qt9bAjz3ui7,C|11|21-713-480-9747|8959.53|ermanent instructions cajole according to the blithely unusual +3304|Supplier#000003304|wLN88ULs9Y|3|13-918-136-8872|1887.14|onic theodolites according to the regular requests detect slyly even packages +3305|Supplier#000003305|GLZJimfuzKoQcqcv4|21|31-676-713-8551|2191.25|lessly. quiet theodolites poach furiously accounts. final pinto beans are slyly among the qui +3306|Supplier#000003306|Vr,BC0CRsL7hW8E4j74Cicu|9|19-270-601-1021|9270.32|refully even pinto beans affix quick accounts. quickly regular platelets sleep to the carefully re +3307|Supplier#000003307|ij6rKFRJjQGU|13|23-603-344-3362|272.08|uests at the carefully unusual dolphins hang evenly above the busily ironic +3308|Supplier#000003308|VVFuoh4ipur7tn1Y2|11|21-291-881-1400|5834.61|riously ironic accounts against the carefully unusual escapades mold furiously slyly ironic dep +3309|Supplier#000003309|zxjUE,0eR7cIyqKj88gzjSS|12|22-607-835-8562|6390.23|y ironic deposits cajole dogged, regular instructions. final instructions doubt. furiou +3310|Supplier#000003310|c7fj5A AZFI1CXmBVNHILDV|23|33-582-553-4093|5618.15|y. furiously ironic deposits are slyly fluffily ironic hockey players. slyl +3311|Supplier#000003311|u6APXuB3WjzN|1|11-456-830-2285|9302.81| slyly silent requests promise slyly blithely final requests. regular packages nag carefully expres +3312|Supplier#000003312|EJLnCuuCYgvPJ9AxZlBui4hlCowVP1M|13|23-218-781-4515|7455.92|ily unusual theodolites. express packages under the blithely ironic accoun +3313|Supplier#000003313|El2I7we,049SPrvomUm4hZwJoOhZkvLxLJXgVH|3|13-684-979-2091|8771.48|hins dazzle. regular packa +3314|Supplier#000003314|jnisU8MzqO4iUB3zsPcrysMw3DDUojS4q7LD|3|13-742-946-1486|9353.26|he regular excuses sleep ideas. quickly final requests mold quickly above the warhorses +3315|Supplier#000003315|5cOojypCFl lugFmPKecUfnKPEFd22|13|23-456-715-4529|8369.43|gular, unusual deposits are alongside of the unusual requests. carefully quick theod +3316|Supplier#000003316|MAiM69JS8VuMQw|0|10-601-853-6400|6003.90| ironic accounts. blithely regular accounts above the blithely +3317|Supplier#000003317|POG7Ne0O10jpXTn,Rj|19|29-150-937-6633|2545.05|ts. blithely ironic instructions wake slyly after the b +3318|Supplier#000003318|bwDVcV,myUN7B5JJR4u,YDYc|17|27-986-768-6648|4558.84| carefully quickly even pains. slyly idle packages sleep furiously along the carefull +3319|Supplier#000003319|Fmi6ygNVYw68xo IzhzyI|17|27-363-151-4668|3501.64|se packages. accounts eat alongside of the special, special accounts. theodolites was slyl +3320|Supplier#000003320|0uizitM07M8oV|14|24-479-278-4263|72.55|refully bold pinto beans sleep furiously final somas. furiously pendin +3321|Supplier#000003321|pNbmUBQtMLDGyie|1|11-342-967-5637|7271.07|kages run regular frets. carefully even forges are quickly aft +3322|Supplier#000003322|3SRzOkrMrdtUUgTwrskvtITQMDYC,YhfDn|19|29-435-293-6191|3851.66|p final requests. instruct +3323|Supplier#000003323|W9 lYcsC9FwBqk3ItL|19|29-736-951-3710|8862.24|ly pending ideas sleep about the furiously unu +3324|Supplier#000003324|1 4gzl4buhm7sHa3QabsltK8UJCZBBYHK666d2pl|10|20-620-939-1569|5759.57|x blithely. silent deposits along the regular warthogs cajole blithely about the furiously pe +3325|Supplier#000003325|rk4bFHher05P22I9BULO5EcAJkXRqFWW|2|12-308-363-5693|-889.35|equests haggle carefully unusual ideas. quickly regular deposits use according to the slyly expres +3326|Supplier#000003326|X5Q46rlBiujw2hBODJD4,GnKzr|5|15-661-672-5861|-335.17|. packages are quickly. blithely sly requests wake among the regular, even court +3327|Supplier#000003327|rSmUgQNNz6a3ZK|2|12-823-554-6874|7781.76|e carefully. ideas are furiously iro +3328|Supplier#000003328|bQ964wtB,GulcdbG9myQsy3CeK17iNw1oOMotM|11|21-682-981-4991|7135.60|fter the quickly final foxes cajole +3329|Supplier#000003329|a91m3F 96b|7|17-835-211-3138|4015.23|ts are fluffily along the slyly final sentiments. +3330|Supplier#000003330|QU sqh0dv64|14|24-462-184-8343|7899.31|ully express sentiments. blithely special frets haggle +3331|Supplier#000003331|UtCSlkdpt0yUN6CmpvwEhNSkjJNK5em|4|14-689-181-7404|2740.41|he regular, regular accounts. i +3332|Supplier#000003332|Uyep2mDFVJn68h|10|20-304-291-4650|-910.23| daringly. blithely even depos +3333|Supplier#000003333|q2aUaUcXVCmrsnC TdFY9M0 pEzC50 tXm3|24|34-727-789-3039|634.41|en sentiments cajole furiously bold deposits. slyly final platelets impress. r +3334|Supplier#000003334|zigrt6rOiwC3fZEA9XzmyZQ8TB2TmqXWKm|4|14-175-251-6018|1524.31|unts sleep carefully. packages wake. fluffily even dependencies again +3335|Supplier#000003335|oIh9tPJv9K8SYGBEuk2 DCUftNJh V|20|30-622-542-7138|8966.67|ly silent courts sleep blithely against t +3336|Supplier#000003336|pU67NsP6 D,|21|31-590-806-1948|921.31|de of the quickly final accounts. always regular dependencies haggle. bold foxes +3337|Supplier#000003337|Bulh5GkjvbvXnb98Y8Y9IdDVmQczmZWJY|15|25-127-857-9918|5452.51|he enticingly ironic ideas are slyly across +3338|Supplier#000003338|bQBVFaIiG46wcY4ebXtyYcAA EGEc|12|22-544-418-6592|8415.20|y even accounts should have to haggle silent, regular deposits; final asymp +3339|Supplier#000003339|LF3PWGqfKzWkyUa1AyAb5Z rK4ME9ONOgSGSy|0|10-892-842-9387|5308.31|y final platelets. bold ideas solve bold requests. quickly even deposits are carefully idly expre +3340|Supplier#000003340|2YpvPC0Foz4750wOE0ZPaRt|18|28-190-411-5979|4738.82|sleep quickly about the unusual asymptotes. regular asymptote +3341|Supplier#000003341|3mAXsWA,NE7TXpeedK11mmRjKT5zA5,Uny|17|27-840-479-7868|3398.99|express pinto beans across the thinly final packages cajole even, ironic theodolites. blithely fi +3342|Supplier#000003342|qraFBSkyoax9iuMkr0L7OI20gj659WD|11|21-726-487-1916|1765.55|yly pending ideas along the quickly regular reque +3343|Supplier#000003343|VKdvwICujAnw9eurKdyl21t6cL9T|7|17-808-624-7244|4579.68|e carefully across the bold, even foxes. slyly unusual pinto beans haggle carefully unusual epita +3344|Supplier#000003344|xRCYxvmsHYbyk9yaRmKYaP|4|14-552-551-7359|1609.52|lly after the quickly final packages. slyly special requests cajole furiously above the slyly expr +3345|Supplier#000003345|CIK,A8 Ka7,TOgmufRFojWkas|24|34-527-623-8157|181.70|ect unusual escapades. close packages must hang among the furiously quick gifts. fluffily ironic i +3346|Supplier#000003346|Pqgx1xdKu4In20 |4|14-672-880-9870|4707.03|riously pending ideas are: special +3347|Supplier#000003347|m CtXS2S16i|23|33-454-274-8532|8102.62|egrate with the slyly bold instructions. special foxes haggle silently among the +3348|Supplier#000003348|ngwfUSRAvm4dBe2JmJi7picb6CHK7|15|25-267-444-9017|9838.16|lar packages use blithely ironic accounts. pinto beans wake quickly final, ironic instructions +3349|Supplier#000003349|wtTK9df9kY7mQ5QUM0Xe5bHLMRLgwE|11|21-614-525-7451|7051.73|ar theodolites cajole fluffily across the pending requests. slyly final requests a +3350|Supplier#000003350|DFagiedXwjhXMyKCN8EmD7v3d I7xe8waL|11|21-151-970-6140|2460.92|ns sleep against the pinto beans. fluffily ironic accounts haggle +3351|Supplier#000003351|vb7b9r9na5slC4 MNIQS8hJcWKZga|1|11-368-852-9514|9703.05|refully regular requests. dependencies detect carefully about the pending, final d +3352|Supplier#000003352|KiW9uVGMM0Ea,JfZM1eXK5CQ783fH|19|29-521-116-9767|2163.95|e carefully bold accounts. quickly final deposits about the regular +3353|Supplier#000003353|OdzdTOTUqJ1|16|26-167-953-6420|1476.93|y bold requests use silent theodolites. furiously ironic requests cajole fluffily flu +3354|Supplier#000003354|eHE1MPMqGDRue5yyU4uaVxuowb2nUWje|16|26-686-354-5975|4665.97|ect carefully. furiously even dolphins +3355|Supplier#000003355|rf2kzjYsoyQb s3Mbqy2MN5aqf|7|17-678-538-7925|18.04|to the carefully slow accounts. furiously pending pinto beans +3356|Supplier#000003356|jtlug57,ke9cq9ECwEXA1EKTp|6|16-135-705-4908|4102.78|counts are slyly regular deposits. blithely regular acco +3357|Supplier#000003357|PcAiX5 7COE|17|27-156-818-6227|5468.94|usy requests. fluffily careful theodolites wake blithely packages. carefully final +3358|Supplier#000003358|,oj sqX2w1lc1A6HA8sze|0|10-401-233-2844|9426.91|ely furiously special excuses. furiously silent instructions alongside of the blithely even patterns +3359|Supplier#000003359|BG6e7oguX8oqqI0Ni|10|20-603-896-5829|3145.14|efully pending requests. slyly pending packages b +3360|Supplier#000003360|7iFFEaWR52NX qgRcktOcvTzn3N|10|20-298-471-7908|3275.70|quests: slyly regular requests nag slyly fluffily pending packages. even deposits +3361|Supplier#000003361|Pznn7dQUG9cRZvD7p7MBd6gcH,jan56d|9|19-468-820-7323|8608.66|ggle furiously final accounts. sl +3362|Supplier#000003362|90hlcceTz JQi|18|28-188-550-1587|855.74|t the final requests. depo +3363|Supplier#000003363|DWjb8SidviR8gs1 s0iyZPb|8|18-760-689-9212|3337.36| theodolites sleep quickly regular ideas. carefully bu +3364|Supplier#000003364|5p6HNshx2rf4EKNYzm9Y1vU3|16|26-278-540-7484|-38.30|ke carefully asymptotes: regular accounts snooze quickly regular deposits. slyly ironic sheaves +3365|Supplier#000003365|s81WO,kcBf69|2|12-257-308-3743|3155.23|ons. fluffily regular ideas nag quietly stealthy pinto beans. blithely s +3366|Supplier#000003366|iev0jGXt7AL5A9cTl7GCeO0lG8S|13|23-320-193-4544|8008.36| wake carefully pending depths. carefully regular packages cajole idly da +3367|Supplier#000003367|P76gRphGSXxNx411Sm5cOU5Ris|17|27-129-619-8835|9582.24|requests haggle carefully th +3368|Supplier#000003368|Ge9m5T5FhV|7|17-274-891-4212|6803.27|its. silent, regular warthogs cajole along the slyly bold packages. final, expre +3369|Supplier#000003369|TU1MuYs4GttdcuplDto7EJ|8|18-832-929-3971|6629.07|unusual accounts boost carefully care +3370|Supplier#000003370|I7agPlyHh,1PD60y|9|19-659-143-7063|4747.04|usly even requests cajole according to the blithely regular requests. blithely regular deposits +3371|Supplier#000003371|LXfDkZ4nPlgvooE3MO62Sbj4a BbaKplWHqJb|24|34-997-988-3905|9556.66|kages are. blithely regular requests nag blithely blithely final depos +3372|Supplier#000003372|4scC6qApOus7G3cwvHKoZW2dl,|20|30-813-135-4799|5819.74|rs sleep furiously. quietly regular +3373|Supplier#000003373|iy8VM48ynpc3N2OsBwAvhYakO2us9R1bi|3|13-951-330-3925|9457.95|furiously final pinto beans detect about the pinto beans. ironic pinto beans wake among the re +3374|Supplier#000003374|T8wXAoMSrOzT4js6Bs|20|30-429-470-7719|7838.94|x slyly furiously special foxes. pending, express deposits cajole carefully +3375|Supplier#000003375|mUAJaPfNxhm1li,YRjeB,k11T1s|17|27-989-899-8432|112.04|carefully permanent accounts detect carefully about the carefully +3376|Supplier#000003376|Bhe4BFpG0vCtr5GS1XJuhebmTG2ft7|23|33-750-995-8411|65.65| slyly special foxes cajole quickly furiously pending deposits. braids are along the s +3377|Supplier#000003377|3Pf5 6HWtLeHVAsBiQnFLFCGNWe|4|14-377-875-3954|1373.70|even instructions. quickly express packages along the +3378|Supplier#000003378|hhYP kcZ5fSOhfExZ6eGvx|24|34-784-222-6641|7782.14|instructions wake above the special accounts. carefully final escapades are. ca +3379|Supplier#000003379|lITn8oL3ITnsiQ0rq|24|34-439-933-5745|2533.73|uriously silent pains. express, ironic deposits cajole carefully regular epitaph +3380|Supplier#000003380|jPv0V,pszouuFT3YsAqlP,kxT3u,gTFiEbRt,x|3|13-827-850-8102|1340.88|nts of the unusual foxes cajole furiously packages. foxes detec +3381|Supplier#000003381|LjhuSMgpjJgc,eCYaEPg5QSCLwkj|11|21-488-211-3732|6050.65|latelets cajole blithely +3382|Supplier#000003382|p6rzU12Gx5FssGd4KAWbklBNdcgALy20W|17|27-904-659-7815|7351.11|p blithely unusual theodolites. quickly regu +3383|Supplier#000003383|XnDy9Rt oA0mmgnr6Fr6|13|23-114-269-2123|7101.25|leep among the fluffily unusual deposits. carefully regular deposits haggle against the ironic, +3384|Supplier#000003384|o,Z3v4POifevE k9U1b 6J1ucX,I|6|16-494-913-5925|9739.86|s after the furiously bold packages sleep fluffily idly final requests: quickly final +3385|Supplier#000003385|5W8SM31i87a6fOfe|15|25-332-549-8527|4921.46| to the furiously ironic requests. special i +3386|Supplier#000003386|VKYkPqgKzIJNVe47gpMEGLkuFLXRt1Ft7980|24|34-136-762-9624|3982.50|nst the carefully even instructions. quickly pending theodolites impress against the pending h +3387|Supplier#000003387|vcqIp2f1VzBshqpHHDr2woIsC99Xh ku19z|17|27-627-769-6020|7846.75|quickly unusual accounts about +3388|Supplier#000003388|5jhre3tArrwL|3|13-271-553-5763|-692.81|requests. slyly unusual requests boost carefully according to the +3389|Supplier#000003389|ERhdA2izb269guC2h1xUWmvje0VVfwSkkR|7|17-121-970-4215|2815.07|beans cajole slyly after the express asymptotes. slyly u +3390|Supplier#000003390|rc9 ogNpFWzWopXXsYxKNJUg,F7FDK250duA7Bf|19|29-687-741-1080|-132.77|lyly pending packages wake furiously after the bl +3391|Supplier#000003391|IgxdCABDpfznCQkvclQ|4|14-118-579-8039|-612.04|above the carefully regular decoys. furiously unus +3392|Supplier#000003392|pvXJ 0zkf LoLsO|18|28-832-668-8681|7410.49|cording to the theodolites cajole quickly +3393|Supplier#000003393|rCuPMo62kci|16|26-454-982-8925|4133.35| furiously even excuses. slyly e +3394|Supplier#000003394|R6D7n3WrQjWNGSQTb7eN ,X0oCMkhyuTHBOSPw|21|31-217-174-4535|2255.03|y slyly special accounts. even, +3395|Supplier#000003395|r2jNecotAVVDT53pqXsZMkPXyZgRghBBuqvk|23|33-715-222-1590|3271.35|es sleep quickly among the furiously regular accounts. ironic accounts sleep carefully against the +3396|Supplier#000003396|u9kOR7f0Lt|2|12-547-429-9434|1518.21|kages are. slyly ironic ideas cajole +3397|Supplier#000003397|mv7qD1qS yXXdi,7KYd|19|29-458-392-9564|5452.27|ely across the blithely regular a +3398|Supplier#000003398| kLslmv lN5GwxZJCebi23v|10|20-712-803-6493|4233.38|en deposits-- regularly final dependencies haggle slyly. slyly even asymptotes wa +3399|Supplier#000003399| OHQGA9o9znoV OfzgvyjThOZQ|16|26-969-246-9988|379.66|ic instructions. never unusu +3400|Supplier#000003400|IZSzKpRL1RNar39LvF|19|29-295-531-2833|9925.04|unts along the ironic accounts must have to haggle carefully +3401|Supplier#000003401|8fs zz37ORt9P1DwItA|15|25-299-425-9900|4167.25| deposits integrate against the even requests. regular, furious pac +3402|Supplier#000003402|t4ZC5Wz1t3srX4WG87pfcINwblWPzgu|0|10-340-627-3391|-88.58|the furiously stealthy theodolites. care +3403|Supplier#000003403|e3X2o ,KCG9tsHji8A XXCxiF2hZWBw|3|13-557-725-6973|5212.44|ely fluffily even requests. even pinto beans snooze atop the +3404|Supplier#000003404|4kPSfI3,ab3NzQMuhE7D|1|11-874-231-3852|-505.04|ways. regular packages are carefully along the carefully pen +3405|Supplier#000003405|3VldvgzWODcfArt6ePfRmNuqOBlRaKl8u3a8k|16|26-717-207-9979|1791.85|lites engage slyly regular excuses. unusual, regular dependencies sleep carefully about t +3406|Supplier#000003406|t1HVIA1MZe,RjX4fxa3VEqr,ktEITslO|15|25-275-145-8212|6654.98|y bold deposits. blithely final foxes sl +3407|Supplier#000003407|iBXKCWbBiwBJ1FORzNGxqTvFNvolqRexgAYA|4|14-504-247-1795|321.01| express theodolites doze stealthily furiously unusual +3408|Supplier#000003408|C0HQ2j4iTwsmzgss6mOZkt|4|14-286-990-6203|3090.61|leep final, ironic theodolites. foxes slee +3409|Supplier#000003409|GA8Y0SYqqRSyFvkZX t7v63jDh6rj|4|14-384-470-7827|-101.27|ily ironic packages. furiously silent instructions detect quickly. quickly expr +3410|Supplier#000003410|b52Zsy0pkB2HpOPgFUuBqr5TX|8|18-862-727-2661|902.36| blithely regular accounts nag carefully slyly express pa +3411|Supplier#000003411|ZMgZGp KDGuSXzo5j7595iv,t,l1L|5|15-333-410-6946|5960.09|accounts snooze at the thinly final deposits. thin realm +3412|Supplier#000003412|BsM,M8v9iyWLbUbLCo0m6oI1,GdiOp|7|17-941-232-6438|3275.80|ilent deposits sleep quickly quickly pending requests. furiously ironic requests wake careful +3413|Supplier#000003413|TC36OF8sxW71zzaDCR0|17|27-431-356-5598|-578.87|nusual platelets haggle after the express, +3414|Supplier#000003414|loSCvoLk0uUDFaCvql3yt5tHp6P1a|20|30-252-605-4415|3771.33|y. blithely express instructions hang quickly ironic sentiments. f +3415|Supplier#000003415|raBmriIU,aXxubjkPuZFL1UeZDwVxT4pdcia|17|27-687-629-3383|684.69|ul instructions. silent packages across +3416|Supplier#000003416|xSA,iuriMU|7|17-550-662-1049|-464.80|s sleep above the carefully even dolphins? packages run agains +3417|Supplier#000003417|tVqlkOTDe5dtnj7CcPPJfKoSKKCp1VprhK5q7|23|33-335-458-8687|6990.37| nag according to the silent, regular dependencies. quickly i +3418|Supplier#000003418|hETn1TJqsCL1bLnHhSv|22|32-673-656-6931|5233.61|eposits shall have to nag blithely. final courts sleep quickly along the regular requests +3419|Supplier#000003419|yt KX357gL|6|16-702-659-6464|3315.60|ons sleep fluffily after the furiously ironic theodolites. quickly regular foxes are +3420|Supplier#000003420| iQI07Fr54mpdyJAXUGpY9iZoPu|12|22-220-338-3832|8331.00|es haggle fluffily regular deposits. regular deposits according to the regu +3421|Supplier#000003421|Sh3dt9W5oeofFWovnFhrg,|3|13-134-954-9976|9128.62|al accounts about the thinly ironic foxes nag slyly toward the pending pac +3422|Supplier#000003422|DJoCEapUeBXoV1iYiCcPFQvzsTv2ZI960|3|13-662-152-4869|5731.96|xes sleep blithely ironic requests. slyly express deposits boost furiou +3423|Supplier#000003423|,,hFmixRsUfI|3|13-477-872-1285|8118.27|e slyly regular dolphins. quickly regular foxes nag. requests boost fluff +3424|Supplier#000003424|F1rDXpNdhrAPD|10|20-458-852-4137|8821.43|e quickly. silent packages cajole slyly. +3425|Supplier#000003425|qpPzLbrWkeO,q1qbsBoG7vIm9aS23G7Tx H|14|24-242-694-4933|1905.71|ily express accounts boost abo +3426|Supplier#000003426|vPImX7JKqMniQEBJuAD|1|11-479-973-3507|6152.01|ccounts sleep across the fur +3427|Supplier#000003427|PwxRqbDHq 8DDYuO5ZSb9EyinZBw762In4|23|33-905-472-5059|5917.87|carefully regular requests haggle b +3428|Supplier#000003428|gV3bZITOKab4Jyhp58KP9h,O3IjFG9,|2|12-685-903-6415|8625.52|nal deposits. idle, even platelets are. blithely even pac +3429|Supplier#000003429|EAn2WPCt0Glq,y6|6|16-542-608-8183|9161.51|ic requests nag. blithely regular ideas try to wake quickly eve +3430|Supplier#000003430|QPf4NJIUnU9ViQOeQvTLygg|15|25-181-562-3060|8551.20|furiously fluffily regular courts. slyly ironic pinto beans cajole carefully after the fo +3431|Supplier#000003431|r4KMmARJ,uRFYDNdHWqz,biMQqT8vxgQjlTJE2|22|32-189-701-4058|3080.38| regular requests. furiously r +3432|Supplier#000003432|z,Xuzy2hzllkHfwFld,TVRK7vUQVDGZxtP6uG|18|28-952-605-6098|3866.76|fluffily regular accounts. accounts are furiously. blithely bold +3433|Supplier#000003433|U,Ccsi8VrSqtkZ1BnPJkpzuViKtuIVOmirXtwu|5|15-334-430-7428|9705.07| requests cajole furiously. fur +3434|Supplier#000003434|dSMKIvhnfulrIDkReyMeCu |9|19-428-178-2867|3236.16|ages. blithely regular accounts cajole among the bo +3435|Supplier#000003435|2CyGFviSdgSDlNDUjsgW1kGVqf3i4CHxcwl|13|23-184-962-1323|4079.13|ep fluffily ironic, even accounts? ironic pinto beans +3436|Supplier#000003436|pVwVAr23pEMeJ|9|19-391-495-4796|4014.62|ess accounts solve. ideas sleep enticingly regular pa +3437|Supplier#000003437|JfgZDO9fsP4ljfzsi,s7431Ld3A7zXtHfrF74|10|20-547-871-1712|8788.46|ickly unusual dependencies. carefully regular dolphins ha +3438|Supplier#000003438|Gw ag5qZU7GwJO9J92WJ0|11|21-839-918-4310|8729.59|onic instructions. ironic asymptotes wake blithely final ideas. regular, specia +3439|Supplier#000003439|qYPDgoiBGhCYxjgC|7|17-128-996-4650|9352.04| according to the carefully bold ideas +3440|Supplier#000003440|93S5pntmWhw R6U4ZD ROQ,2gI17b3|23|33-495-208-7899|9822.22|ake carefully after the fluffily re +3441|Supplier#000003441|zvFJIzS,oUuShHjpcX|3|13-842-877-1438|3194.13| ironic, quick ideas. excuses may cajole ca +3442|Supplier#000003442|LwGgz,uyiMekRtWXrQyrurTZofjPkED|0|10-492-470-4563|1688.66|refully along the platelets. requests use furiously. special deposits lose furiously. somas +3443|Supplier#000003443|XqVQLbwQ0uG|5|15-703-549-2363|-783.15|r foxes-- ideas alongside of the d +3444|Supplier#000003444|CtVJmOkNVUCgOQX,CcDkrTmb8D4SAuRv|19|29-660-228-7079|2519.12|ss the bold, regular foxes. requests wake slyly along the carefully s +3445|Supplier#000003445|gWX6w2yDE10TXisLZkLPcoFl,FN|3|13-208-253-3605|8784.98|fully ironic theodolites promise furiously final, even dugouts. unusual, +3446|Supplier#000003446|63YyLYXRdGzc|16|26-494-262-3731|5229.51|nal requests. furiously regular accounts dazzle slyly ironic dependencies. final theo +3447|Supplier#000003447|h7En559MI3o454De6Ye75|5|15-806-372-7742|322.43| cajole furiously ironic deposits. foxes wake slyly ironic dependencies. platelets cajole +3448|Supplier#000003448|eVa5XwFZOM6p8d9g6Pmn4YhbD PUnFAI|3|13-246-324-4442|4984.99|leep furiously carefully ironic accounts. +3449|Supplier#000003449|3hzACsQegR5TNCwoAxxNHyrV19eJmn|17|27-578-751-3895|4789.73|nding platelets above the slyly regular packages wake after the quickly even deposits: requests in +3450|Supplier#000003450|DLushTTMhePgb4v84GE1Zg6HB ifx6JNfR2|22|32-954-252-9016|1042.95|y quiet requests. doggedly pending deposits will have to haggle sl +3451|Supplier#000003451|UT3xQWCe4YXLlsxTt|16|26-586-567-2571|3790.51|kly silent accounts engage alongside of th +3452|Supplier#000003452|7tMycIKhE,pe4OL3Du|21|31-644-663-7633|4150.42|even accounts. slyly ironic forges about the even, final deposits are carefully express theodol +3453|Supplier#000003453|75t7xVMCAHyeyMbhk|1|11-164-380-2986|3173.49| never blithe requests. carefully special epitaphs hinder along the permanent +3454|Supplier#000003454|AE2EfmgORyk26eAgr|3|13-176-784-6973|6854.35|ously bold theodolites cajole. regularly pending packages haggle carefully +3455|Supplier#000003455|UZgM61NKyO4bSMEzmSfAta1hEKjt8P00C8pZHIUz|19|29-315-593-9537|7076.03|inal excuses cajole silently blithely even deposits. ironic, regular pinto beans play across the f +3456|Supplier#000003456|kxNIOHJ7JYRsOKqbpoL,dnVlVlbzhgAiLHLaju|16|26-308-141-9160|3899.29|thy deposits breach blithely above the slyly ironic instruct +3457|Supplier#000003457|wFhgQuzKe5qv7FyFD7l|13|23-874-357-9431|6417.95|gular requests. regular excuses boost furiously at the q +3458|Supplier#000003458|AIawyKf4tIjmY5HsGCMgcAZFmeqkoL|15|25-645-475-6688|9469.81| maintain slyly. pending, final pinto beans haggle blithely according to the ironi +3459|Supplier#000003459|nt3w6L9fHbXL7,dYQ2gWssBqN|24|34-455-818-9309|9183.92|ruthlessly ironic platelets sleep carefully deposits. slyly dogged deposits serve after the bli +3460|Supplier#000003460|0iXLHfjoBD9oySRtdK|18|28-775-198-5196|221.57|requests. express instructions boost quickly ag +3461|Supplier#000003461|M9lhQScS9qAJe,jYoBP|22|32-923-877-7912|3890.96|final requests. ironic, regular +3462|Supplier#000003462|se8groc4wD7|3|13-423-684-7692|9364.91|uffily regular packages; fu +3463|Supplier#000003463|8m0HaduJeFZSVd1GRWvJkNhv4O KSM|24|34-194-463-1760|6120.24|g the ideas was slyly unusual reques +3464|Supplier#000003464|EPcvYyNhwc5lGvLXLEGepSL EWkvsQkXJ|5|15-869-904-6882|-168.86|e final asymptotes. carefully regular pinto beans wake care +3465|Supplier#000003465|Mjb4oXmNQLa4zWXOnoQUs5hjMCqDGV Ex2oFnBi,|2|12-417-441-1370|8937.79|encies. never brave accounts about the express, silent ideas detect fluffily quickly ironic de +3466|Supplier#000003466|uiveU6ODVbwZYKjJFBXpZ7reVEr7Qx|10|20-682-993-7524|-477.87|dolites are furiously ironic packages! excus +3467|Supplier#000003467|p6DZOiOzk3cXG5cx57mElGRuIPMGD1Le 0zuwqF|7|17-726-314-1724|7346.39|ic packages run finally. blithely regular sheaves cajole quickly across the quickly even ac +3468|Supplier#000003468|uauXeECdlFyj GDUl4b9YScxUAzSWY12uVP Vw B|6|16-219-806-1236|4772.19|structions boost quickly. accounts alongside of the carefully ironic deposits sleep carefully agains +3469|Supplier#000003469|Paa wUpqFS|3|13-316-406-9378|3145.49| the instructions. furiously final packages sleep fl +3470|Supplier#000003470|5SnLW3RZ9s01wm1bzyMWKh0DemDKwGo|14|24-780-294-7324|9169.42|unusual dolphins boost. q +3471|Supplier#000003471|NTBnbcz5A 4Hqk7aqCH2lV3xiv39aD,gMf,|7|17-448-210-2434|9455.40| the regular decoys. slyly pending excuses wake quickly. furiously ironic requ +3472|Supplier#000003472|k5ubBei49egPa4KIKsPKfMNDRE3tHiK|5|15-479-819-6508|8999.66|ole quickly regular deposits. regular deposits cajole fi +3473|Supplier#000003473|NCy OGuV,0YHamgqAxkIyDA,De6P|2|12-495-553-2521|5405.70|sual theodolites. pending dependencies cajole. patterns doze after the furiously special instruct +3474|Supplier#000003474|8vRKYYAAdx1Sgb|23|33-503-251-3282|2657.50|theodolites cajole across the slyly even deposits. +3475|Supplier#000003475|WK9,iCi6p6A9,2Eu Gl|21|31-259-380-6616|2560.09|arefully. slyly final excuses wake furiously pending asymptotes. asym +3476|Supplier#000003476|KIhrrC609o0q6|13|23-224-944-3864|4427.05|pendencies. carefully unusual packages use slyly fluffily pending packages. final +3477|Supplier#000003477|t467MQUTaZg6oAtR9er2A5U|5|15-826-822-3562|-554.06|he pending instructions-- dependencies affix fluffily: bold epitaphs among the e +3478|Supplier#000003478|mRuc7TmfUuV4YK93HhggTA4lyhoQDbEEO|22|32-511-233-7238|2754.66| ironic accounts according to the blithel +3479|Supplier#000003479|OuQuffXBJFwrjPQCrqqVkj8jJA15Ex|15|25-609-427-3896|7461.28|sly. fluffily even instructions detect carefully! final theodolites haggle blithely fur +3480|Supplier#000003480|yEZR6wFaZfG|13|23-672-198-6527|-178.48|ross the quickly special sauternes. carefully ironic packages would haggle quickly. acc +3481|Supplier#000003481|6uTNGVc b,n|3|13-823-970-1031|1249.81|yly final theodolites. furiously quick packages alon +3482|Supplier#000003482|Hpd93ZPiuCbghy4qWEw3LHIyPtm0f8Zz|11|21-697-912-9129|4583.29|cept the final, final requests. thin courts against the fluffily final pinto beans cou +3483|Supplier#000003483|bfMbYXb7KtZKc8,IY6c|1|11-219-915-8886|7331.51|unts. slyly special tithes sleep; fi +3484|Supplier#000003484|3GWJxKy TGoWs4l 3G84Q Ly1|17|27-174-208-9441|7657.83|sly regular instructions! carefully bold gifts alongside of the furiously silent deposits b +3485|Supplier#000003485|XVkY,cv,02R5jwiAayQvhQ4vz 5YIQzoY4SJU|14|24-545-692-6496|3139.63|l requests. pinto beans cajole +3486|Supplier#000003486|DgPB,ASUAF|5|15-316-749-3098|9820.82|refully furious packages ca +3487|Supplier#000003487|y3sR8ixv1gf|11|21-292-258-4757|1569.81|ar accounts. evenly regular requests above the special requests hin +3488|Supplier#000003488|zwe,Xa15W,EALyO8lF5I51hKEoqMtWlb5r7|22|32-845-486-3686|3772.10| unusual instructions sleep: slyly furious dolphins run. reg +3489|Supplier#000003489|A46,Y89qj47meR4CbcMWNy|19|29-287-680-7484|2749.11|, pending somas. unusual, ruthless deposits int +3490|Supplier#000003490|QRAa6mutr7mK9pX1OKcMYEWra,w2fIBmxt8ezNx|16|26-729-522-7327|2328.80| ruthlessly blithely sly requests. slyly +3491|Supplier#000003491|6,Gli,y4iYx5YCk|8|18-845-131-6020|109.43|ackages wake according to the even packages. regular, regular foxes wake carefully doggedly fina +3492|Supplier#000003492|,dMQCQEegpPd9TPAT7X32ZmVxcZTA|10|20-472-137-1934|101.78| slyly after the fluffily pe +3493|Supplier#000003493|Vmw05lwtM79PgWdM0GX5u9i30NuLiuln|5|15-563-783-5864|3607.97|yly instructions. blithely ex +3494|Supplier#000003494|mRZ3NxGJIO|12|22-131-263-8086|5178.43|phs-- ideas wake packages: deposits believe furiously. +3495|Supplier#000003495|pAAeW9WWKOnvC|1|11-927-394-9110|-93.29|lar packages could have to wake permanently up the carefully pending requests. ironic, regular +3496|Supplier#000003496|Pi4sY9dBGlns|14|24-215-447-2845|3063.13|nding dolphins boost pending deposits. instructions +3497|Supplier#000003497|k,,DNvZ8XHvkepAky ,22QHj4MAoxhd|7|17-762-516-4410|60.50|s breach accounts. express dolphins along the quickly ironic deposits hinder furiousl +3498|Supplier#000003498|BsM RxD9QN6nJPzSJM|22|32-515-960-2533|2782.85|ously slyly silent accounts. final warthogs alongside of the regular, even instructions boos +3499|Supplier#000003499|xN5XkMvjkLHsVb LI2yPK0XRn,wx9sRRr0|16|26-671-664-6412|8404.12|re. slyly pending asymptotes serve foxes. furiously regular packages cajole across the slyly spec +3500|Supplier#000003500|9K7v1f FDj84g2s67F8jDsGF6tjBSMo4XG,,U|13|23-328-717-2189|6071.83|packages boost quickly above the exp +3501|Supplier#000003501|4HyDpK,vIAqmn|15|25-630-743-9991|243.73|mes regular, bold accounts. ironic ideas are along the furiously regular platele +3502|Supplier#000003502|BVL4eKFE95VzjtE7BU|14|24-692-401-9392|2261.90|uests affix along the carefully bold requests. furiously pending p +3503|Supplier#000003503|8cklTaUWDAJSr8xqe|14|24-389-158-9471|8760.04|kly carefully regular ideas. furiously even orbits are fluffily +3504|Supplier#000003504|FHr0MRFGEDInYaqPsqVOU24TLJCiJiR0UrVcmap|0|10-676-110-6873|4079.04| ironic ideas sleep across +3505|Supplier#000003505|,YQvvqAAlz3k11VggQ,xdsYU1yqSQ|24|34-210-263-5006|2337.68|ly. regular forges wake quickly +3506|Supplier#000003506|kJoSJKvCYigjYeOhM74tpsnkAdKto7u7jEE8B|0|10-535-118-1133|6318.14|lar accounts are furiously furiously silent instructions. carefully even deposit +3507|Supplier#000003507|AwBZJDMPNUFNj1iZhoPlXmjkJj2HCll|6|16-101-546-3862|3932.82|lyly among the slyly even p +3508|Supplier#000003508|1fKd2SgeKAmy|24|34-675-589-4443|8846.92|its sleep furiously after the even deposits. waters use ironic, pe +3509|Supplier#000003509|bfcnS,,zeGNeSfYyz8M|24|34-326-804-4747|1319.04|express theodolites. blithely final theodolites sleep carefully? final, +3510|Supplier#000003510|pR63bCNOWsykHtuD 0y7ede4P4,LJ0lJX1UwZEMJ|1|11-925-703-8843|2028.23|osits. furiously express ideas impress furiously ironic theodolites. +3511|Supplier#000003511|UmDDReXgZCgfQigAdPq1aKn5E6Xv|17|27-391-473-9631|8485.78|l packages integrate blithely ironic theodolites. silent, ironic accounts are regular, regul +3512|Supplier#000003512|SmHR5WazVt|1|11-623-505-6498|3552.83|p above the instructions. quickly final p +3513|Supplier#000003513|Rj i6dgjYQlM3bQiDcbGg Ax90RggjVqhSjwnI,n|22|32-912-584-5642|2357.54|s print fluffily. slyly unusual dependencies are. instructions sleep furiously. ironic, bold p +3514|Supplier#000003514|63uWeHPVfzygvwEivG|23|33-482-787-8079|-927.13|y unusual asymptotes boost carefully. regular deposits above +3515|Supplier#000003515|LViFC1u,dkOZ|10|20-159-105-7351|2622.87|carefully final pinto beans are c +3516|Supplier#000003516|,SQUAz8ZNT g|17|27-431-841-3625|3495.80|xpress, pending requests boost slyly. blithely quiet accounts cajole ca +3517|Supplier#000003517|t7bFb,xedL07Zyxeft BnM6VK|20|30-200-501-6000|871.25|deas cajole quickly even theodolites. slyly speci +3518|Supplier#000003518| Vmz9fkGm rRd9,IFekdsCJnWAwRikASxDykp9|17|27-302-463-9407|58.19|he ironic, ironic requests lose on +3519|Supplier#000003519|DmV0LiZ8s,x pTRq34|20|30-276-122-2717|7871.86|nt orbits use carefully pending excuses! carefully regular dinos h +3520|Supplier#000003520|EZawP6B5zHL9miN80qSKugkd,GPL2|5|15-847-816-9775|3459.79|s above the pending theodolites sleep +3521|Supplier#000003521|Jq04JXYA9uPeEhE5bLyo|20|30-940-946-7423|643.05|theodolites nod deposits! blithely bold +3522|Supplier#000003522|ODcWij9FOSN|8|18-750-795-2411|3690.83|pecial braids. furiously bold requests beside the a +3523|Supplier#000003523|QliqObPv6XqEWceH4Q3Bj|18|28-678-138-1714|766.28|e slyly unusual foxes. unusual, silent instructions cajole slyly. +3524|Supplier#000003524|vHmrFEkW Qj2CmPHObAtd|5|15-814-332-4222|6856.64|equests are alongside of the b +3525|Supplier#000003525|o7P4WyAXhI1qxY3s1Qfha0bIjWzi W el|18|28-266-188-7528|1597.03| theodolites kindle blithely ev +3526|Supplier#000003526|3y7PbMmXAzBzYc11caUNK,oy09|10|20-920-744-5029|-684.35|ecial ideas cajole blithely. furiously even requests around the carefully r +3527|Supplier#000003527|1boAwaOOJS9RZ A,WTOioYcITGq|2|12-111-867-5440|2033.53|ously express deposits. carefully even foxes are furiously. carefully express platelets are afte +3528|Supplier#000003528|qJB,nuoxi0AGuX,9Z3tzDK|3|13-249-999-5949|9486.63|ress fluffily. quickly special platelets sleep. ironic, final wa +3529|Supplier#000003529|OoHSlmvwEkuM|0|10-304-558-1313|5888.10|ly final pinto beans are carefully ironic hockey players. regular packages breach express, s +3530|Supplier#000003530|0t 0sStT pH7DD GWU,D1F 6BvSk4|18|28-774-141-7569|3388.72|cial packages thrash quickly slyly pen +3531|Supplier#000003531|7kfwtcLBlYSpTOPhwcx|15|25-171-945-3912|5139.74|e slyly according to the fluffily furious instructions. slyly final +3532|Supplier#000003532|EOeuiiOn21OVpTlGguufFDFsbN1p0lhpxHp|23|33-152-301-2164|9558.10| foxes. quickly even excuses use. slyly special foxes nag bl +3533|Supplier#000003533|81fVAnBKYv|11|21-311-899-8957|8064.94|cies. slyly regular theodolites thrash furiously. express pinto bea +3534|Supplier#000003534|vGLHyvqDzoGJ,9QF4S|7|17-420-181-2025|3530.31|ndencies cajole quickly. special dinos cajole bold theodolites. pending depo +3535|Supplier#000003535|u8I9LNtaG4tTe9zvcazd6Rgac0PEdrywl|1|11-193-489-5402|8988.09|al pinto beans nag furiously pinto beans. quickly brave excuses +3536|Supplier#000003536|vzCsRjennl78FjFMURrgihVJ3lie8kOJ|5|15-505-807-3709|8934.62|ly unusual excuses use furiously after the slyly fina +3537|Supplier#000003537|upAlzbn0gUSW4fq3r2pIn7abaiSedT,7nVqkA|19|29-525-534-3904|1231.69|ever according to the slyly sp +3538|Supplier#000003538|i9YJEjDfWE0As55QBPjpWo,r qQDz|23|33-835-987-9769|7971.59|dependencies boost carefully across the quickly regular accounts. fluffily express asymptotes +3539|Supplier#000003539|uPKu8p0Vv2MzTU8y POo19yNgM4Hz6JB|0|10-699-417-3533|7640.09|. packages haggle across the deposits. quickly final deposits at the sentiments haggl +3540|Supplier#000003540|CvZqZfPshuE6mBTPEERtldwPVbWbRMGDvQc|23|33-371-643-2160|2036.01|s affix quickly against the ironic packages? special requests cajole furiously slyly special +3541|Supplier#000003541|DbwyOxoaMEdhEtIB3y045QrKCi2fQpGRu,|4|14-508-763-1850|7166.36|ages. carefully unusual requests across the pending instructions aff +3542|Supplier#000003542|N1IWhZiIIlKYcimMD a6|2|12-751-653-8957|4087.29|press ideas mold carefully dependencies. ironic theodolites nag fluffily. slyly even theodolites +3543|Supplier#000003543|0E6Hi3yS3XtRbHrTG5sL|22|32-738-785-7813|9067.93| cajole carefully excuses. regular packages snooze carefully regular requests. furiously bold p +3544|Supplier#000003544|i1wE3ayJn7WW9YClekXHq2Xyjmro4X|24|34-444-386-4758|8808.45| ideas cajole. carefully +3545|Supplier#000003545|9a0mvwZA3bAlXEs3v5 aVrosabG4lzAn|8|18-861-618-5938|3652.07|gle carefully boldly even deposits. carefully regular requests haggle furious +3546|Supplier#000003546|luZmPW4eLY3qH02F|6|16-956-719-5117|9378.65| carefully pending, ironic instructions. furiously special platelets sleep? furious +3547|Supplier#000003547|5z96HW9oQ2b hqfJtJ|11|21-178-297-6374|6033.24|ions across the pending courts nag c +3548|Supplier#000003548|Gj4v6V76 8Cu44,Sh2P5R7s2 f|20|30-381-458-3286|659.88|tes. pending requests against the slyly regular requests nag after the express re +3549|Supplier#000003549|s,HHd C3rmE99jrIy1805hXcWVFBB3sEbQec|16|26-651-385-1511|588.03|slyly even accounts cajole about the fluffily regular +3550|Supplier#000003550|Zlc9iIJr8HG25Fq|6|16-706-183-7553|2918.36|ronically regular theodolites. slyly final deposits after the carefully regular requests +3551|Supplier#000003551|bumlGkLc0PiTwlwIVoIM7yvF6EBPZ0|18|28-242-414-9331|8843.87|ke furiously. furiously pending dependencies sleep fluffily ove +3552|Supplier#000003552|PMWgNdKj2YRTgdEaZZdqizMj|17|27-230-716-8375|5019.77|lar accounts. furiously regular ideas sleep quickly slyly ironic ideas. fl +3553|Supplier#000003553|u3nDX5MdZaYCfhoGN3uoOAHQYFKUgwcVeHR|3|13-427-197-7290|6365.96|al, pending deposits cajole. quickly reg +3554|Supplier#000003554|cmXWaaUy 3PMb7NdN5BXEE0UR0TP|1|11-249-333-7495|4704.16|ages wake quickly. fluffily even deposits nag blithely unusual, final asym +3555|Supplier#000003555|mx1puagouYCETNPNERVuC5lBMWz|21|31-313-659-5450|3751.26|le slyly according to the slyly final frets; furiously pending +3556|Supplier#000003556|orv,fQlamFT EQxgLhUf4nkAaSIN5eQsA9j,|3|13-668-780-1932|4462.25|ies nag carefully. unusual, fluffy requests lose sl +3557|Supplier#000003557|DsMJylgjWDub YHW8Y|16|26-156-686-5493|8081.97|. furiously bold requests are quickly +3558|Supplier#000003558|CYnj79lLULxf,2fHZ1Jp,2N97uREQpHJr1|12|22-739-421-6188|3735.35|ns cajole above the slyly express deposi +3559|Supplier#000003559|tc0tqE65uRsAielk4 NrBiAtWSK|1|11-331-124-2627|980.95|ross the furiously express dependen +3560|Supplier#000003560|g0cYh,7irrnBIPpIjONxadT,z|8|18-535-386-5741|6778.10|packages haggle slyly. regular packages haggle fluffily. carefully final accounts sleep +3561|Supplier#000003561|Zc9L62QD0dCSXxykBJS1K|3|13-942-547-2178|1508.64|s. quickly final packages affix even packages. carefully pending requests are across the f +3562|Supplier#000003562|Vb,V OPeCZXR8BLFzgwKM83s0aRJW|16|26-574-942-5088|9693.10|ully bold packages across the +3563|Supplier#000003563|uaph8YpaCfVZztQroqkBMA2sVtrbH|18|28-721-203-1298|67.70|ironic deposits sleep quickly pinto beans. regular deposits are quickly alongside of +3564|Supplier#000003564|tYOfapsdmZRAq VMX73Mc9BsGa|6|16-720-599-1105|2848.81| packages haggle after the silent de +3565|Supplier#000003565|zsLX9sXmmN41pzdYp Z8,InaCs8Hqk|10|20-367-342-7605|5949.69|finally against the ironic, pending packages. even p +3566|Supplier#000003566|vdZqsVZ57Kpb8PK|24|34-172-569-5987|4444.36|as nag across the boldly ruthless packages. ironic foxes are furiously alongside of the thin +3567|Supplier#000003567|gtd,z,nUBHkR6Aq8JtE6VYSWNzKx|16|26-407-943-4159|4420.09|long the busily special pinto beans. ironic, pe +3568|Supplier#000003568|xFM8ppLiFfMeuzGCTPnL|10|20-783-888-6094|3534.85|after the carefully silent +3569|Supplier#000003569|jt4w8rOANtZ|17|27-509-980-9173|8253.13|e even, bold packages. carefully ironic accounts wake fluffily. f +3570|Supplier#000003570|dXLhbtaebBIgqPc8Iksno|1|11-463-717-2694|9353.40|ing to the carefully final wa +3571|Supplier#000003571|hYkqn6DzQ2b2PuZBjBNdaUxEyAMA|24|34-246-744-6090|-255.51|structions at the ironic excuses haggle slyly against +3572|Supplier#000003572|Jl4sqGiwS0|13|23-514-712-7839|4513.39|ironic packages. instructions play quickly. regular requests after the +3573|Supplier#000003573|kwtzuhsdpNt|21|31-324-183-9738|9580.96|against the furiously stealthy accounts. slyly ironic deposits use. regular, ironic req +3574|Supplier#000003574|SemAqtYkM5ejkSsaR,O2hWy3ANNR86MMfI,ZR|10|20-612-946-8333|7197.08|bold pinto beans cajole furiously even asymptotes. even excuses inte +3575|Supplier#000003575|dc6eZZyHWCqmiKIpwUWs0VAVQ1ZvTI9gkjgE9p|11|21-446-180-3000|3309.64|ages sleep furiously along the slyly sly packages. iro +3576|Supplier#000003576|RyjBp84ymqE3Dbj3Di4ijafS C8mJA 0HRIq55vP|16|26-106-371-3903|7178.90|d deposits; regular asymptotes about the bold r +3577|Supplier#000003577|LUObOZh2GykMUzj1w3oU2pc,r2fZeLTQsUGUr|12|22-995-516-1296|5806.66|to beans. careful, special deposits are blithely after the furiously express pinto bean +3578|Supplier#000003578|ULPrImvgYA,kmQUs|21|31-172-107-7363|9569.55|ests. slyly special accounts integrate furiously carefully ironic +3579|Supplier#000003579|Lf59rlSC0JU6N AyHJyuNcIkN7,PErIXCp|10|20-214-263-2876|2583.93|hins nag slyly regular, special instructi +3580|Supplier#000003580|K7YdexxTIN 2CmKvmfBqaqo9LAnXDKH|24|34-287-927-8968|9143.31|ss the fluffy accounts. fi +3581|Supplier#000003581|MXGcBDKvEd5aifcSXiE1BfFK3z8zZoZbmXhjDvB|1|11-700-180-8215|-280.90| carefully final courts boost. slyly quiet deposits about the slyly regular account +3582|Supplier#000003582|Sh8U4Y56IRO,2r|9|19-446-550-6296|6912.43|es haggle quickly after the dependencies. final instructions after the furiously quick +3583|Supplier#000003583|4Bitvou6Z4ySgmgFszC6TzTdY8Zi09LNPQHELT|20|30-717-260-1115|4173.42| instructions. furiously busy requests sleep carefu +3584|Supplier#000003584|To60YGgHdAgldyJWT6rpPZcJto32jgtJ00RaI|5|15-470-512-2898|4664.84|es cajole quickly slyly ironic asymptotes. unusual req +3585|Supplier#000003585|rd0xOzg2Zvki43ssTzqErfS4ZxSN6Kaleo|5|15-923-605-6967|2910.69|ctions are furiously against the bold, pending accounts. slyly specia +3586|Supplier#000003586|OxKA5gwi VkWgSf8r4ZUx5KWqMdeaYKQZ|19|29-223-914-6598|957.59|uffily according to the express acco +3587|Supplier#000003587|h UBYa wnNQuPDW3 9Jst7ohrl5ckLlF8M52|23|33-984-680-2326|-898.30|ickly pending instructions wake closely pending instructions. fluffily fi +3588|Supplier#000003588|Ho9S4ceJ1J99dh63 pigYI7Sf78Mbkoc7iutI|4|14-715-559-8344|3242.94|aggle furiously across the blithely busy instructions. express accounts are. regular dolph +3589|Supplier#000003589|SeVq5mliNtWVvH0NmB|10|20-895-735-3345|8015.11|about the bold asymptotes. slyly ironic platelets sleep aga +3590|Supplier#000003590|sy79CMLxqb,Cbo|3|13-868-737-3600|2004.44|special somas. special deposits haggle quickly regular accounts +3591|Supplier#000003591|bhZpobbcKcyc8BXLIiN0hjDgnvZn7|12|22-921-489-7203|4600.37|! furiously special platelets run according to the even packages. pending, ironic d +3592|Supplier#000003592|,38WkJUa2O81UXZKpmxDbiY1ecwI4 2Ru9hOzW2h|9|19-819-117-4065|8462.78|ar, pending packages integrate platelets. carefully ironic accounts haggle car +3593|Supplier#000003593|sKgUTcgnr5a7qSkjS9F|15|25-275-944-7043|2016.37|posits integrate carefully pending, +3594|Supplier#000003594|vH EF6ltzb4KcGc9qpt |8|18-433-804-1742|7117.38|st have to sleep furiously ironic instructions. +3595|Supplier#000003595|Ch8fr3a56ybpePFgHOKTAbCKiq1|11|21-376-292-9197|5882.24|round the instructions are carefully blithely +3596|Supplier#000003596|pyXt0UgE4TvoiL|7|17-179-432-8934|3761.10|ven requests. carefully regular deposits are blithely furiously final requests. spe +3597|Supplier#000003597|4S5g1qSVNDXpE0kk1pU|4|14-866-468-7679|4800.45|uffily silent dugouts among the special, final dependencies lose along the slyly e +3598|Supplier#000003598|Mqv0Dm5GVoW058oa|0|10-201-817-4893|5245.20|ffily bold accounts; slyly pending instructions haggle alongside of the +3599|Supplier#000003599|fhO2UADLAeTzH6uB10wL4,vBzTQBpj|16|26-980-403-2965|72.50|sits are quickly. carefully bold packages among the quickly express accounts d +3600|Supplier#000003600| CJkYtpnue5Eri8tFXowxxIySMfXwXq2|13|23-870-894-3721|6516.83| unusual accounts; excuses nag alongside of the blithely unusual pinto beans. never pending theodol +3601|Supplier#000003601|Y8GfvG4U9dUfTFwu1kpsXwapWPvdodRDr0hd|1|11-798-621-2278|-576.52|kly even packages. theodolites nag above the regular accounts. special asympto +3602|Supplier#000003602|5oNjsyXdigyvfcwal|19|29-860-295-3532|-578.26|uffily. slowly even packages cajole furious +3603|Supplier#000003603|dlaJMEG5rh8U3MwOzdQO8d|6|16-210-870-8496|3187.71|uests: stealthily ironic instructions hinder furiously even deposits. fluffily special dep +3604|Supplier#000003604|Wh54BLC 6VSu0WjNv WHnslIfEQ6qa3PJUdLlyP|11|21-762-284-6273|3799.21|ar asymptotes cajole according to the careful +3605|Supplier#000003605|xdRVBnukNC3QIuf,,1lbmd800P5 Gx|22|32-708-366-5881|7349.08|s. fluffily ironic ideas cajole carefully blithely ironic foxes. +3606|Supplier#000003606|zDrqlF MegBKfXn,hqaeRdAWfjYkFCmj|8|18-573-230-2668|6045.06|kages x-ray behind the unusual, fluffy ideas. furiously stealthy frays wake about the +3607|Supplier#000003607|lNqFHQYjwSAkf|3|13-152-935-8310|1263.97|accounts engage furiously +3608|Supplier#000003608|b5zVYs7ziq|6|16-355-434-1153|541.30|final theodolites sleep deposits? deposits detect above +3609|Supplier#000003609|kVw7JwQLjSGkM|12|22-417-679-1450|9610.23|leep furiously final dependencies. bl +3610|Supplier#000003610|YjjToWCrnEe5en8pIVN92h nq1nOPyMbwroosJw|22|32-670-281-5775|4261.70|around the deposits are quickly ironic pinto beans-- final deposits according to the carefully +3611|Supplier#000003611|FSPfboa25EjNKVDf|12|22-344-349-2209|3414.57|nag slyly furiously quiet requests. furiously pending excuses haggle +3612|Supplier#000003612|bAV9ZD QRt2WxJGltie8o3ihwffMrqMrkvN95|0|10-750-846-9322|-326.96| the slyly pending deposits wake unusual deposits. carefully bold theodolites integrate slyly? +3613|Supplier#000003613|r1aW3duU4 EJEg2lilZyY9Y7vrGYcyl2lpV0akz|23|33-557-797-1075|2775.44|unts nag fluffily across the ironic, final ideas. carefully regular ideas integrate silently final +3614|Supplier#000003614|om8XnsBFPKTxjvKhn 168SlP5drWoP7r tZP|23|33-876-776-9763|4815.44|ithely. regular packages detect. final foxes cajole. carefully ironic accounts slee +3615|Supplier#000003615|tMQyLZmdndVb9Q9VUh,yQNXRnYE6nv|16|26-633-200-8239|4183.98|ffily thin theodolites. requests mainta +3616|Supplier#000003616|2hodFxSvvvrTFE8V|7|17-632-210-1401|6115.22|lowly regular accounts. c +3617|Supplier#000003617|LB 87BiV4saHACdKeIqnp5SrBvLAu|3|13-249-810-4491|3533.30|aggle. fluffily regular packages haggle blithely. blithely unusual pi +3618|Supplier#000003618|GyUWkSMZ3Zgrlr7pfznrNvrC5HbQAQ|20|30-998-346-4311|4158.63| asymptotes dazzle slyly around the furiously unusual platelets. requests about the furiously fin +3619|Supplier#000003619|9d8ORCzNKHhK 5VfdTNd9Sp6HGOuM5tBfh5|23|33-969-752-6348|9085.55|le carefully above the packages. express, final excuses affix blit +3620|Supplier#000003620|H0WF1XyKJJ|14|24-273-204-9382|7858.14|ly pending accounts run fluffily carefully express deposits. even, express accounts agai +3621|Supplier#000003621|fW1IcCOBk55bSqBA|5|15-803-438-4439|3792.49|ar pinto beans sleep fluffily. qui +3622|Supplier#000003622|0ALYufv3wHRLXOl3rWpYxGW3 w|4|14-708-519-7655|4311.63|courts. ideas are carefully above the foxes. quickly bold grouches haggle slyly quickly i +3623|Supplier#000003623|0bmFB1Ux VvF30f4b62otQ1zeWETRgXw5oY|24|34-115-155-6095|6514.08|al requests. slyly even account +3624|Supplier#000003624|gbXvyWmGOMR2g,UizoehIDlhoWKN8Mad47X|24|34-926-564-4481|8232.28| beans above the blithely regular ideas integrate furiously final instructions. furiously fi +3625|Supplier#000003625|qY588W0Yk5iaUy1RXTgNrEKrMAjBYHcKs|3|13-799-445-4707|53.81|ular dependencies. idle pa +3626|Supplier#000003626|gKCZZFgEQJpykB9|6|16-378-648-1055|354.14| along the slyly special deposits boost furiously against +3627|Supplier#000003627|77,1uiRw rXybJh|6|16-568-745-4062|-986.14|closely blithely regular dolphins. fluffi +3628|Supplier#000003628|Kl0PD5Va5M9ln|17|27-648-194-8955|777.72| deposits across the final packages sleep slyly +3629|Supplier#000003629|fco32LPL55pLFoH0|18|28-780-309-4610|1022.15|above the dinos. furiously ironic packages affix slyly furiously ironic deposits. car +3630|Supplier#000003630|x YhwKhY,c1A|6|16-376-892-5137|6174.29|r accounts haggle slyly even accou +3631|Supplier#000003631|kR4ZiwufGGPWhIDCxK0bLov0Lt4sNgxSh|17|27-913-575-5363|6376.66|sual foxes wake fluffily. carefully pending ideas haggle furiously regular accounts. +3632|Supplier#000003632|YzyEutwcOJ6Urp6ct|23|33-998-670-6463|8619.47| pending epitaphs affix blithely at the ironic, unusual deposits. carefully ironic requests haggle q +3633|Supplier#000003633|1Md2Gjf0IiFEKpDUwxL0Hj0M6piL7KC|24|34-692-340-8808|6463.32|side of the carefully regular requests wake carefully accounts. unusual foxes wake slyly sly +3634|Supplier#000003634|NIhKhh3almAcs2eLuCVyigUlf93xJZTI|22|32-841-841-4952|2070.74| mold furiously across the daringly regular dependen +3635|Supplier#000003635|iZVQF YThR0AJ5kW8QaHZh|6|16-285-983-2036|7842.17|inal, regular requests. ideas nag furiously. +3636|Supplier#000003636|kznUP7,Sn50fsK0W3zvFyboL25gVtJNE|1|11-865-296-4467|-542.48|ckages. blithely ironic accounts nod. express, f +3637|Supplier#000003637|jygtKm6sxHag3q1oJ3cwnv6BZ9elN7a|16|26-543-557-9072|-608.02|ven accounts cajole blithely across the even deposits. even requests except the special asymp +3638|Supplier#000003638|IuI1SJAneyEiKkGAy7fjq365CJICTFJvm5WErx|17|27-506-576-5802|8881.25| slyly regular accounts. bold req +3639|Supplier#000003639|WD22esS63VSd806yKIIbtur7izOo0|19|29-131-308-1571|6222.66|ic, even packages among the regular instructions sleep carefully even accounts. sl +3640|Supplier#000003640|u,RXMa,idA9 6Q6UZTezJKamrRdTvq|2|12-246-842-6314|5224.22|. carefully ironic requests among +3641|Supplier#000003641|oZkYieP6fCKlV6LK5v2Cs6qNYfre 0IVd8Edd|23|33-802-670-8953|-497.89|ronic theodolites are according to the unusual, +3642|Supplier#000003642|mI,vZ2Bqj4LsSH6IbtlL,NEyi9xaeaSg1Hd0|4|14-678-737-7593|831.88|efully even theodolites x-ray. blithel +3643|Supplier#000003643|DPMM fUKicM rvf3pg27FFtux,XG07bcom5o |18|28-892-419-1661|9306.51|y special requests among the slyly pending courts was quickly even pearl +3644|Supplier#000003644|y5StdLT9IszUi,lRX2XwT5QF|11|21-243-887-9418|3658.67|ray slyly after the fluffily bo +3645|Supplier#000003645|b55jNjHerLlegKZVBwdGIu8JDn8PPFA ZA|20|30-588-257-3908|-802.86|e carefully silent accounts. express, regular deposits use. bold packages are furi +3646|Supplier#000003646|M9Gjke9gEVzL eCx48n 7V|20|30-649-186-3927|1312.83|after the carefully final instructions. +3647|Supplier#000003647|XZ4XX,PE05ojnDJHs|0|10-408-429-3313|7692.58|e quietly after the blithely silent requests. dependencies use furiously regular idea +3648|Supplier#000003648|WXBrWit6if2ZMtqWJlI2V6hBg68|2|12-256-611-6867|4296.86|al deposits: furiously even instructions wake slyly sly +3649|Supplier#000003649|3VYKcpYiF8|24|34-765-925-1924|343.47|ackages. special instructions sleep! carefully express packages hang fluffily deposits: bold pack +3650|Supplier#000003650|WM38VVLFoqc2g648vFs5tNfpF3J71ccG|5|15-534-720-9698|6688.42|ress deposits. fluffily even ideas are: slyly regular braids play slyly into the regular ideas. sly +3651|Supplier#000003651|szujolUDXyUVznyYhYppHnNGKm2|4|14-403-491-4192|1331.82|gular theodolites sleep carefully about the silent, regula +3652|Supplier#000003652|BJVxk,zw5nau6FCSum9xWVZ|3|13-626-476-3206|7898.00|yly final deposits. furiously +3653|Supplier#000003653|9t,3txrVc35Sb4PhBF6S|2|12-747-742-1223|2791.84|pending instructions. carefully regular deposits +3654|Supplier#000003654|84uF6Xn9YVe3IDfkJB|14|24-715-793-6866|5386.72|ly express excuses cajole quickly across the ac +3655|Supplier#000003655|YCvXBNgPkoNAvD2hHR,njixO1|24|34-916-618-4033|2600.24|ymptotes cajole among the furiously regular deposits. express deposits hagg +3656|Supplier#000003656|eEYmmO2gmD JdfG32XtDgJV,db56|3|13-215-118-6468|8206.98|ng to the furiously special pinto beans maint +3657|Supplier#000003657|QpnprdyZgyW67moCuE muWx8NRB76u3BB|17|27-382-463-1892|3748.16|hs haggle after the special packages. furiously unusual asymptotes us +3658|Supplier#000003658|hMMvNn8Z6SvV2B|15|25-674-521-2213|7239.76|t blithely even pinto beans; final packages detect thinly dependencies. carefully ironic deposits +3659|Supplier#000003659|G,6IKNjV7Kz9ux6|14|24-855-440-7662|5696.06|ress blithely. accounts impress always acr +3660|Supplier#000003660|WkbM,vhzlbSS2y8rnnw LS4oKRT9C|15|25-956-863-7727|9727.49|erns. slowly sly pinto beans print carefully regular excuses. furio +3661|Supplier#000003661|fNvBB3d0A7Nnuo6ZhOf5XPKzTjER O6zm|1|11-310-974-9282|5975.22|rns. carefully even frets sleep carefully at the +3662|Supplier#000003662|iXT9k11t8M7fQWyEWhpeeB3PgK5Dsm6xRz|17|27-338-707-1126|4528.42|ly ironic platelets. bold deposits wake furiously. special ideas sleep. s +3663|Supplier#000003663|0r0W5zyJGaYCLaxD19CgSa|14|24-218-659-8495|9216.29|sts are alongside of the fluffily final asym +3664|Supplier#000003664|aCqlSlxw,ouZcQdn|18|28-326-140-6125|5234.48|ly regular requests sleep slyly. regular foxes nag carefully. even, regular accounts +3665|Supplier#000003665|vQEsRjcsJukdwIQ6F7A0g8WYj74LNFMu|19|29-931-790-4275|6955.39|eposits play furiously ideas. th +3666|Supplier#000003666|ENS fE9iSrSzw,iTwA,zGorkflw|21|31-974-495-7616|234.45|nal instructions. regular courts haggle carefully a +3667|Supplier#000003667|qXG0xhg3kWLm yeNMgUncK|19|29-126-841-4688|287.37|s affix furiously above the accounts. furiously +3668|Supplier#000003668|U7HrMSGbwuwLC|17|27-539-572-9175|9826.53|. regular, regular warhorses wake carefully silent accounts. slyly unusual +3669|Supplier#000003669|yxO31BBDfe5vIKGwLWhZGsWzngdAgs8oSq6L|23|33-852-933-2119|9595.09|ly beneath the slyly stealthy pinto beans. qui +3670|Supplier#000003670|LkBBo8Qfq0r56OHMaS2uV0E|19|29-896-212-1197|2744.53| cajole carefully according to the bold deposits: regular packages wake blithely according to the f +3671|Supplier#000003671|WpUjKrK38i3Vme2OCxx8Zcm4sMw |7|17-677-466-8220|2247.55| fluffy pinto beans. carefully final dependencies detect fluffily unusual accounts. regular packages +3672|Supplier#000003672|oeypLDwHF1wD|18|28-230-110-4814|9483.17|inal decoys. final packages are slyly regular accounts. carefully special instructio +3673|Supplier#000003673|njOXUCLw4em|24|34-340-430-2327|2754.33|kages. bold accounts nod blithely requests. +3674|Supplier#000003674|0pOkKOHE30U1MWZ tC81Fk0OY5tLpt55z3nLc7M|14|24-780-366-3010|1985.27|ecial ideas. blithely regular ideas haggle. blithely final requests hag +3675|Supplier#000003675|4PiatZZ iVMXhJ VKG|20|30-121-667-7383|1409.61| bold platelets detect bold requests. furiously ironic accounts sleep carefully. carefull +3676|Supplier#000003676|pR,aH Rz5XCsbao2|13|23-761-301-3749|6273.13|tect furiously. carefully brave +3677|Supplier#000003677|vylQOXxiGWp7cfAl,CRS6jC,Ees4E0gradxi|12|22-405-990-5050|6589.78|ickly bold foxes are carefu +3678|Supplier#000003678|LN9MXDkJ2g,MDiWKdujrnBCzm0d9PzTYMsi|14|24-116-658-8539|4488.70|p blithely. furiously bold accounts are. silent requests hinder about the fluffily steal +3679|Supplier#000003679|CtP5pEp8k7H|3|13-871-153-4001|1542.67|he unusual foxes. furious +3680|Supplier#000003680|w b4NWNtVCfK BF|12|22-993-207-3523|9586.20|gifts boost after the silent pinto beans. +3681|Supplier#000003681|spxhsMZtpYyLdgGS21dKMjjd3RDC9qYGj8kb|5|15-879-583-1708|7324.80|s sleep furiously after the even, pendin +3682|Supplier#000003682|TjvwdfHoZe N66xzc7xa|9|19-219-861-5752|5026.20|sleep blithely regular deposits. slyly even foxes are furiously according to the fur +3683|Supplier#000003683|K,BCJ,GFMzwO318my5eeipXflhJhYdIbbU|12|22-565-903-4819|4369.13|posits. slyly even Tiresias along the blithely ironic excuses lose carefully about the blithely eve +3684|Supplier#000003684|wDE48tQ6hY5d,pnu50AE,npUOh5C|23|33-950-827-5956|4947.52|olphins. blithely even excus +3685|Supplier#000003685|ROI42SF7q Wo|21|31-756-950-1669|1926.26|ust have to boost blithely across the quickly special accounts. furiousl +3686|Supplier#000003686|9fwXT1RCQohLoNEOiF2|14|24-873-490-2508|-371.66| accounts wake furiously blithely bold instructions. unusual, +3687|Supplier#000003687|,vh2KiOz1ZWSKwbi|7|17-589-469-6235|-589.09|ructions use carefully regular accounts. packages wake. furiously ironic deposits det +3688|Supplier#000003688|W8BQPLzJNBzuIwjL|16|26-604-435-4993|8143.47|olites detect slyly. bold packages +3689|Supplier#000003689|KuH5dUsSzixv|6|16-950-731-5933|3309.56|osits. regular, regular excuses according to the blithely e +3690|Supplier#000003690|83mGyrFbG 6tJ38U|12|22-292-458-7250|5736.46|y bold accounts among the furiously even +3691|Supplier#000003691|fL3BRzsIz4oMxOY8hIEBTnGJigm90w72Mm616vB4|19|29-228-528-6522|6734.62|onic instructions sleep carefully ironic accounts: special instructions kindle abo +3692|Supplier#000003692|KH fnJjsJVeaZbVkjGLtEvpWpTU2 Y9sHR4|15|25-535-577-3607|7730.65|t the furiously express hockey players. carefully final deposits wake blithely ir +3693|Supplier#000003693|A9NzAktL6o3quvmoM6Fzcodo6OY6x,OncHfGGU |10|20-288-867-2379|6071.37|ray blithely carefully careful accounts. ironic asymptotes nag always special accou +3694|Supplier#000003694|3FSGtsO1AGTgDX|6|16-731-341-9923|3477.63|ts. even, express accounts maintain carefull +3695|Supplier#000003695|g8dtMW3MyHKfV1uYUTB7veUEQom8Ocic|12|22-424-749-1175|3446.26|es use special courts. furiously furious packages ar +3696|Supplier#000003696|WmHsvbtR4DD2BT|16|26-289-199-4350|4354.44|st. final, express dependencies cajole furiously deposits. unusual frays cajole blithely +3697|Supplier#000003697|EBQJU8cRZVBZE8e 3M7|9|19-595-924-1110|11.78|ructions sleep furiously across the theodolites. slyly even excuses are furiously. carefully iron +3698|Supplier#000003698|lnSEu64ca4B53BfznJPg|21|31-804-633-2078|8940.75|nal theodolites boost fluffily blithely final +3699|Supplier#000003699|yb8xEBvbGVfElnxjzZPSxZ mUbWq8oWOCmbErLLx|0|10-373-811-3558|5336.50|otes: final requests wake slyly. theodolites above the finally bold accounts are +3700|Supplier#000003700|q1xYaKyPFw4xacddUJtQ1H|11|21-452-537-2128|5607.19|tect furiously carefully final platelets. re +3701|Supplier#000003701|RhXkYsRMx6SUaFg6RslNl2,9v7iU8S6q2W0g|5|15-825-550-1220|1067.25|final deposits sleep furiously regular excuses. carefully regular pinto beans boost even req +3702|Supplier#000003702|HzPbcxd6nPXU4wtvM7DiPBihBCEbI6|19|29-419-845-6897|5174.82| across the quickly pending accou +3703|Supplier#000003703|AQbAWvZy7yB16NGoQ9cLdhvF73RqUB2n |15|25-357-935-7420|5202.28|longside of the blithely ironic packages cajole enticingly above the furiously ironic requests. ir +3704|Supplier#000003704|Ipq oyCih25j9LaPoKpE2eqGBZSnN|24|34-131-384-1909|4757.79|asymptotes boost. carefully regu +3705|Supplier#000003705|bdDmDdjqoeQhAyH5a|10|20-848-197-8828|3206.24|ven ideas. carefully enticing theodolites sleep quickly blithely pending asymptotes. ironic +3706|Supplier#000003706|Nj0WKcb8eLEoK4e gJSZlo|6|16-153-912-3529|7275.55|depths. regular, express instr +3707|Supplier#000003707|E y,Bok1gTKqdaSdnA9|21|31-834-669-3387|2202.16|ound the bold asymptotes: quietly regular accounts use idly along the ironic in +3708|Supplier#000003708|7o,xU09kadI|12|22-518-213-3462|5695.87|ctions integrate stealthily. ironic frays use furiously blithely daring pac +3709|Supplier#000003709|sKb0zizjb5,fiWrpvyQIGQu|6|16-992-515-9915|6523.65|osits lose blithely bravely unusual ideas. even instructions cajole along the bold, even excuses. f +3710|Supplier#000003710|qSAl6zmSagdv|6|16-559-766-9709|4008.58|s haggle. slyly ironic instructions haggle quickly carefully regular platelets. unusual, c +3711|Supplier#000003711|8,0BJbF0d1TI0D,49d|14|24-904-562-9289|6155.70|uriously even foxes sleep qu +3712|Supplier#000003712|O8KIWj0KQCJ8PagC2q18|22|32-896-185-1763|1905.04|ajole. furiously even theodolites wa +3713|Supplier#000003713|er8RX4OzuGmQ|12|22-596-754-1746|5974.34|iously-- orbits alongside of the blithely express packages sleep blithely final theodolites. qu +3714|Supplier#000003714|zp9PksqggawYa6OK0qDpD O6|23|33-640-138-6987|728.08|ly pending requests are fluffily. quickly express excuses wake slyly. furiously regular d +3715|Supplier#000003715|tEgnSYHQkY|13|23-414-190-3025|2815.13|le among the ideas. carefully even accou +3716|Supplier#000003716|,euFw72LEMyrS4b9htTFbC,dgok4X|7|17-937-908-5632|832.55|ts cajole among the final foxes. slyly final asymptotes haggle aft +3717|Supplier#000003717|Rv i0dr JcK9yvpE6E1DP7eh70Mxj|2|12-578-877-7477|9653.36|theodolites haggle quickly at the blithely even accounts. sp +3718|Supplier#000003718|VPtTYIzJwSIEvyOSe0BCtrY6I|0|10-512-549-2865|3037.75|ounts use furiously against the quickly ironic instructions. fluffily special deposits sl +3719|Supplier#000003719|MXFSR9lWKlwkE5tTn5BQNt0d tXbQjL5IjPhD0|6|16-515-625-2000|2416.58|ependencies use. carefully special accounts so +3720|Supplier#000003720|FShLUvBH QdZGX|8|18-372-752-3019|1364.40|ccounts nag slyly-- unusual requests cajole: furiously pending platelets dete +3721|Supplier#000003721|6kMrz4UiOhnC3BQoLi,swzq|23|33-694-584-3100|5354.26|nic epitaphs are packages. regular accounts sleep sly +3722|Supplier#000003722|8sfegSmxTbBqT,CW,eTT3|2|12-697-843-4160|7142.30|ironic packages nag according to the blithely regular platelets +3723|Supplier#000003723|jZEp0OEythCLcS OmJSrFtxJ66bMlzSp|3|13-730-940-2626|6663.04|ent deposits! furiously regular ideas along the final deposits haggle care +3724|Supplier#000003724|AtP0R4b3hh Ccokt5DpusV|12|22-263-722-1229|3689.96|. ideas mold quickly instead of the carefully pending foxes. packages haggle s +3725|Supplier#000003725|Mjn3EvR4uNQ|12|22-909-405-1726|8205.52|e quickly. regular requests wake slyly among the final p +3726|Supplier#000003726|mgbHN7BL,J5iANEvMIsyh7tTyD5uAnuTl0MeHt|13|23-266-101-7338|1221.79|ose furiously alongside of the ironic foxes. careful +3727|Supplier#000003727|dIE91,KrCd2y|20|30-262-738-9329|6540.82|regular tithes wake blithely even packages. slyly regular instructions use +3728|Supplier#000003728|l2Q01AnPC0yOpEoHhDSwERUMLMEWf182j|18|28-365-225-2206|6670.10|onic packages. packages cajole blithely. carefully final packages against the blithely ironic deposi +3729|Supplier#000003729|I4Lj0t,uQ2crCtGc2ksJ0R7hdDeNybtWqPr|24|34-435-872-1573|1440.27|yly. quickly pending theodolites are slyly at the carefully close requests. packages sleep never a +3730|Supplier#000003730|CQwSVgaug86Vhwt|9|19-147-682-6287|4613.41|ide the blithely bold asy +3731|Supplier#000003731|vJYRW63TbvmpOGCeG16w5D|23|33-849-804-9421|7602.80|y final pinto beans integrate furiously. blithely silent accounts cajole slyly. fluffily iron +3732|Supplier#000003732|BQZvWD1FJb94fu9DC66zN|17|27-491-104-3581|2233.47|theodolites impress according to the always pending ho +3733|Supplier#000003733|4GFmXZviw 5|5|15-546-392-8568|3838.39|thely regular multipliers-- ironic, express asymptotes haggle fluffily final pac +3734|Supplier#000003734|gGFEu2K06gXjYFq, Ya7PCULfry2HNa|9|19-487-478-5574|5727.22|. carefully even accounts according to the furiously +3735|Supplier#000003735|FJuiwzmI3lXATV0RFJzCMoPPnspiE|20|30-969-220-5677|5420.83|o beans sleep; furiously final acc +3736|Supplier#000003736|PU4sZv3Fb,e|11|21-737-308-8024|3874.91|. slyly regular instructions integrate. carefully final acco +3737|Supplier#000003737|P65R G6YRQqMyDN84q3Q8SJMhLOa9rmOoQ4Jpf|8|18-557-733-9147|3566.75| furiously special instructions thrash carefully quickly final acc +3738|Supplier#000003738|4opzzN4Ws F|16|26-806-697-2662|5421.86|its. final deposits haggle quickl +3739|Supplier#000003739|,M7BbUGl58JfQ1N4pL5,AC5C0wzqPo8ovEHTMQ|2|12-425-404-4099|-750.31|y ironic deposits. furiously special deposits sleep carefully despite the u +3740|Supplier#000003740|5DXIVeJ0cTdbChqyQI670,2KP|7|17-979-910-3262|4534.10| carefully requests. slyly bold foxes are abo +3741|Supplier#000003741|1Pjm9w2,H1puAucyde3Pd339VVZY,tBGhJ|4|14-585-376-5778|5660.90| special, silent excuses will cajole blithely. wa +3742|Supplier#000003742|XpkoI4N mdygllzVlC1SXvE29Pk0wm|14|24-782-883-7076|1760.72|ptotes cajole slyly express accounts-- slyly regular packages print +3743|Supplier#000003743|VCEZjKiG El32A|1|11-124-390-7542|316.95|sual asymptotes are slyly. furiously daring instructions boost carefully according to the even +3744|Supplier#000003744|0v5NnczcZxJBGhk2t1iKDiHWABdcHuQM4WBz|13|23-208-541-5988|842.29|e furiously. blithely final packag +3745|Supplier#000003745|h8Y,54J1eu9nBAQQz4|22|32-412-377-8563|8267.09|inal instructions cajole carefully even deposits. even ideas cajol +3746|Supplier#000003746|O43Nikgv5lasOik8Ez2mOt3uU|6|16-626-258-5750|2033.20|lithely slyly regular pinto beans. carefully final excuses slee +3747|Supplier#000003747|3nI7yfkXomumy7LI,Udb9m6ULcOt|5|15-810-590-2125|3921.43|uffily express theodolites mold express pinto beans. permanent +3748|Supplier#000003748|8F1E1XqrNX,WZZu|7|17-174-809-4911|4471.56|aggle. requests nag slyly. fluffily final dependencies about the ironic deposits nag furiously af +3749|Supplier#000003749|L7rz3XecCb63AYwHpccbb2FcKXLtN s|5|15-736-222-7223|-466.21|sits are quickly along the even, regu +3750|Supplier#000003750|FpR1 VC8Qxt|9|19-662-709-9423|5369.33|ecial theodolites will solve until the carefully unusu +3751|Supplier#000003751| 2oTtXQ7M,|19|29-278-905-7511|7299.85|kages; finally express requests sleep furiously after the blithely regular deposits. carefully +3752|Supplier#000003752|srn8lPVy08QV0i7EXkKbu4YAURhu8u8|19|29-181-700-5552|3690.89|ely final warthogs. blithely final +3753|Supplier#000003753|26,OuQK5ZAVhinkxUo1KiBCMlmFPlp6DqTM|7|17-409-903-4484|1856.44| blithely according to the quickly ironic accounts. ironic dolphins sleep across the iro +3754|Supplier#000003754|Gg29BanCLe5uLL0kMD84|22|32-598-107-4530|3149.77|g packages. final accounts engage fur +3755|Supplier#000003755|veomj,f0Pp3r IC0,US|24|34-729-799-2771|4129.18|, final dependencies wake furiously. furiously even deposits wake furiousl +3756|Supplier#000003756|uKAIfW8hidCiViTFHF9J, PGHuM|6|16-138-378-7472|9068.47| to the pinto beans cajole across the furiously regular asymptotes. final requests wake. final +3757|Supplier#000003757|gRK2je4czRRx|23|33-232-143-9442|6780.33|ts cajole permanently across the accounts. ironic tithes wake slyly. slyly reg +3758|Supplier#000003758|KRy66SUhGwhqLF5pGYevJGaCiCsrB|11|21-748-267-3043|5968.58|ending instructions. pending excuses can haggle care +3759|Supplier#000003759|hOq1CPZ9AAw|11|21-832-997-1794|2873.83|iously special foxes use. blithely regular requests was ironic requests. regular +3760|Supplier#000003760|WR5a5f7o8tC|4|14-111-294-1590|8491.72|ests. express, express deposits are carefully. blit +3761|Supplier#000003761|j3zjAxTOpLPZW7aarFoFLN0IN,,eGaH81LvFof|2|12-897-856-6893|4374.00|ts use. unusual accounts are foxes. furiousl +3762|Supplier#000003762|WSG6rYnCGTb53M5Xg,vjZtQuQVGeGPPMEt4z|4|14-265-178-5568|9735.65|furiously final requests above the carefully ev +3763|Supplier#000003763|pj1BzZfawlj8 zLBiX b|18|28-672-268-4052|219.83|le blithely against the slyly e +3764|Supplier#000003764|888kpdqayPqPBrSZ2aKNh9fSY,aaFPQb|5|15-733-458-3598|7595.03|es haggle. pinto beans use furio +3765|Supplier#000003765|y18N,,38p4OnakyclJKsHRVKTEaI8|9|19-954-496-5991|3644.05|run carefully against the silent deposits. blithely ironic excuses are. quickly +3766|Supplier#000003766|mO suhe 4yZkt4|24|34-325-690-4941|6587.61|packages. pending asymptotes nag blithely; platelets haggle after the furiously final packag +3767|Supplier#000003767|rFIT4Trrv9jvOtsHIuXmiBHA K8TyCuXX3T|16|26-321-180-4149|1063.22|lly regular packages wake according to the quickl +3768|Supplier#000003768|CZi0Iw9QSDL,eBtGf6Hz8GXrhnyvq2n4P|19|29-996-504-5066|3500.63|st slyly never special asymptotes. deposits promi +3769|Supplier#000003769|lYOcs39GwDE2kxsdk0mLwZRHyp9m|9|19-719-440-9567|5051.25|e fluffily special theodolites. carefully regular accounts +3770|Supplier#000003770|Y4wWM1fN 0ca14XX9oc3PJnrCiC|6|16-243-260-7453|7339.46|s sleep after the carefully bold accounts. accou +3771|Supplier#000003771|fKMoUiWDIp8y mwFuVj7,K|0|10-321-511-3865|9282.60|beans. blithely special accounts boost permanently instructions. slyly daring asymptote +3772|Supplier#000003772|Lrb RV6 RmDhe4jc3Q0V2TnGKKOVaCI|16|26-683-949-8919|8743.08|ing platelets. express packages solve along the furiously regular dependencies; +3773|Supplier#000003773|UWjSotAjkAD|21|31-827-904-7926|4941.78|ges nag carefully pending packages. br +3774|Supplier#000003774|dT7fNpZQtxjlg9 8P5wV|2|12-564-472-1411|4439.87|nt slyly above the blithely regular pinto beans. special pinto beans hinder. furiously ironic +3775|Supplier#000003775|UskvgZnGOgFRv3O3|14|24-697-911-9257|5975.58|unts. furiously ironic deposit +3776|Supplier#000003776|JI6e6wke5yQuBO9PYAZd|6|16-899-331-8993|-330.68|thes. fluffily regular packages solve slyly against the +3777|Supplier#000003777|,6SE2f977,1Y7A npCbhZ6Om1h3E7aQRmIsBc|20|30-356-838-8323|6157.92|ven ideas above the reques +3778|Supplier#000003778|383SX62PlsdEej OozFQAw2OUy4Snapb3,|18|28-602-123-5306|6150.09| quickly final requests sleep. slyly ironic requests are blithely careful +3779|Supplier#000003779|wqg3D4lsbKXFuXAPUqqIgt1fQGZC7|17|27-270-101-2668|6653.55|s cajole after the furiously iron +3780|Supplier#000003780|lisLI5wekcJj6HjQRwHxQmrDLul5D8H7|2|12-721-501-5684|9252.37|furiously special packages use after the quickly express +3781|Supplier#000003781|gRGnXksOWKXzd229|9|19-732-806-1331|6642.29| express, final packages boost above the furiously bold pin +3782|Supplier#000003782|iVsPZg7bk06TqNMwi0LKbLUrC1zmrg|3|13-877-526-7704|6961.25|ackages are against the unusual, regular accounts. slyly regular instructions caj +3783|Supplier#000003783|YXf9FmafXyvoyMT5Bb|15|25-254-427-7069|-314.59| maintain furiously slyly special deposits. blithely final ideas boost enticingly-- slyly regula +3784|Supplier#000003784|vmlB8fQ1MaX81ohCxe|19|29-420-513-9188|1555.70|unts haggle evenly final requests. carefully final accounts nag quickly acro +3785|Supplier#000003785|8S59M6PcyaZA2jrvM,dAYcr|14|24-486-584-6949|8616.10|ial instructions. accounts eat furiously accounts. carefully regular dependencies haggle b +3786|Supplier#000003786|71J09axxPr W6eZ|0|10-721-299-8611|4936.76|dolites haggle quickly. even, pend +3787|Supplier#000003787|bCEvfdmHD0DFRlZGCM6tD5DguS2q|1|11-436-875-7720|8135.92|are slyly! furiously unusual accounts haggle blithely daringly bold pinto beans. forges slee +3788|Supplier#000003788|AO67puBfcpnY|16|26-474-988-3311|8465.40|cial ideas. slyly ironic package +3789|Supplier#000003789|iDiVfkHeNky|1|11-732-905-1016|2104.99|he permanently even packages. special attainments are. +3790|Supplier#000003790|vKEajCmMniaZBxwyOb1qwk63IlQJL8bENqQIJn|1|11-386-956-2273|463.92|ainst the special, regular asymptotes detect across the unusual instructions. +3791|Supplier#000003791|bvUwGHnDocDRwhoAIAavU8xQlXmyhpPjNlcI|1|11-480-272-8893|7079.18|side of the even, bold accounts. slyly special requests haggle quickly quickly +3792|Supplier#000003792|wQn56oHuRHmGmRm,9fYrtQ5nJcG|12|22-448-515-7638|1812.57|ing accounts affix. furiously final deposits wake agains +3793|Supplier#000003793| 0PXTvv,YEplyHmw3u|24|34-465-967-6604|2855.19|yly special warhorses haggle slyly unusual asymptotes. fluffi +3794|Supplier#000003794|RZBP5zv,DVmlQUyr5Q 15NuLJGrLzYdPN3UpLDlj|23|33-754-466-6655|3225.83|counts. carefully ironic theodolites boost slyly special theodolites. quickly even requests +3795|Supplier#000003795|Q6DPJ4AogZ RFB0TAs,F2FtIW8C67wax9pE|3|13-171-831-3034|6484.89|egular deposits nag furiously final courts. furiously u +3796|Supplier#000003796|gC,28F ofakz0ZdgKQ2nrW7JFO35 RJN|6|16-248-453-7753|1638.69|ithely. silent requests above the blithely special packages boost carefully after the quickly even +3797|Supplier#000003797|UuWXXv4ID LLOj|3|13-501-217-9665|6396.93|y express ideas-- fluffily pending pa +3798|Supplier#000003798|F5g3WteYQiK1rxnj7x0aPv1TdSE7hO6d,LkZ96B|5|15-508-254-5516|8169.54|during the close platelets. carefu +3799|Supplier#000003799|fFNV7NiIdwh7uMK|7|17-660-903-4812|1547.92|l ideas upon the busily final pinto beans cajole thinly along the accounts. +3800|Supplier#000003800|x4PxtpKcMqS1L8J|8|18-930-103-2299|9863.46|are daringly express ideas. special instructions wake furiously. bold, ironic instr +3801|Supplier#000003801|MMSgwtRFCJJi,AOihagY2|16|26-987-341-4752|3333.88|e furiously even ideas. flu +3802|Supplier#000003802|PjcKEJb 9DODZXC|11|21-686-265-6697|7567.32|ng requests are after the regular packages. carefully i +3803|Supplier#000003803|PyP 5Qdwy8x|6|16-619-989-3845|-61.94|the carefully ironic courts cajole slyly regular, regular packages-- blithely express +3804|Supplier#000003804|,vg4jVUW83vMxGvB19NvMsZvDmkM8AjBkfyya|1|11-224-172-1030|1887.82|Customer nts. furComplaints +3805|Supplier#000003805|UgBZkbdqN286S,3NzCtSFaz,2su1HZ55uSq k4|21|31-550-945-6309|2623.84|express requests sleep furiously. ironic, unusua +3806|Supplier#000003806|n ElQaZan8EZ42YVVtEHmFlLYwKw8xcY|20|30-636-160-7336|8777.40|e quickly since the quick +3807|Supplier#000003807|ot3nvn3kdvL9YcxMp8fhWN CsorSKs0LN4|7|17-850-427-8587|9903.42| the quickly final deposits nod carefully ideas. regula +3808|Supplier#000003808|pVvbVjKuExkDl7ka2c1jPO8,sSkty8|13|23-385-483-5966|3857.03|leep blithely furiously final deposits. slyly ironic realms boost blit +3809|Supplier#000003809|55E3UPlPoG7|10|20-122-491-1283|49.77|e. furiously pending tithes +3810|Supplier#000003810|1zPgEZHmn0fbUaQDRNH,uZ uci|21|31-319-356-3632|9380.43|l instructions play furiously bold accounts. bold, final accounts a +3811|Supplier#000003811|XJ7JXy4QLODzT0sggg4H Io8hvAxhM|13|23-539-304-4494|3043.20|eposits kindle carefully. unusual packages nag regular, daring theodolites. furiously final pinto be +3812|Supplier#000003812|Vq0LLMinZe,pYmrW|8|18-393-997-2833|6099.30|leep blithely. blithely regular instructions d +3813|Supplier#000003813|YMAdHiuuX bevB5|1|11-184-678-5219|16.61|solve? carefully final packages are slyly around the +3814|Supplier#000003814|vTnXsG9KzNlsiO6tpbL6fp7|23|33-280-183-9315|2172.42|efully regular platelets. fluffily slow asymptotes alongside of the spe +3815|Supplier#000003815|X6Hv7E2gTir0 XMliDjn1NxTzDCiK|2|12-795-283-3574|767.18|the quickly final platelets. final, regular requests sleep: unusual, even pinto bean +3816|Supplier#000003816|Aw0PyGoFM7SgOM2Wted EPgidEfH|1|11-268-642-8469|13.06|. frays around the special, final excuses haggle after the carefully special theod +3817|Supplier#000003817|hU3fz3xL78|6|16-339-356-5115|8441.40|ely even ideas. ideas wake slyly furiously unusual instructions. pinto beans sleep ag +3818|Supplier#000003818|KD65BupNh6SuX9EFebFNWoO|21|31-883-696-8085|5737.32| deposits. regular, silent packages are quie +3819|Supplier#000003819|kfZQU 0CGcF43hgGfVYHSVBaR3FD|12|22-346-938-2374|-918.04|ng to the asymptotes wake quickly ironically bold instructions. special, +3820|Supplier#000003820|zsuoXVzqz,s|24|34-178-282-5651|1461.57|ag slyly. regular, ironic deposits nag quickly of the regular deposits. pe +3821|Supplier#000003821|XbIPHWen658Kd,9MGe|8|18-239-284-5694|-473.95|eposits are against the fluffily pending foxes. packages use slyly dependencies. depths nag caref +3822|Supplier#000003822|XHhC 55Ip6UIed8ZtMjaEmOzeg0W2s0wYk3ew|24|34-169-166-9819|7192.72| regular dependencies. even, regular theodolites cajole doggedly. r +3823|Supplier#000003823|4ex,LmqMHTsTXzQmD69P6hcH2AYo9z31g|6|16-321-642-8507|7009.72|olphins. quickly final platelets are carefully. slyly bold instructions around the express excu +3824|Supplier#000003824|m7fIwVues7ktkv5aSrQz1BJCAcnZXzNm|0|10-997-713-3641|429.61|hely instructions. blithely even accounts detect about the carefully special dependencies. bo +3825|Supplier#000003825|hK1aUlbzeTz MSPwcPVyRGY|6|16-851-552-6405|2437.45|g epitaphs. slyly ironic asymptotes cajole carefull +3826|Supplier#000003826|2 UlGzry,3W5HFyu94xTXli8OSxX|8|18-334-808-6009|134.05|hely ironic accounts. blithely final tithes detect slyly final accounts. +3827|Supplier#000003827|uQ6qsZc0UbA,9 B4|0|10-390-421-9681|6904.16|ove the permanent, special courts. slyly regular requests +3828|Supplier#000003828|fiYZWClYfiCjNPsjrT6FGI7cQKtDY14vkdweiDZ9|9|19-603-288-3421|8297.49|y even platelets nag fluffily! quickly unusual depos +3829|Supplier#000003829|RLYte,by6vV9TWHXOJ|5|15-644-782-9969|6993.21|ests wake quickly to the blithely regu +3830|Supplier#000003830|S6RWDUy0ZWuua1pk|19|29-347-363-8660|8865.17|s are. final deposits cajole quickly +3831|Supplier#000003831|qvU9EGNzoBF5fvW3ckO7fMzVP60k7|17|27-846-413-4861|29.61|jole. pending, silent ideas cajole above the furiously ironic instructions. carefully ironic account +3832|Supplier#000003832|JXlbKeJULboOm6nrcae6Cc0nLHQJPDZ2iTKq|14|24-658-312-4516|523.67|requests. fluffily ironic pinto beans wake slyly. pending pinto beans nod bli +3833|Supplier#000003833|8A473YqK3RNPeqgRk5GoWsx|17|27-405-586-3570|-657.08|arefully regular requests. furiously regular accounts use carefully theodolites. quickly ironic +3834|Supplier#000003834|cxGQnW3nShks59xA5bPoaC bIvcBWUt|20|30-823-353-6520|9318.47|regular instructions. express packages run slyly pending +3835|Supplier#000003835|5q2U5RvcC5|9|19-120-380-2806|6608.79|yly carefully final requests. requests detect furiously slyly regular dep +3836|Supplier#000003836|tdBz4J0l7wDJJu Dej1|6|16-958-229-2290|5049.32|regular frets. express deposits sleep furious +3837|Supplier#000003837|SYXpXaKop3|21|31-780-420-4359|8011.69|ular foxes haggle accordi +3838|Supplier#000003838|a1xM3fFNl0z6KFqs1tScaSyBRQf2d4kE|16|26-371-108-3599|-467.39|y against the quickly final foxes. dugouts solve t +3839|Supplier#000003839|PxJGn,bL7sv0o,SaU Uzt|5|15-949-119-2553|7531.51|d. express accounts sleep slyly after the fin +3840|Supplier#000003840|QRgbi,RFSbtKb|9|19-998-455-1093|3075.20|ess accounts according to the blithely final de +3841|Supplier#000003841|gZRwdTLh4L9uSkOc|1|11-315-160-3060|4900.91| blithely bold accounts. excuses are after the +3842|Supplier#000003842|AW6OymN4xrhxYKaP|23|33-490-183-3410|-437.60|ly even theodolites sleep slyly alongside of the regular foxes. +3843|Supplier#000003843|y 5Wl63UOfOUpa4l5no8poPlQ4PJJPlKE7s|0|10-431-239-1380|2189.64|y closely ironic accounts. final pinto beans are furiously. furiously b +3844|Supplier#000003844|flYrwpli7eUMIv|13|23-857-948-7743|4302.26|oxes. even depths are furiously against the slyly regular theodolites. final asymptotes doze a +3845|Supplier#000003845|B2DZOdIZ43ELcKgJV|23|33-908-326-9389|3150.92|cies. furiously unusual instructions solve. furiously express ideas sleep slyly about the furiou +3846|Supplier#000003846|wl076KfcEpYLRegb1LfIf93b3n5HBabFK2R,mEM|21|31-407-304-1018|1549.03|ages. deposits haggle alongside of the carefully pending d +3847|Supplier#000003847|fZI7MkOKUxtbb|5|15-797-203-6297|6517.55| pinto beans wake furiously blithely unusual instructions. furiously express acc +3848|Supplier#000003848|ukVhf2GhFD2CT6RXEhX9npo0P7bE6g1U|6|16-970-968-4257|468.09|s. pending deposits haggle quickly. fluffily thin packages are even excuses. bold instruc +3849|Supplier#000003849|KgbZEaRk,6Q3mWvwh6uptrs1KRUHg 0|3|13-582-965-9117|52.33|hely pending pinto beans are carefully about the ironic, final packages. blithely express deposi +3850|Supplier#000003850|,27mYEAukUi JHLAjUTMCX3hkL8uzcq88|6|16-354-949-7591|8399.64|nding warthogs integrate carefully. bravely final ideas wake about the ironically pe +3851|Supplier#000003851|7LAScolWqqf1CjlgKMGgH|16|26-424-853-9777|7803.29|g slyly bold deposits. furiously bold ideas are slyly above the fluffily final theodoli +3852|Supplier#000003852|H,903oWK0uBieFmm OeFLwc8mn7Gd2YBl|7|17-310-721-3486|5873.81|cross the slyly pending requests. blithely ironic requests sleep sl +3853|Supplier#000003853|,XhDB3dz2sL1PFx2BR4NnSVsoA sBiqj8pJKHvj9|0|10-974-583-4268|5196.75|ously regular accounts. fluffily express theodolites use. carefully even theodolites nag +3854|Supplier#000003854|thO,6KxFMs5NYmRZCC|11|21-697-869-9271|-153.70| asymptotes haggle blithely regular packages. finally unusual foxes boost slyly along the q +3855|Supplier#000003855|S7vYVTJ6TDH4,KAZ8Cs|12|22-224-350-5888|-273.32|even ideas. ideas play carefully; +3856|Supplier#000003856|z2smebcttB1A s1ZA|13|23-420-921-6150|2155.00|s cajole furiously quickly s +3857|Supplier#000003857|4BbobbQPcWShS5AkbNSBEYSp3GtqS,|17|27-472-324-5460|5130.62|phins. blithely regular deposits after the pinto +3858|Supplier#000003858|1jOstK 5zRYrKcTTsjYpN 0,MrBTKp|12|22-631-836-3694|-708.72|quests wake slyly. ironic requests boost slyly about the carefully bold ideas. furiously +3859|Supplier#000003859|E3g IbuGPaCIZ3grVU95Kfr7HzjZ|1|11-545-322-2116|-476.92|y among the slyly slow frays. blithely final i +3860|Supplier#000003860|hHRLypDxFb eNAcuBcE|14|24-408-465-3866|1330.91| unusual pinto beans? requests impr +3861|Supplier#000003861|kiZ4YdXV0Ah,kRkyIJ17R6Qh|11|21-850-998-6046|3498.39|blithely regular theodolites. sly c +3862|Supplier#000003862|0XXFhF1IDBh|21|31-641-472-3724|410.70|ve deposits sleep furiously about the slyly final epita +3863|Supplier#000003863|2Zv1tuSpNP,cO|7|17-221-510-7174|3529.66|ts. regular dolphins cajo +3864|Supplier#000003864|G4A7fDgFofmhL1lXcnG,Vv6qkWuzwil|22|32-230-963-7050|8998.95|e the unusual requests. furiou +3865|Supplier#000003865|FJgZRN1aro,smB3bAhicC|16|26-569-545-2780|4490.15|riously among the furiously pending packages. furiously final ideas haggl +3866|Supplier#000003866|F d5ptZhKCBX pMd1 QOZzS5Sxs62By|7|17-981-521-4757|3953.50|cording to the doggedly quiet theodolites. unusual, final gifts +3867|Supplier#000003867|Vl0it8GSnzR3rthsVcI5bk2qxEi|0|10-523-466-5029|4524.31|tegrate blithely slyly unusual pinto beans. ideas haggle blit +3868|Supplier#000003868|5aP4VBn0t666NbGYB|21|31-503-515-6191|6105.00|s wake furiously above the packages: fu +3869|Supplier#000003869|,HZ,uHgmGOyUSyXE8A5FKm|18|28-193-913-3630|7720.17|special deposits are quickly. unusual warthogs kindle fur +3870|Supplier#000003870|Q8mzUM5BaMsDOolmRXuAIhwY|23|33-796-500-2325|6020.98|y express accounts wake around the quickly ironic asymptotes. regula +3871|Supplier#000003871|0l9vr6DDagytSNo,SD2mkG4XezH8L1loBW7uXV|0|10-678-935-9330|3597.63|ove the pending, bold warthog +3872|Supplier#000003872|fk2YzCQeGM5Pt5gLFlobjSby fBApdl5qwVy2|5|15-185-891-8461|7436.46|f the fluffily special packages sleep dogged, ironic foxes. blithely silent deposits cajole +3873|Supplier#000003873|SfOTvlK6qYvJRvizMC|2|12-487-167-1621|623.49|ng to the accounts. slowly express platelets cajole blithely brave, unusual accounts. courts haggle +3874|Supplier#000003874|CY,arVY8zA5kT7KrawlZ|17|27-340-601-8395|4865.53|ronic requests. regular, ironic ideas integrate carefully whithout the silent +3875|Supplier#000003875|q3HGPD9NYXaxYX4Yfc, M |14|24-741-527-3990|-979.44|ay integrate carefully. regular pinto beans impress; silent, pending deposits across the +3876|Supplier#000003876|QwBoyz1REFfvdFi8,3Xa1xPVyYh9Q4NL|18|28-339-544-9583|3464.35|express platelets. blithely regular +3877|Supplier#000003877|5GY4QEnWvFTzXvBPiLofYMqNIP Omj7T62ag4Qr|9|19-132-153-1633|371.77|ffily express asymptotes cajole. o +3878|Supplier#000003878|Qc0XoSuAoQs 8rbd9nC2,RKJJZRf|17|27-313-774-4161|9197.41|s use. furiously regular frets along the bold, express foxes haggle +3879|Supplier#000003879|vm0IFVSdSgmXgskLojAanP46aslgr1pDEus|18|28-440-312-1631|6871.99|ckly packages. bold theodolites boost furiously +3880|Supplier#000003880|DZo80mSznrhCpb8|21|31-253-201-4873|6584.15| even instructions. sometime +3881|Supplier#000003881|vmqfjM73hPvUEaE3dR0ITCrqUDl4ccNd77Q|5|15-818-549-6778|8865.64|ular, regular accounts. regular foxes wake blithely accordi +3882|Supplier#000003882|4fAALx0klt0jiU7BJSOk GK1WBarg3GKZdeVf|24|34-138-314-4773|7932.67|s integrate fluffily at the carefully +3883|Supplier#000003883| yLPQ474SMseqoOTY2ff5p5kTiwXjy4q0RzGm|16|26-268-102-8092|2382.14|st furiously bold deposits: fluffil +3884|Supplier#000003884|reY pD5laTjWSCweVOf7OVU89yRYx1 qU|2|12-990-865-1875|5597.26|thes x-ray furiously. furiously pending depo +3885|Supplier#000003885|M i ERhP9U FS,0Hi8O4|22|32-209-102-6679|-778.30| carefully final requests. furiously even theodolites integrat +3886|Supplier#000003886|c,VP75QAJqi ig1gNOWsxXdsuZQbi4ZJ5 3|24|34-418-995-7074|6965.13|oggedly bold instructions. thin accounts e +3887|Supplier#000003887|zufaFycuBCn,E3DV,X7|9|19-897-428-7137|-964.32| along the thin, even instructions. regular, final deposits nag carefully according to the +3888|Supplier#000003888|1jH4h28NcC|12|22-388-795-7649|4665.15|s. foxes wake sometimes above the thinly silent packages. fluffily +3889|Supplier#000003889|xEsTSa5oG8BjL0SJdEdCcTg2LhOguyvQ|14|24-804-460-5552|8123.70|g packages haggle according to the quickly regular frays. carefully even accounts wake f +3890|Supplier#000003890|ypnljNgJOfCnSUndIlWsZ|14|24-737-836-4540|1042.13|g pinto beans sleep deposits. silent, regular pinto beans cajole carefully pin +3891|Supplier#000003891|f5Kj1yA5MABa0NYCXt9mVfmngDB|8|18-224-804-1070|961.53|ecial dependencies according to the express deposits are blithely expre +3892|Supplier#000003892|7upn3 0JxQtolUElV7uffY|6|16-356-490-9592|-701.24|ss. final, regular dugouts cajole after the fluffily ironic theodolites. furio +3893|Supplier#000003893|czTNFPTvwoGe,lZxSzf00Q3kR9,i|18|28-174-729-8408|4214.91|its. pending ideas wake. blithe, regular deposits will sleep bold ac +3894|Supplier#000003894|vvGC rameLOk|3|13-295-878-3930|8996.14|side of the blithely regular deposits. slyly express requests among the slyly fi +3895|Supplier#000003895|VtxUIe5P3tzk3ykaq7mVdEzO5fqllBcxpEHkG|24|34-712-627-7168|1219.48|finally express requests wake blithely deposits. furiously even reques +3896|Supplier#000003896|mWr2VTZVOlAdk6RXnSAPSNHL3Xdu3FfnrPQh|19|29-261-495-9146|709.99| above the packages-- foxes against the final packages are carefully slyly pending pinto b +3897|Supplier#000003897|AOtckHviCl 1XE0HfVh|0|10-907-392-9194|6804.26|ly regular deposits integrate fluffily ironic, quiet t +3898|Supplier#000003898|m72eJDVjdCxG7EoomOvmbwQAU rj|22|32-821-776-9171|1165.59|r grouches boost carefully blithely unusual asymptotes. furio +3899|Supplier#000003899|sLSuM,Ce4yHgzs4llKKAX79S|7|17-340-713-7717|8096.51|r asymptotes nag past the fluffily final deposits: blithely pe +3900|Supplier#000003900|CD6mwylfDc2CrjrEKzSbK9LbGEC9irn|24|34-833-259-2802|6176.45|riously final packages. dependencies wake quickly. regular deposits sleep furiou +3901|Supplier#000003901|fPUFeXE2Q QjicA09isYtmSve8PC0ie,AnxHRWTV|12|22-857-383-2685|9691.08|fter the furiously ironic accounts. bol +3902|Supplier#000003902|94Cum0,H 0|3|13-218-944-2974|-859.70|nic, even foxes. slyly special accounts cajole. furiously regular requests sleep q +3903|Supplier#000003903|9ngFBw8HJytf|15|25-891-797-9185|2958.76|arefully even foxes. dependencies against the quickly final ideas wake +3904|Supplier#000003904|4odTocND0hC9gm|17|27-284-415-6947|5810.71|the slyly express ideas. carefully special +3905|Supplier#000003905|WHSmIJjQYgfM9Vtdkvris4MX|4|14-587-899-9591|618.68|ldly foxes. quickly even pearls according to the requests +3906|Supplier#000003906|742VDBn2PvQRZpqT7I9tlTGDvqg|11|21-239-494-1132|1052.71|y. requests haggle quickly final excuses. special ideas wake a +3907|Supplier#000003907|cN6jbd1 q6v3PDE|19|29-335-916-5312|2834.61|uffily unusual deposits. even packages haggle carefully regular dependencies: fluffily express pint +3908|Supplier#000003908|Okl1FHH574YroExB|23|33-154-728-5624|9277.30|thely regular packages. quickly express foxes integrate regular accounts. blithel +3909|Supplier#000003909|TjJ21ZlzfN|11|21-203-379-9975|7659.41|final theodolites maintain alongside of the ironic, express requests. package +3910|Supplier#000003910|KDNx2edzTSU6 Az9A|17|27-166-548-4497|7673.62|ronic foxes. dependencies over the ironic p +3911|Supplier#000003911|FC1wUhUk99QJ8|14|24-968-475-6184|4593.17|s. even, even pinto beans affix quickl +3912|Supplier#000003912|Op7,1zt3MAxs34Qo4O W|11|21-474-809-6508|8473.01|es. regular, brave instructions wa +3913|Supplier#000003913|LdloqMCaGGbbgNkLpyQoCTktj5GqnVwyBt|23|33-945-264-1748|8991.16|quickly. foxes affix furiously unusual, ironic foxes. instructi +3914|Supplier#000003914|uBG4kbR1mp6LQZCp6DlCmpzlw9sh2XnSPMT|22|32-389-834-9741|5562.66| blithely ironic requests are permanent accounts. regular deposits haggle carefully +3915|Supplier#000003915|Apr7orJmH6UZR3FrVVrrtY2c1x6kt2z2S9w0|2|12-111-834-6302|664.90|e theodolites. slyly even pearls doubt furiously final packages. slyly br +3916|Supplier#000003916|80e d7a,oyePPMfCAW AhGe|1|11-405-825-7031|7977.20|e blithely ironic accounts. even dugouts detect slyly-- unusual accounts sleep slyly. unusua +3917|Supplier#000003917|Bt0VKR1pfFZrLQKliZ2|13|23-283-379-7778|2875.05|ts unwind quickly toward the furiously fin +3918|Supplier#000003918|meRvRCsJoAbfqd0Re4|3|13-786-859-6355|6950.16|regular accounts. slyly pending asymptotes after the idly special +3919|Supplier#000003919|uKT4wOITgzQI,tCG3p7R|8|18-953-307-2993|413.72|y above the even instructions. slyly special deposits wake above the requests. unusual +3920|Supplier#000003920|AxtQY7SssyEibvtT29eBgbhEb3Dtt2iFW|5|15-691-940-4136|-727.91|slyly express waters. packages haggle furiously un +3921|Supplier#000003921|0Tb66Vdro2XWujs2Ya|15|25-489-165-2757|7010.68|ggle carefully whithout the special, regular pinto beans. blithely fin +3922|Supplier#000003922|uZbzFZ8,tZEecLPWporrD6XAODlIaLBn78cKh|22|32-602-676-2354|5464.31|ccounts. special, regular packages boost slyly. somas sleep thinly across the fu +3923|Supplier#000003923|,2BXRxonGyY5B9|15|25-212-319-9550|8715.48|s print about the furiously even accoun +3924|Supplier#000003924|D4SSJXYsvf3PbLcImfJ1n0f2mDZzN g,f6q0Q|4|14-218-561-6073|3096.19|pending requests after the ironic requests integrate slyly pending requests. closely un +3925|Supplier#000003925|Wef7Ha,mMw2ryb2pGBXzHT|8|18-955-286-4346|5587.70|counts. slyly silent hockey players haggle quickly. fluffily close asymptotes sleep slyly among t +3926|Supplier#000003926|iEskW4EqJRr6lmnL56,D3hfohT72hP4Gx4AU|9|19-842-211-4782|3477.34| fluffily ironic packages. blithely even dependencies according to the final ideas +3927|Supplier#000003927|fASJO,Hi6pi1wp3ExJCttf3GGZuewRkoBW8V|23|33-835-635-4974|9668.95|fully silent deposits. pending, final deposits wake according to th +3928|Supplier#000003928|fuu,slIkrxNz8|14|24-914-931-3627|1110.10|gular requests use carefully. carefully ironic accounts sl +3929|Supplier#000003929|q0,VNJd1TvQAR6Lpt5FnuYC|14|24-953-294-9573|-544.15|. deposits wake slyly alongside of the final accounts. slyly f +3930|Supplier#000003930|k6G9l,i3lv4oGkhHyVadfACjIvsw,VD6o|24|34-351-898-3665|4396.43|ts. carefully express pinto beans are ironic excuses. silently ironic packages cajole final deposi +3931|Supplier#000003931|dkqDyoyHmywcP9RkjZEbwVYCWetbPX01|0|10-890-887-7011|1821.58| blithely above the slyly bold asymptotes. carefully +3932|Supplier#000003932|NNsx7jMngxxQhii1cU9ZQm|23|33-396-117-5168|292.35|endencies wake carefully after the quickly even acc +3933|Supplier#000003933| 64JfUPlS9GQN4 WQG2Ctyq7N|10|20-167-303-2899|5468.97|y above the furiously ironic instructions! blithely ironic accounts are carefully ideas. +3934|Supplier#000003934|,KA9SfSroSuN5YMb46WxIzJQA|9|19-683-401-1348|-994.73|lites wake above the even deposits. carefully pending requests kindle f +3935|Supplier#000003935|rDYYLkD,3wGqNVTo0TdmV8EJlDwwQbnYlHgZ1zVH|8|18-547-201-3057|-475.39|le quickly. quickly regular r +3936|Supplier#000003936|QCBRQcSyomEn7G78pdsb3o,0|23|33-442-349-3167|4997.79|s sleep excuses. regular, special accounts after the platelets cajole +3937|Supplier#000003937|kqEOwdVW,qJsJdcv6PwDJ6ii14mugDK3OgZN ngI|7|17-621-453-7063|-63.88|y pending asymptotes. foxes are. deposits sleep quickly b +3938|Supplier#000003938|lVx0lB8r9ZnKIPsSTJnnSsiqpnwnz6S|18|28-643-172-2015|2289.05|unts cajole slyly. carefully special requests whitho +3939|Supplier#000003939|NFxpr5yBocaAYJg84CIsv8GIdKt|5|15-634-278-8420|2983.31|refully regular pinto beans. pending requests mold along the carefully ironic hock +3940|Supplier#000003940|ScNf9cfd8g7RdcrtpU7Afk8NXgASA|24|34-725-479-8331|464.74|eposits are slyly slyly pending packages? furiously regular packages try to affix amon +3941|Supplier#000003941|Pmb05mQfBMS618O7WKqZJ 9vyv|3|13-668-659-3631|9900.08|ithely quickly bold accounts. bold pinto beans about the furious dependencies cajole qu +3942|Supplier#000003942|woBReojDp I7QB6G4T7,qRimIA,cLU|16|26-753-926-4210|2654.16| ironic packages. slyly even accounts use carefully according to the packages. blithely regular +3943|Supplier#000003943|k,5rvnqP14,iPFU7ezbrr1beRq5|6|16-898-613-3470|1517.87|fluffily quickly express ins +3944|Supplier#000003944|lNZ3SumitRq98Z4Lhsd|4|14-872-560-1945|-160.20| regular accounts are fluffily never regular excuses. even, even pinto beans use fluffily. +3945|Supplier#000003945|ySKYJurtApuknjXIa0h|4|14-858-866-6572|460.13|luffily according to the pearls. carefully thin +3946|Supplier#000003946|PRJJlZVVizUPiKY7w,TeeB5HoAiPMxzPUqx|15|25-819-326-7248|9424.19|into beans wake blithely around the ironic, final pinto beans. +3947|Supplier#000003947|lHoIZSTuK0sJxoQ,xC0XVrbd6NAbRJ,6H|2|12-617-332-9152|1366.49|es use alongside of the accounts-- furious +3948|Supplier#000003948|XYiD 7s4dikkL8B9|5|15-543-886-7762|7527.65|. special, pending packages sleep carefully. special pinto beans are blithely regular theodolite +3949|Supplier#000003949|vmuyF Svg3liRwxY|20|30-602-767-2188|4242.84|inst the deposits. unusual, regular +3950|Supplier#000003950|w8E7psTO2Bx8C6H,AFeXFWsjKmLQitdq|17|27-402-593-2481|5436.13|re furiously ironic pinto beans. quickly unusual accounts are carefully across the ironically expres +3951|Supplier#000003951|qajkuLHgP cRkdooPutjgKa8zxnYW|8|18-718-405-7747|7992.60|fter the quickly special theodolites. fluffily unusual pac +3952|Supplier#000003952|4gMQ45hKo4sVE X28MV|11|21-380-318-2207|8715.41|sly about the unusual instructions. pending requests +3953|Supplier#000003953|SjM LAIjh8Vp|18|28-338-515-2433|1881.37|ounts integrate blithely final accounts. furio +3954|Supplier#000003954|x6cRxg62 t7dRyxvC|23|33-100-296-5999|4160.69|iously pending requests cajole permanently above the unusual +3955|Supplier#000003955|piECPB8qbn7s3XP|21|31-861-222-6689|-336.37| blithely express packages. ironic dugouts are furiously quickly even pinto bean +3956|Supplier#000003956|r0QOyhOZ ov 7Dn|17|27-326-227-8739|9536.82|s. blithely even theodolites sleep. slyly ironic theodolites mainta +3957|Supplier#000003957|d yi8lsdOXX48fHJX5iTV0|15|25-673-681-9031|2446.74| sleep blithely unusual packages. slyly close patterns sleep alongside +3958|Supplier#000003958|imTYciDxoeTAlCaCCbnw JbHCMVIW3Duxn|2|12-707-932-2924|2391.99|s. carefully pending ideas nag furiously foxes. furiously express theodolites haggle carefully. thi +3959|Supplier#000003959|4Lhe94Vyn47R|14|24-291-433-8219|6084.06| even instructions cajole furiously bold +3960|Supplier#000003960|gTlGLliBZ5tejTU|23|33-532-108-8148|7761.36|ep across the special deposi +3961|Supplier#000003961|rOZqLwnMyYacgiRb2D|2|12-971-170-5168|5224.94|, regular accounts. unusual, special accounts affix. +3962|Supplier#000003962|oNM1okCwUVKFOzke0ol7iDviZa|18|28-618-658-8737|502.97|mong the requests. regular, bold requests sleep furiously. +3963|Supplier#000003963|r8VL5GKQMJTx|7|17-350-945-1631|3429.30|posits. accounts after the quickly +3964|Supplier#000003964|,sEzbwQRqZYvpRZP9my|16|26-534-139-7853|5195.91|thely. pending, special ideas sleep quickly a +3965|Supplier#000003965|gCrNvnltMP,OjeDDPSP5JUoK,wutPQDg|23|33-738-115-2838|7725.80|sly instructions: slyly regular requests among the silent, pending deposits are fluff +3966|Supplier#000003966|TZo,pamBUlr2|19|29-705-909-9615|1618.55|gular pinto beans nag. special ideas sleep. +3967|Supplier#000003967|eHoAXe62SY9|19|29-194-731-3944|8920.59|aters. express, pending instructions sleep. brave, r +3968|Supplier#000003968|pHlakj,V6Cxz3IaM1y0ZNBY|6|16-826-121-5142|7389.60|final dependencies. fluffily unusual accounts cajole carefully. quick +3969|Supplier#000003969|jTZvvV2MU7hqLZiI6ZqufbfGX1|16|26-329-999-9943|4986.19|nstructions. quickly ironic pac +3970|Supplier#000003970|c3w93u7z,OFZYuQJJeBn9vdBTjAHvVjlRPpZufD5|13|23-727-929-4919|2979.92|n foxes boost quickly about the quickly regular pinto beans. regular asymptotes sleep slyl +3971|Supplier#000003971|rmppI,ycgDGtReQ8ME0f44woPCuaceAIFjsl6B|9|19-685-551-8039|9493.43|lar accounts wake slyly according to the quickly ironic forges. special, special accounts sleep pe +3972|Supplier#000003972|KQu4 mgLZkaEiNGlkOYz,p|4|14-901-775-3308|-801.08|, ironic accounts. ironic r +3973|Supplier#000003973|d18GiDsL6Wm2IsGXM,RZf1jCsgZAOjNYVThTRP4|6|16-722-866-1658|9249.35|uests are furiously. regular tithes through the regular, final accounts cajole furiously above the q +3974|Supplier#000003974|TxJxG8f12BC8F|23|33-207-983-6026|-87.05|ges doze quickly. fluffily ironic instructions beside the bold theodolites sleep against +3975|Supplier#000003975|CT9,kyMKXk6XPAs13gMhMs6Ist,FT7zA|10|20-270-703-9793|8880.71|gle furiously multipliers. express warthogs use fu +3976|Supplier#000003976|FXTXVXGZecnK2LMNzyv,suyHOxJ hQ|3|13-782-169-4167|8083.33| carefully final deposits wake quickly ironic, blithe asymptotes. regular +3977|Supplier#000003977|hcq,RUMaS5aQKKqG8KULXMb9aMhzPMNYdsvofo|24|34-959-969-1364|4656.74|al packages sleep bold asymptotes. foxes boost silent, ironic theodolites. carefully ir +3978|Supplier#000003978|OwxlJz9EjmwI6r0MDi,dhQdSCzeshloJHbnB93|15|25-425-505-1375|9831.05|slyly doggedly regular pinto beans. platelets snooze against the idly +3979|Supplier#000003979|BfmVhCAnCMY3jzpjUMy4CNWs9 HzpdQR7INJU|19|29-124-646-4897|8553.82|ic requests wake against the blithely unusual accounts. fluffily r +3980|Supplier#000003980|wUUr1a2E17V|23|33-758-679-2661|3037.36|, final dependencies cajole slyly against the furiously regula +3981|Supplier#000003981|mhzV86Pr48Wwb 4JDJbULR44SDx3nfiDKu|10|20-600-189-6010|6879.24|arthogs use blithely slyly ironic deposits. express ideas according to t +3982|Supplier#000003982|jx9EloF33Ez|11|21-890-236-4160|7376.11|s the furiously special warhorses affix after the car +3983|Supplier#000003983|51aAi CtErnk,wAHIVjVRJMv4QH2YPEW9|18|28-683-215-9377|9251.05|s. ironic, express epitaphs are after the furiously final accounts. patterns poach ruthlessly. +3984|Supplier#000003984|,ppvG12P82IlWno|16|26-873-556-7574|6521.70|ccounts wake according to the express warhorses. ironic packages are a +3985|Supplier#000003985|iq8EWKwGvJsKur1puSAnm cbptVohbwE|2|12-723-867-5722|4320.70|tainments sleep above the regula +3986|Supplier#000003986|V6GVF6ZxemZ3TV18cmocGr6GdYhb|0|10-973-297-4698|4957.46|riously regular accounts. regularly even platelets haggle carefully along the caref +3987|Supplier#000003987|sGkiOHDoPgJmxMHjviJ0tMkUW|24|34-889-317-5802|4025.78|asymptotes. carefully final inst +3988|Supplier#000003988|MF7EzpHHs3IpUsAZ1cB7M9lxYuqQUWUO|24|34-705-531-6512|9239.24|ons. quickly bold gifts wake furiously blithely ironic foxes. blithely special frets are s +3989|Supplier#000003989|EnaROLO3tkWD9jrnfm826yp8uTc5EKhxQ|12|22-295-616-9440|5970.22|haggle. blithely even instructions cajole furiously: +3990|Supplier#000003990|wehBBp1RQbfxAYDASS75MsywmsKHRVdkrvNe6m|22|32-839-509-9301|8432.89|ep furiously. packages should have to haggle slyly across the deposits. furiously regu +3991|Supplier#000003991|aAhi4BFN6rbiczk|9|19-724-962-3506|5688.95|nstructions along the excuses c +3992|Supplier#000003992|lTYs,HhjZf,j6TM8AC13vn21uT4ykyXd|20|30-573-866-6706|195.49|ithely bold escapades are carefully regular packages. deposits +3993|Supplier#000003993|UAcFXUkNgI0r9gjUYdqQ1JS9T|4|14-720-330-9788|5573.05| blithely along the silent excus +3994|Supplier#000003994|W00LZp3NjK0|3|13-767-429-1732|-712.17|eposits lose slyly furiously pending escapa +3995|Supplier#000003995|ziDsQHfuoMyaCJhv4WJC7SlqjOOUyFMBeWOOMz|7|17-630-620-2132|7548.48|blithely even foxes. furiously ironic deposits nag alongside of the regular deposits! unusual +3996|Supplier#000003996|9x8T RaR4hEaoI|21|31-259-816-1258|2274.53|wake daringly about the furiously even hockey players. fluffily unusual packages +3997|Supplier#000003997|GnRELJY15Z6SR6rZXXL|0|10-859-139-8958|1024.91| instructions nag. quickly silent dolphins cajole af +3998|Supplier#000003998|s8yatBEk9U6HRd,4qQkfiEblD7|12|22-499-388-8247|8610.75|cial theodolites. regular, special excuses ca +3999|Supplier#000003999|v53dfNGjAGBIdki8QXsVwXyF8|22|32-802-512-3693|3007.34|ithely. busily even ideas wake blithely according to th +4000|Supplier#000004000|q bLQQi5J,rohLugmz|24|34-900-705-2420|8368.23| after the quickly special requests. requests us +4001|Supplier#000004001|3M39sZY1XeQXPDRO|13|23-109-632-6806|7885.17|efully express packages integrate across the regular pearls. blithely unusual packages mainta +4002|Supplier#000004002|MbWjR,serF9TsLjrAnK|0|10-570-768-8782|7034.56|theodolites are fluffily across the special requests. blithely +4003|Supplier#000004003|7w3Uyzi 6Q,e2 fJrE60Y9HFhPfBM|21|31-655-482-7421|-172.97|s nag silently blithely regular asymptotes. ironic, bold dependencies lose slowly furiously regu +4004|Supplier#000004004|YGKTXvmYNpxoqdllFqbQjnv69Ydtm7YpUiqQkd|1|11-463-394-3524|5878.41|the dependencies cajole along th +4005|Supplier#000004005|V723F1wCy2eA4OgIu8TjBtOVUHp|3|13-927-435-3990|5497.97|structions mold furiously. carefully expres +4006|Supplier#000004006|eialYLhP,,gkxUf5jbsaSk|9|19-305-605-6821|2342.08|lyly. blithely regular Tiresias sleep slyly; fluffily final accounts +4007|Supplier#000004007|cvlSgCCKGOwpaB iFIPx4vU2qA5b6K hz9Z91|21|31-218-453-2389|3361.70|s asymptotes use across the furiously even asymptotes. busy waters haggle. blithely iro +4008|Supplier#000004008|KwUFb,C, JmF|22|32-237-882-4487|2139.67|ecial accounts was quickly carefully final f +4009|Supplier#000004009|rMbsZhBvnrVGRFRVQ1,KcJLXgOL|20|30-654-164-8626|322.65|lar asymptotes. quickly pending a +4010|Supplier#000004010|hzXpAwbeH0D|12|22-191-309-4339|8772.78|deposits. final deposits wake furious +4011|Supplier#000004011|OD8TJY5h ne3M4Q8W3|15|25-941-875-5346|9185.36|y regular foxes wake slyly. unusual packages integr +4012|Supplier#000004012|DPIV2MOin9ulkc4AZfLZ0 VOLzp1ohojo|4|14-366-631-2930|2395.57|lieve past the quickly regular dugouts. carefully final accou +4013|Supplier#000004013|,6B4FqtOOsFeJeDy38eNaRkAr5Jm5CXKthK|14|24-221-110-6139|364.06|ly about the final pinto beans. even theodolites nag carefully above the furiously +4014|Supplier#000004014|Spla6 0RV3oK9Y|3|13-587-428-2644|772.46| deposits? slyly bold accounts wake carefully above the carefully special packages. f +4015|Supplier#000004015|VmHTIcPN83tZ oVSyamcgH4GgnUXsvX0TWGq|13|23-689-976-9856|602.57|ons boost furiously final deposits. fluffily ironic pa +4016|Supplier#000004016|rxIU3SVvxD3en5QsvEkjq|22|32-312-878-8329|5413.62|sly regular platelets cajole according to the sometimes final requests. fluffily final platelets wa +4017|Supplier#000004017|ryVd3twx7WnifLPEWrZCIqap|24|34-797-367-3657|3663.98|y bold packages thrash ironically according to the carefully unusual accounts. +4018|Supplier#000004018|LxJTnYMv2NyIRCI1mJgY0rnVNRZ|6|16-616-621-1011|415.20|ng to the ironic requests grow above the furiously unusual requests. bold requests sleep fur +4019|Supplier#000004019|MdV7gEsO8dEF Z2OSSeZaGrNOLNv11Lm4|14|24-363-949-7620|2216.34|. instructions at the final asympt +4020|Supplier#000004020|67I11uSvIB6LsbRxzpGTOsW3diE3M23|19|29-484-433-5619|2038.28|efully regular asymptotes. final, final requests x-ray carefully quickly fi +4021|Supplier#000004021|Q5OtYOoYjsSKUtYlf7DSl,757|7|17-355-893-5738|1280.85|ecial deposits. furious requests engage? fluffily unusual theodolites ea +4022|Supplier#000004022|hBM8bCaPoIiubGfQAWhkCC9TLsgZAE6Wd0O0ap|24|34-922-888-1759|9167.84|olites nag bold theodolites. requests boost slyly even package +4023|Supplier#000004023|tVt87IttC0pixFV,kybPakPz1Hal, qMDfOo5|21|31-163-966-8630|5031.87|d deposits cajole. blithely express theodolites sleep ironic sh +4024|Supplier#000004024|kl3Fg4mghjGogro1OYGt|11|21-140-220-9155|620.12|. slyly bold excuses alongside of the b +4025|Supplier#000004025|GaSoP57h,H3bpBq7PysoP5vVKH|16|26-724-854-4072|3360.41|e furiously bold accounts. regular waters cajole carefully. fin +4026|Supplier#000004026|I692fFGW Wyp5QvrFLsI6K|11|21-477-196-7874|2367.75|arefully since the grouches. slyly final pinto beans nag instead of the regular +4027|Supplier#000004027|RmLtqBJW5X9Q88Vs2W6|12|22-932-823-5365|5690.56|its maintain. carefully regular foxes +4028|Supplier#000004028|Lz,ZhzL1AQFuML|13|23-209-805-1449|5261.94|above the blithely regular platelets. express foxes sleep blithely about the doggedly pendin +4029|Supplier#000004029|dSCJo3zeBWOvlROaMg729Vq415d|8|18-351-791-5430|6518.82|sias after the unusual, unusual forges haggle throughout the furiously regula +4030|Supplier#000004030|mwVlSms1SrhbUreGSW3,iWovysS|14|24-407-262-1986|-200.86| alongside of the pending packages. blithely express dolphins nag slyly according to the requests. +4031|Supplier#000004031|69Zkj1G4bDrvUSOAiFqszEOntjvyXm3v55PoPr|23|33-965-284-4886|-970.73|. quickly pending sentiments are bold, ironic packages. pinto beans are furiously. pinto beans +4032|Supplier#000004032|qX8BzAlJRPpdy2|5|15-371-244-4107|6907.08|the final, ironic accounts. blit +4033|Supplier#000004033|ncsAhv9Je,kFXTNjfb2|3|13-767-263-7621|2249.25|kly bold requests use carefully furiously regular foxes. pending realms according to the special +4034|Supplier#000004034| 2NQRRAXCAOhQot5xjf OGgeKnTq,9e|24|34-258-589-9145|5159.00|boost slyly along the final ideas. deposits haggle furiously s +4035|Supplier#000004035|,8jhWpbqUWx6z9xQ43xiHpAJ RUv5WKAhY4|8|18-979-554-8348|3341.62|latelets sleep alongside of th +4036|Supplier#000004036|X4Fs 3LQIxqypZcSIm mxMz9A2qvhM|7|17-571-841-6977|5641.40|l platelets. final, unusual requests cajole blithely even requests. +4037|Supplier#000004037|V9JQR bGGlQ0Bo0vovBVxBkWhB5dvM5tQAUVZBTZ|11|21-720-641-9335|3951.63|lly ironic packages haggle slyly carefully ironic deposits: pending requests sleep care +4038|Supplier#000004038|DiFpY311MtBPT1OzcWLPi r9srEjBL ewMrk2d6A|14|24-952-753-5674|5963.78|es nag blithely. packages above the furiously ironic asymptotes detect furiously about the +4039|Supplier#000004039|0q,Nz9X3EzkAA9PAbCH1KxfAF|5|15-104-769-4241|-732.55|al ideas haggle alongside of the slyly bold packages. slyly final deposits along the furious +4040|Supplier#000004040|FvCZTd0FQ3cnnE,BfaR7Yg3Xvq1oYi9Ol|5|15-730-739-4440|6009.55|e regular warhorses wake blithely carefully ironic deposits. furious pinto beans along the fin +4041|Supplier#000004041|TB Vfbmpzu6uclxsSKY4Yw5AOpme|5|15-628-284-1230|9544.13| beans. unusual theodolites serve blithely after the carefull +4042|Supplier#000004042|lDDnGbQOJb|18|28-664-662-1771|4296.49|ully ironic courts haggle slyly ironic theodolites. furiously pending deposits across the carefully +4043|Supplier#000004043|0BryrUWkXJLiWt u,QYJ2UGINZwd|13|23-938-285-5439|5030.23|s; carefully express accounts sleep slyly toward the requests. unu +4044|Supplier#000004044|0uWmeaH1rCG3NoFT x1i|10|20-257-361-7961|6021.63|ymptotes. even excuses sleep carefully against the dugouts. ironic, bold instructions +4045|Supplier#000004045|kmduAMCt3w|4|14-155-920-1150|3475.80|are blithely. blithely even packages nag furiously after the slyl +4046|Supplier#000004046|mGdjv1oJHJ8b 6zjzy5h2KvP5SZQUsxld|18|28-216-432-2720|5806.64|ic requests sleep furiously against the slyly regular requests. blithely special dependencie +4047|Supplier#000004047|YH7KD9kGfJ4zQSm4r9fxlTwOg,MB,7c|0|10-469-620-5086|9173.80|s! carefully regular foxes sleep along the carefully regular foxes. furiously regu +4048|Supplier#000004048|uM4A1WUm8K3f2SW2hWGMS6Qo6lvqyy34mqWrB0|9|19-579-815-4694|6112.60|cing accounts about the sl +4049|Supplier#000004049|4bGbv74ARNjKp,qFqi00|11|21-266-753-7608|3448.99|wake against the final, special attainments. furiously ironic t +4050|Supplier#000004050|MKXNIx7zJuSB98i7kTiqCEA2k|1|11-991-167-1187|6157.88|counts cajole carefully. packages sleep among the pending foxes. furiously special pinto be +4051|Supplier#000004051|6pfbBoZ3ShZMK|8|18-894-840-1480|5471.53|ully slyly pending requests. furiousl +4052|Supplier#000004052|VKdqaqVtgFXhivaNQyUtxnoHNxX4O 03InxbRI|14|24-287-141-7265|9064.24|regular foxes wake quickly regular theodolites. fluffily even frets cajo +4053|Supplier#000004053|2JMJv1dJ2Q9GrOt2,qDIfYr6i,fHnoNk|1|11-430-753-8401|1759.73|according to the carefully final ideas maintain carefully slyly express dolphins. daringly pending +4054|Supplier#000004054|ezSvqjxZPBEdneyB9XMEAHVFU|7|17-745-455-9997|5758.55|th the furiously bold accounts. quickl +4055|Supplier#000004055|pRj8MK0LkYSIeckmEEHnrV H|12|22-266-274-3808|3976.23|nic courts x-ray fluffily final requests. final accounts wake. ideas nag upon the express, +4056|Supplier#000004056|cHP8axhX4jEhoLR2j|8|18-954-925-1625|1486.03|atelets boost silent, stealthy packages; blithely final excuses cajole blithely across the regularl +4057|Supplier#000004057|c8VlmzJTjjFd6mbhxDUn6AR,v096mUn|17|27-629-618-9634|3693.38|thely. regular requests haggle-- even requests around the quickly final ideas wak +4058|Supplier#000004058|v5sj1ZbwPn3M|12|22-338-390-6980|-312.57|ctions hang blithely special asymptotes. carefully regular theodolites wake slyly fluffily spec +4059|Supplier#000004059|umEYZSq9RJ2WEzdsv9meU8rmqwzVLRgiZwC|3|13-563-895-2947|2579.78|es. blithely even packages sleep slyly. carefully final foxes serve blithely after +4060|Supplier#000004060|YrkOHv5RWoL,oTblG|4|14-154-817-3962|5392.46|its haggle carefully ironic accounts. final ac +4061|Supplier#000004061|B7ephfEDaRZd |19|29-550-254-2191|68.29|low instructions! special accounts solve carefully ironic ideas. enticing foxe +4062|Supplier#000004062|BR,3UXBpQ85yXn,Mdpp|7|17-829-494-2944|1666.70|ccounts along the furiously pending ideas x-ray blithely slyly ruthless +4063|Supplier#000004063|FN6Cy0PtDb 01IKxcVzmO|17|27-145-373-3507|4106.48|cajole furiously about the regular accounts. blithely express platelets according to the s +4064|Supplier#000004064|W7lP8LAhnBnDdcF|7|17-661-859-5777|1608.06|manently silent instructions wake quic +4065|Supplier#000004065|7nABVd ntFsyxn6yImDb,cb iTjWUpZ8v|4|14-391-849-5143|8581.88|ages are blithely after the ideas. accounts detect carefully. un +4066|Supplier#000004066|TNBnJFDScUmsjBy6pSWTS sfMg9jpfKx|21|31-304-122-5188|1789.03| are along the pending excuses. carefully even dolphins above the blithe +4067|Supplier#000004067|ip9T3DacGd9CT79BTFzQKIOiF7AJaM|0|10-448-907-7028|3231.93|ng, express asymptotes. special foxes wake fluffily. even instructions are alongsi +4068|Supplier#000004068|w2MniB04fvEPPAHTfEi1bS3e7eQboYSB2bq|12|22-290-743-7808|-963.24| deposits. unusual foxes integrate slyly; blithely regular reques +4069|Supplier#000004069|b4Al0OuBTy3 u1Zu|4|14-301-473-6179|734.94| ideas cajole blithely blithely final pinto beans. carefully ironic request +4070|Supplier#000004070|fTacTQnOX3OwNmkzn1GHoYbPl5j|19|29-634-147-4221|8041.25|es. foxes sleep furiously. slyly fin +4071|Supplier#000004071|6nU9RahtRwcp30YbobWoftQDsB5fJCnc5XuU ISO|17|27-438-434-1563|1300.59| slyly even requests use quickly along the instructions. unusual packages sle +4072|Supplier#000004072|lAYDI98l4wGJ98|6|16-170-193-6812|-19.78|gainst the ironic theodolites-- slyly ironic accounts use furiousl +4073|Supplier#000004073|c2G,jK,7ZQMS|11|21-402-657-3824|6410.58|ically silent packages sleep across the fluffily express packages. fluffily +4074|Supplier#000004074| jYAhKOq67V6|5|15-801-984-9558|6628.91|ly express packages boost. requests among the packages boost quickly +4075|Supplier#000004075|hK6J7VA GsAmrmRLV5EWka7h jh |22|32-699-618-1932|2205.58|y final deposits above the sometimes ironic courts impress carefully thi +4076|Supplier#000004076|U9Ud8,xgsTUncU4v0kw7Hl7BRe,dhjM|10|20-989-161-6648|8159.18|ages are across the carefully +4077|Supplier#000004077|q15WSZZMzWG2|18|28-174-389-5502|3347.86|le. ironic tithes play blithely above the slyly even instructions. fluffily unusual dugouts +4078|Supplier#000004078|eem4mxDJCf be7a5EVsJ3XTGGae|0|10-473-484-5398|541.97|ctions. blithely pending asymptotes wake across the platelets: unusual, bold ideas sleep carefull +4079|Supplier#000004079|liy8fz UzaIQFzpgFU6BEq5gutXjZhY|16|26-459-882-1161|9801.17|ickly ironic pinto beans. ruthlessly final pinto beans might haggle across the +4080|Supplier#000004080|qt0UwJjCCqm8F3pxUO|1|11-889-871-5275|1276.51|equests. carefully express pinto beans wake blithely carefully ironic requests. +4081|Supplier#000004081|6lvRJvSrtCgT41z,NcDKPswTAPyG|12|22-603-507-5487|2220.38| furiously dogged ideas integrate carefully carefully regular deposits. ideas boost b +4082|Supplier#000004082|guJSg4r8dqVEUuJ7nERP6lMCVDfqbydkXoyTxq7|10|20-154-685-5666|3955.33| ironic foxes haggle carefully ironic instructions. blithely special instructi +4083|Supplier#000004083|ag x rRbSF61|21|31-230-841-3385|2180.98|arefully quick realms are among the asymptotes. blithely unusual requests impress slyly even excu +4084|Supplier#000004084|OY0,DSHzmX3op|4|14-256-608-2643|1774.94|y regular packages. furiously ruthless deposits +4085|Supplier#000004085|0yclYyDPd9l8ZTc|4|14-901-186-4338|8327.74| even dependencies. special deposits cajole carefu +4086|Supplier#000004086|KTYGIMlW8QAwgnW3brLo1IJKkT8Jh|8|18-307-627-1756|9452.73|ions snooze quickly quickly regular theodolites. fluffily pending +4087|Supplier#000004087|gT8xS3tEyvgX9P3ZeEuQ|12|22-944-125-9013|8143.30|es wake slyly between the special instructions? carefully regular waters believe accord +4088|Supplier#000004088|5ynJA9pQAO2MsDkgGeqaP8Z3JBCUNkeg9U2bf0IH|5|15-945-528-8622|465.23|ly ironic excuses. quickly final accounts haggle above the slyly re +4089|Supplier#000004089|Lsa4Kdr1kJEee2co6A|8|18-180-548-6033|7089.65|fluffily. final ideas sleep +4090|Supplier#000004090|vRKDWYYcJ9xGtf4xHcWTjXW22|6|16-558-247-2116|1847.98|d about the carefully final braids. regular deposits among the regular foxes sleep slyly +4091|Supplier#000004091|lCHbIdlbMqfUEmDExPjl7WF68jQ4Nf|10|20-654-710-4514|3971.86|he carefully pending dependencies boost slyly +4092|Supplier#000004092|WPUH06LR5goleIM0McfrAGmxP0FdbzTFfKl8Z0X|22|32-799-299-9099|5007.04|are ironically regular, special ideas. ironic packages are always after the slyly express t +4093|Supplier#000004093|NqmTeIrh7Lyy,N5kn3cid|9|19-195-597-5704|1359.50|ular, regular asymptotes. pending, regular dolphins haggle fina +4094|Supplier#000004094|sQasHEwkxpKs4hY3QlCdaXplmDg|17|27-668-190-5473|68.44| instructions sleep quickly about the furiously final decoys. ideas accordi +4095|Supplier#000004095|tqzTeCdPrAZ2vHGQV1dKz|11|21-926-762-4190|4229.25|he fluffily regular foxes. ironic accoun +4096|Supplier#000004096|T9OqQ8TzKgDo RUuwXcEePKA6tfRKyn9hd3jVJhh|16|26-638-189-2205|7998.30|. regular requests alongside o +4097|Supplier#000004097|959 JrnyclmoMkfgiHOn0iEN7p7NR4BUJyeI4la|8|18-644-641-3609|5436.28|express deposits are fluffily final +4098|Supplier#000004098|qsYuM4iJpUVa99ihCd7vUt18vjwfUOs1DmYEq|15|25-888-375-3220|-440.01|s are unusual instructions. quickly close theodolites must unwind +4099|Supplier#000004099|i8bJfWbFHB3,vVpIu AhVoSexW FO6Dc4NEwG|21|31-359-746-4332|5634.51|ar packages. carefully final theodolites detect never bold pinto beans-- ironic courts affix +4100|Supplier#000004100|Es kTX,qKxc 9rHSvWHd2V|20|30-786-857-4470|9162.47|s quickly about the final requests. final accounts cajole furiously amon +4101|Supplier#000004101|61PPU02b8s71S8BVs1Dw,2dlJR68l2pgW3VL5bZs|7|17-849-850-9483|2667.26|osits use carefully according to the requests! dependencies across the bold instructions haggle a +4102|Supplier#000004102|61JlI5AuCyu4ofIwTVUwqo|2|12-833-954-8094|1693.72|rash slyly. quickly bold accounts are final dependencies. quickly +4103|Supplier#000004103|EYdd,noRdDff|24|34-391-171-7480|7346.23|lithely among the blithely bold gifts. furiously unusual ideas w +4104|Supplier#000004104|GeXZl1DKV33wMLSDvknEXiX6GIvg4|21|31-582-440-5170|2056.74|carefully final deposits wake slyly. silently express requests use unusual, un +4105|Supplier#000004105|FVJAa1ZG3WvJFf53jhtlzxvEUjxyl,e3msMGOBy|13|23-467-138-6608|3531.44|ly silent accounts wake idly regu +4106|Supplier#000004106|EJUpZra574iVKsvcaMER92FH|12|22-995-670-6182|5835.62|inal requests. quickly regular deposits sleep carefully +4107|Supplier#000004107|a5szQnHbzIbBIjNk|18|28-449-573-5038|1288.34|ccounts. furiously pending requests wake quickly across the sauternes. furiously enticing +4108|Supplier#000004108|sCvgIk0Gu6cetKwUIQyeZFy8vnX09|11|21-635-738-9467|3013.44|y final instructions. quickly pending reque +4109|Supplier#000004109|C JLgSsemHV3VH0|20|30-243-710-8524|888.50|inments above the ironic theodolites use daringly final notor +4110|Supplier#000004110|MdRKQyBm6DpRtpsUiOIzJX Ug7L1m0oTXAoviQm|9|19-100-479-8965|3963.37| nag about the quickly blithe instructions. express deposits are fu +4111|Supplier#000004111|fEPasEoS6uaJ VsPTh1qNht0|3|13-304-658-1811|5300.19|quickly silent requests haggle across the special instructions. ruthless the +4112|Supplier#000004112|fqKCcsHejokoPSxpJ|20|30-292-857-7457|387.49|d frets ought to haggle blithely. slyly unusual instructi +4113|Supplier#000004113|35Re27oAr8h sP3V|19|29-821-992-6070|9529.36|express theodolites are b +4114|Supplier#000004114|eOk2ClJl jmJQ4HfCqTFxANk5K4oLm2|8|18-649-284-8146|9106.84|equests use. foxes under the furiously unu +4115|Supplier#000004115|QSr7SNEvD1e6hgM,|6|16-858-156-8401|2718.00|ckages wake quickly ironic instructions. ironic, special platele +4116|Supplier#000004116|dljIeOzzKqEpczZ1rxdcnUW3|17|27-582-223-3834|7423.10|e the express instructions. platelets sleep fur +4117|Supplier#000004117|7,iPNWDTA32y0Yhtp|9|19-869-181-5251|7041.20|ometimes dependencies. final sentiments nag fluffi +4118|Supplier#000004118|gb,ZOOVw0yzEK4QWf7YLOHzV0n05Qh0|23|33-572-732-8810|1427.47|ven deposits integrate fur +4119|Supplier#000004119|ivpk4fNxM8zcbv6228Q v|16|26-580-885-7679|9493.02|r pinto beans alongside of the blithely silent packages maintain furiou +4120|Supplier#000004120|N42zLM6QQ,zI3dh1vF9XJSROd4Fwe|0|10-145-515-7985|3325.23|olphins. slyly regular requ +4121|Supplier#000004121|RyVJNHSjgZf05iILX,vBEECitv3AF4S4ND9,no|24|34-434-254-1999|5161.55|equests sleep along the slyly ironic instructions. regular platelets wake +4122|Supplier#000004122|p9nQVmRwhjmH|6|16-420-401-4362|2159.81|ests sleep furiously. ironic, ironic instructions cajole slyly against th +4123|Supplier#000004123|cj mnWwEmCpO3LHL2|12|22-394-647-5982|8853.39|ual pinto beans alongside of the quickly pending packages detect alongside of +4124|Supplier#000004124|V7eiEsuWVK5FQos,YWNZW|12|22-903-472-5647|3.44| final platelets; quickly ironic excuses haggle carefully. pending foxes +4125|Supplier#000004125|qa YL0jDzSogjG|13|23-983-768-5797|715.29|furiously unusual accounts alongside of the furiously regular instructions haggle furiously final +4126|Supplier#000004126|3s9EL2QxI5lAEeSPr9aDv0 O0X7SP PA4TQWAAYn|19|29-497-666-4765|7814.84| fluffy packages. furiously ironic r +4127|Supplier#000004127|EduKm3NcCc75Cd|21|31-266-858-4899|2002.34|ely alongside of the never pending pinto beans. blithely regular pains doubt furiously agains +4128|Supplier#000004128|BnmCG2fbzFeFnrWSQdygRpSw9Q0wQlw|2|12-551-740-8419|2345.12|ep carefully: blithely ironic requests snooze into the accounts. slow accounts snooze +4129|Supplier#000004129|gdAUjZ6SpR7L6n1tV,Iqc9eGAkTdfU1WX5b5nH18|9|19-920-465-4477|-654.55|r requests above the furiously pending deposits w +4130|Supplier#000004130|2Qd5FYKuN,sT|14|24-766-166-3005|7856.00|finally fluffily final deposits. unusual ideas sublate slyly furiously regular accounts: slyly ex +4131|Supplier#000004131|hWsnfB9eInc4Ru SN48|11|21-606-922-9922|875.72| after the excuses cajole furiously against the furiously ironic requests. final, regular packag +4132|Supplier#000004132|93NSzNRlTdFtzO6Y7YAcJVMm3k6Kejmi|18|28-258-314-7612|9094.28|s haggle above the carefully unusual accounts. furiously fi +4133|Supplier#000004133|ojv6Hwnm6sSaPr7bVbxptcmZpG5|8|18-616-299-8361|569.47|ke special attainments. quickly re +4134|Supplier#000004134|mtcNe9kFAvH1cbrhZd3nnepaeug6BbNadQ1Dd6l|24|34-211-814-6217|-953.13|ly. unusual requests around th +4135|Supplier#000004135|dbHS8xeJMoEG yEVI,eRk6LZrj 89XoJ|7|17-244-262-1877|5647.88|lithely regular accounts against the ironically un +4136|Supplier#000004136|NhKJIHTz1df2V|22|32-753-406-1180|2856.24|es. furiously express foxes haggle slyly slyly unusual foxes. slyl +4137|Supplier#000004137|uqz8Ch9WbSiYYosWTBRwLaGu Oajq|15|25-568-662-5131|9861.11|ly carefully permanent accounts. carefully even foxes use fluffily regular packages +4138|Supplier#000004138|iKg4xdtI7BQbfNNc|17|27-522-884-2385|7819.06|ly ironic deposits. regularly final foxes use fluffily along +4139|Supplier#000004139|RPGuceC07eTzSMinFHQA35A|14|24-842-966-9888|6956.48|ly bold orbits. permanently +4140|Supplier#000004140|0hL7DJyYjcHL|3|13-865-411-3277|7536.24|y ironic frays haggle fluffily bold accounts. +4141|Supplier#000004141|aTWXZY7ogPN8|9|19-982-421-3473|5642.65|epitaphs affix fluffily even requests. ironically even excuses detect ruthles +4142|Supplier#000004142|AyAtTNfpIxllZ0oMG|10|20-534-464-1846|3004.78|n above the slyly ironic packages; carefully regul +4143|Supplier#000004143|0wxhz5ApPeRrkEyH5rQg35MsYCh|19|29-195-404-4570|9296.94|ymptotes. even, express hockey players +4144|Supplier#000004144|nNcfeDcUBUZbwRkYFz|12|22-365-581-3053|7741.10|eodolites. bravely special packages cajole be +4145|Supplier#000004145|BIKx3OmFTieb5WtD34c6N6iY9j2K|11|21-111-429-3324|5706.63|ctions. fluffily express foxes maintain dolphins. slyly ironic requests affix carefully regular req +4146|Supplier#000004146|BGWK5FgOxJtuosJnFyvv2Tu7kUS8YT,7bA,fI|15|25-979-553-1844|330.26|ing pinto beans about the bl +4147|Supplier#000004147|,D fUyF6vDvmaf05 kCdHQc9h|2|12-811-183-7674|4638.57|ymptotes boost blithely regular ideas. pending, even dependencies cajole furiously! regular pin +4148|Supplier#000004148|ycg9af69X3Wh2UBtEK5cTgGzT fqRwKOEALJoe|17|27-397-456-2436|1365.10|s. carefully regular packages are blithely express dolphins. furiously even +4149|Supplier#000004149|yMUXO0,YCZcmG|0|10-431-966-2386|6118.21|as. final requests detect among the fluffily final requests. quickly final excuses are quickly. sile +4150|Supplier#000004150|0jGjCmN9fwgJ2jG,|8|18-994-717-4433|9827.80|ons cajole slyly according to the special +4151|Supplier#000004151|ZnDgZenJq19th5hf|14|24-355-135-8448|8868.12|cial, final accounts mold slyly. final packages slee +4152|Supplier#000004152|CsxW4AcucGTtTdod8FCAPn|6|16-530-394-9077|5500.31|ily ironic excuses: slyly even requests haggle fluffily about th +4153|Supplier#000004153|PnftHjIOKN6VUOD|15|25-205-843-3592|8697.12|ckly idle requests cajole blithely. sl +4154|Supplier#000004154|et0DhpDSfxa|18|28-337-814-5666|2497.06|uctions cajole quickly blithely qui +4155|Supplier#000004155|1kFL1WiHShkArhTmnzyE2x4QLO|18|28-897-120-9617|1103.22|posits sleep quickly at the instructions. carefully final excuses sleep. bold, special pack +4156|Supplier#000004156|FmZUY0DIA34sxL Bf|21|31-530-306-1884|4766.73|ld excuses wake slyly slyly ironic ideas. quickly unus +4157|Supplier#000004157|zRTkRcaVtCHv6|17|27-846-933-9715|-584.07|es boost quickly across the carefully sile +4158|Supplier#000004158|Z 2zg8MRUx9JttwvUXCpSEoEa4Ym9V8mWF|19|29-465-570-9335|4395.08|ironic foxes use carefully. unusual theodolit +4159|Supplier#000004159|vCppZkSdm0jvPiaq85mk,6vt|24|34-109-267-3342|1095.01|nic pinto beans. slyly ironic grouches sleep fluffily about the quickly silent req +4160|Supplier#000004160|DpYX7p iwphihHJz7h|13|23-751-712-1339|9812.39|he ironic escapades. blithely ironic deposits are fluffily. even theodolite +4161|Supplier#000004161|YQTPCAPvWMH3Dx,zjuunUQrD3UfzCKQsvpW l9|10|20-887-739-1420|2976.54|efully blithely unusual packages. sometimes ironic deposits among the +4162|Supplier#000004162|U1O ZxpOkMI7K|14|24-652-643-6695|7654.28|he fluffily even theodolites use blithely fur +4163|Supplier#000004163|2VnQypC7pJPJRu6HzYRg7nAvhzckcYAQFbI|20|30-544-852-3772|9513.31|he regular requests. blithely final +4164|Supplier#000004164|f60HY65zdJb6eSCUYOmm|6|16-161-750-6814|3240.41|y pending requests. furiously fin +4165|Supplier#000004165|wTJ2dZNQA8P2oi99N6DT47ndHy,XKD2|3|13-845-894-5027|9615.49| ironic deposits. courts wake fluffily across the ironic, special excuses. blithely express accoun +4166|Supplier#000004166|dEGlhdQfTkadp|22|32-753-533-6531|4696.43|. carefully special accounts sleep slyly special, final pinto beans. quickly regular excuses +4167|Supplier#000004167|ZeTibFfxIahMkQSCmlxk 2zJYiSQOaR|19|29-842-894-8667|5832.40|even instructions above the braids doubt carefully across the blithe pinto beans. +4168|Supplier#000004168| s,vfse eqAdabCk|16|26-443-304-5059|2237.64| blithely ironic foxes-- ironic, even depos +4169|Supplier#000004169|,YM5rGvb5bEDDdY|20|30-542-513-7503|7926.37|ronic theodolites cajole slyly special excuses. ruthless, unu +4170|Supplier#000004170|Hy,jcuzFjmL57c|18|28-480-378-5074|6499.80|eep quickly blithely pending requests. platelets use bravely. silent ide +4171|Supplier#000004171|N UgH,tE,9Bs,|22|32-137-311-1336|-878.57| alongside of the carefully even platelets solve slyly ironic packages. even, spec +4172|Supplier#000004172|f8OSCEERjJIELPv7zkkoSMew0gw59gACm42c|13|23-952-476-7364|4754.93|nal instructions affix furiously furiously regular requests. carefully i +4173|Supplier#000004173|wA,sCR0eZJGZXhl RH tkxGZgJFSmX |18|28-879-181-5180|-894.94|ckages. unusual instructions sleep stealthily silent instructions. special inst +4174|Supplier#000004174|Bk97olQYwXmjYdQjwyt N|21|31-178-422-9841|4636.57|lly regular theodolites around the blithely pending foxes are according to the carefully furio +4175|Supplier#000004175|MVBQawEe,35kLvhEdC8F8tJMWenyu,apYhHEDohw|0|10-920-963-6166|912.13|ronic packages. carefully even packages haggle-- quickly bold requests b +4176|Supplier#000004176|56vSHOzP6vBy|17|27-118-809-1468|5180.41| cajole carefully around the final, ironic packages; fluffily final packages wake furi +4177|Supplier#000004177|tlmGv4IoRL|12|22-661-621-8522|239.06|tegrate alongside of the carefully regular theodolites. carefully regular foxes +4178|Supplier#000004178|fDxQjzwbid,rccxUV5m9krDH8sHoH|22|32-538-388-4683|872.08|ccounts haggle against the even deposi +4179|Supplier#000004179|miD3DjuD8h8HD|9|19-101-968-7094|3015.62|egularly. platelets cajole furiously special ideas. fluffily s +4180|Supplier#000004180|g4DgkJDL9q5Sh3UOOm7cEGVRvLmnH|14|24-834-640-7047|4521.71| carefully final requests. even, even deposits haggle carefully blithely b +4181|Supplier#000004181|whr6sztd rB|19|29-130-131-7296|6412.91|g furiously regular deposits-- even du +4182|Supplier#000004182|VTP66AgaGFwQ4jHNVVq5jDwBwShO2x13m|18|28-734-336-5623|1867.22|g dependencies integrate quickly quickly thin accounts. pinto +4183|Supplier#000004183|OD7fNRcFuwgThsj2j7rC|10|20-206-616-7911|8323.29| haggle after the slyly final asymptotes. blithely silent deposits sleep quickly +4184|Supplier#000004184|KgIknUf2VZcXo VbD|8|18-442-744-5424|513.75|ounts. furiously pending accounts run furiously. waters ha +4185|Supplier#000004185|Q1,8W2TRvPlVM3qFf9|7|17-429-604-6315|8566.57|quickly regular foxes cajole slyly unusual packages. final ideas are quickly. quickly +4186|Supplier#000004186|WKrQMVh5a7rutLDNkvce,H6RMnSeqS6|17|27-979-270-8037|8428.99|re slyly about the blithely final packages. fluffily bold foxes was +4187|Supplier#000004187|6yf50yi0Nwor,EGB|0|10-585-511-4887|6015.66|ependencies shall cajole beyond the regular, regular requests. carefully b +4188|Supplier#000004188|2M4Uuj0Ciur|24|34-978-302-1714|2258.45|usly according to the slyly pending +4189|Supplier#000004189|NZ8VSADsc5C|1|11-516-663-3922|5722.60|ake furiously idly final pinto +4190|Supplier#000004190|4c9flImYQWVVk3n2ZQFFuj|12|22-159-447-1770|4774.22|ggedly pending pinto beans are carefully after the bold pinto beans. express, unusual +4191|Supplier#000004191|8DVAQePX6TmTP|21|31-750-984-5481|4353.46|the even instructions. furiously regular accounts would haggle abou +4192|Supplier#000004192|93oY1dMuRsP4aaCK0QrsoFgZf6yB9CHy2Ba|7|17-858-849-2452|5557.64|s. blithely ironic excuses nag fluffily special pinto beans. c +4193|Supplier#000004193|cYVyLCVpDyxrIIwXRjQ3Q7XM aPDzRrn|18|28-181-213-6182|4289.58|uriously among the express ideas. carefully unu +4194|Supplier#000004194|A8AoX9AK,qhf,CpEF|13|23-944-413-2681|9973.93|t fluffily. regular requests about the regular, unusual somas play of the furiously busy +4195|Supplier#000004195|NHCu,qYwb21TrXtL8iXEI4QZ|11|21-437-493-6911|945.79|s. furiously special requests are. ironically regular packages doubt alongside o +4196|Supplier#000004196|M WMlXALjG85VYh9mkvo81,|13|23-679-372-7879|9825.61|to the blithely even theodolites-- blithely bold accounts are about the furiously ironic package +4197|Supplier#000004197|M dDlXDL4OL|6|16-556-895-4554|5038.36|ss deposits. slowly even accounts use slyly. slyly pending ideas haggle. unusual +4198|Supplier#000004198|SxRHOjpQz7vl2rjs m2xzQUPL|1|11-246-592-1391|-257.71|usly blithely final deposits? fluffily regula +4199|Supplier#000004199|zM5Kiu1llc|20|30-674-871-2102|613.65|l packages cajole carefully silent requests. pending +4200|Supplier#000004200|tCZuhXdCdu|7|17-277-324-6951|4480.50|ss the furiously silent courts. quickly even foxes nod pe +4201|Supplier#000004201|C5x43M76dnb1w2LFbhB|3|13-431-798-5549|6735.20|ies use furiously. carefully unusual theodolites boost slyly +4202|Supplier#000004202|Pxtjk35pK2apVyGQzxktaH6UTe Ob|6|16-920-942-6992|6584.95| the carefully special theod +4203|Supplier#000004203|MIzyGBM4h07HSX3Mzs|4|14-994-888-2741|273.49|ns are among the carefully regular requests. quickly bold theodolites against the carefully fi +4204|Supplier#000004204|0rGZJ6VZXdH|4|14-520-667-4690|7472.88|y pending pinto beans. even, final requests sleep care +4205|Supplier#000004205|S2OR,fUOmf3GSlKnW6|11|21-538-923-6240|7707.12| ironic instructions sleep blithely around the reg +4206|Supplier#000004206|MuQVGGHxNCfrnK5D8KLpDJZEd6z4obIgq|13|23-215-218-6180|6574.57|use express accounts. pend +4207|Supplier#000004207|tF64pwiOM4IkWjN3mS,e06WuAjLx|3|13-514-410-8413|1807.67|rts? slyly regular pinto beans nag. slyly even foxes wake fluffily against th +4208|Supplier#000004208|x3jZYF7ZYN 8a4LY1c1kEsh|11|21-468-998-1571|8522.70|furiously regular accounts! +4209|Supplier#000004209|HlxV6Iw49NHEA,GNT|8|18-848-514-9363|8084.48|. unusual requests sleep inside the daring patterns. final, permanent notornis poach fluf +4210|Supplier#000004210|U3PW,CIE P226655v|1|11-364-327-7484|1901.23|final, even deposits was furiously furiously even accounts? final dugouts wak +4211|Supplier#000004211|2wQRVovHrm3,v03IKzfTd,1PYsFXQFFOG|7|17-266-947-7315|7912.91|ay furiously regular platelets. cou +4212|Supplier#000004212|9yBvOycd0LCgj96MSFwPj|17|27-121-443-6101|393.12|ost regularly bold theodolites. carefully special d +4213|Supplier#000004213|irhmW87hVlyK zTaSW|1|11-113-216-9762|5147.60|nts. final, ruthless instructions amon +4214|Supplier#000004214|jgfdFT00xVAzzaLMAh,z6F|1|11-934-860-6515|9689.56|ily in place of the blithely blithe requests. slyly final requests sleep slyly! +4215|Supplier#000004215|NEDU5ZeLyyn,EDkL|0|10-365-101-5443|6469.18|nic packages sleep quickly outside the qu +4216|Supplier#000004216|Llx127Mkqt0rE3H6PLNMeOY|22|32-990-787-5400|1622.16|ffily furious deposits cajole car +4217|Supplier#000004217|ydoKPl17,5vSEc0HCA|16|26-866-290-6557|3661.09|onic dependencies. regularly pending deposits along the blithely ironic +4218|Supplier#000004218|FzGC7DOsv5mlS56Cd46JKkfXVZ|24|34-485-986-7558|6952.60|nstructions haggle slyly quickly +4219|Supplier#000004219|rEkQNtm91h,QzZ3gTOR|19|29-828-181-6222|8425.38|ng, final deposits hang slyly according to the carefully busy requests. foxes +4220|Supplier#000004220|lj7Vjmlzj2R|21|31-201-829-7881|405.68| regular requests. unusua +4221|Supplier#000004221|4xhxUak1,gmFgNlwt|7|17-682-721-5080|7219.59|nstructions are unusual accounts. dep +4222|Supplier#000004222|y9KrpjIHV2hrEN4UjQsgcXT|9|19-316-548-3899|7757.76|fluffily express asymptotes nag enticingly fluffily regular pinto beans. carefully +4223|Supplier#000004223|1IMT,5uy4n5v7TDCFMQDhTG523R06zffJ7|19|29-930-356-9108|-201.93|telets boost blithely. fluffil +4224|Supplier#000004224|czIAOSaOsPxmB1S|11|21-610-199-4406|3053.75|g, ironic platelets. requests will have to d +4225|Supplier#000004225|orvvW5KbRa0tR3r|18|28-792-509-2921|7362.18|uriously ironic requests nag slyly alongside of the fluffily final requests. deposits print along th +4226|Supplier#000004226|TLWMyc6v5CV2fEhXF0JzzTaN70X588o|12|22-534-251-8116|6256.89| to the ironic deposits. carefully express p +4227|Supplier#000004227|3BO8G2ObAldW4YnuYjoDytLx8CmnYoGb|1|11-223-593-6259|8581.15|pinto beans. special, final accounts snooze. slyly silent dependencies bo +4228|Supplier#000004228|p82YcdMAVKXGr|8|18-101-805-3371|411.18|ully waters. carefully ironic dugouts behind the ironic instructions haggle slyly ironic fo +4229|Supplier#000004229|Nt,JGB8zs1QNzeLEoRp,uNZot,pPgbU,fNIoHi7|4|14-171-536-4910|590.79|, ironic orbits cajole slyly r +4230|Supplier#000004230|hbZADmI,Fb41r0|8|18-955-229-7900|5394.59| bold epitaphs lose. boldly regular dependencies cajole fluffily ideas. request +4231|Supplier#000004231|c,kTPderfPwsHQZQhJF6oX7vYbVYuT,|13|23-721-789-1977|2949.59|e of the even, furious requests. final, pending +4232|Supplier#000004232|20Le8PqmvpoWnTSZaEjQR9,wUCRMD524VW|2|12-827-734-8007|5754.78|efully ironic theodolites n +4233|Supplier#000004233|NSgt4kMBUwpWuPSaeXASzfCE|24|34-928-485-9615|6647.78|longside of the furiously even theodolites cajole furiously about the fluffily pending accounts. i +4234|Supplier#000004234|MCpt8W2oOwFg KjwCerLgx rZfFljVa|6|16-299-915-3794|191.08|nstructions believe slyly-- slyly express deposits haggle fluffily bold foxes. blithely special id +4235|Supplier#000004235|1tL4ZGIjgj7oWDvAgQ0QIaiyIz9Tg6Tfh9V8Kv|14|24-372-176-8066|8486.70| sheaves sleep blithely busily express theodolites. +4236|Supplier#000004236|dl,HPtJmGipxYsSqn9wmqkuWjst,mCeJ8O6T|3|13-838-515-8794|7963.89|g blithely final, special theodolites. blithely final pinto be +4237|Supplier#000004237|d3O4o1UcVjarffpAfzrMnsd4Vzc|16|26-956-589-3242|-821.53|xpress epitaphs cajole. furiously unusual theodolites mold slyly. blithely +4238|Supplier#000004238|ycSsD71UyOr|0|10-689-876-6705|6380.51|ously. dolphins haggle before the quickly ironic dependencies. furiously ironic ins +4239|Supplier#000004239|qLU92ZXkuRXBkgmW0|2|12-468-683-6101|7229.29|s breach. quickly dogged packages haggle along the blithely special d +4240|Supplier#000004240|cTrHkuZZcx7U5Joo9ldLftgHX4|15|25-127-452-3901|5651.10| dependencies cajole final excuses +4241|Supplier#000004241|6pxosiF1EGdO5Oy1v8TmgG7XJtgH|10|20-106-726-6490|2003.19|y careful accounts. carefully bo +4242|Supplier#000004242|SalyknolA3M0Hh1c3WXbfmZ|18|28-928-132-6073|5599.91|ngside of the slyly blithe packages. slyly unusual ideas haggle evenly. express, ironic packages u +4243|Supplier#000004243|vXJjRIpbdTsDaIcQC,2EewOhhopeHq6xDTjRg|15|25-821-326-3980|-357.98| fluffily silent foxes nag after the requests. regular +4244|Supplier#000004244|FQkWBaQltAEuUfXjpt,NBJr|19|29-378-859-2844|9277.74|nic requests cajole furiously. permanent depo +4245|Supplier#000004245|1oTU7eTB3MT5QmFK8pghsb1UC|0|10-245-577-9925|4053.00|the slyly regular packages sleep fluffily regular accounts. furiously express deposits cajole +4246|Supplier#000004246|Xha aXQF7u4qU3LsHD|3|13-169-592-4124|4685.16|unts wake slyly. quickly sly accounts affix +4247|Supplier#000004247|lCntS,Bm 10CjIMHjBTLwAUjvsO|0|10-680-931-1664|3452.26|kly slow instructions. sometimes exp +4248|Supplier#000004248|nCh1nEMOTfizK fDDo3VTgiVerYbiHIhH2sFRrU6|1|11-406-712-9604|6169.94|ding deposits are quickly bravely special package +4249|Supplier#000004249|dqjRKOF5zt6IBAEHkbniUdyu4kREGJGqQV9|5|15-272-365-4305|-734.11|e quickly regular instructions. carefully ruthless theodolites +4250|Supplier#000004250|QGRJ1zGIqSKy6i82K7P947ermR|6|16-206-461-6505|1743.91|ng platelets. packages among the packages integrate according to the pinto beans. furiously unusual +4251|Supplier#000004251|bvEX6FcSNwAErYcRa2DoQmFhUdCeadDun|16|26-728-297-8399|3807.93|nusual, idle ideas above the blithely ironic forges nod quickly fluffi +4252|Supplier#000004252|pyUiXm8P37ihSJVb3rPDRarGu|1|11-477-385-8392|-167.04|requests boost slyly along the pen +4253|Supplier#000004253|m0z2VT1IRNaZ6GWeGHOQDk9pri96|22|32-341-145-7575|6548.23|ously even instructions against the ironic asymptotes mold excuses. slyly fin +4254|Supplier#000004254|jRDRD68ax iCiF,Zz7ZVu ROeotx,i|5|15-163-912-9576|-801.65|the pinto beans boost silently about the unusual, even requests. slyly even ideas sleep according t +4255|Supplier#000004255|Qve06kw6vAOIeAavZqWRiHeEJo7eh|7|17-993-215-7051|-406.67|ourts. fluffily special theodolites haggle furiously. fluffily express accounts across the express p +4256|Supplier#000004256|6M1,gdZ1En8QP4oZQEdrkxqCuI6xA0lh|13|23-276-880-3602|4957.61|bove the requests. regular braids sleep. furiously regu +4257|Supplier#000004257|j2AGQ7OWa7HZDt8A|12|22-929-508-1442|6635.08|s cajole quietly slyly regular +4258|Supplier#000004258|oXKtTTKlpcYIbuiMgfnP0sWD2P2Ngas|23|33-173-309-5477|6806.27|. ironic, even requests above the regular, final +4259|Supplier#000004259|GozazL5xiDkcFKmg04MYTxri|0|10-174-516-3980|2029.03|ges nag fluffily. quickly express platelets serve enticingly accordi +4260|Supplier#000004260|2VLKh21kkmqrnkLoArkPnBMnxczfyF,Pux6bjDB|2|12-473-120-3120|-71.77|each according to the slyly silent pinto beans. careful Tiresias haggl +4261|Supplier#000004261|5,TavIRWV DnXL3CsyYN,h7x7 37|2|12-362-986-9587|1753.14|ronic instructions are furiously ironic packages. +4262|Supplier#000004262|W2rvSw2sSsrpj y55TxGLRVdd5fZq2q|16|26-271-701-3912|4969.25|s requests. express excuses sleep slyly. silently bold foxes haggle final requests. blithel +4263|Supplier#000004263|DTd4Cftgq4BgEnbTW0|24|34-837-771-4937|4201.05|elets. slyly unusual pains sublate blithely after the final, pending plate +4264|Supplier#000004264|caSehoQiSlTnW|4|14-599-654-1986|7105.57|uctions-- slyly special accounts sle +4265|Supplier#000004265|MksIfOHfEBMdVppoHM6znnWz8,zKViClfw0dK|10|20-140-789-8069|5290.89| special pinto beans. regular deposits boost blithely pending instructions! furiously sile +4266|Supplier#000004266|gN,uZm1z0Z2MOH171deuGcrXzmOp|10|20-791-144-4523|8951.34|eposits integrate furiously. furiously express excuse +4267|Supplier#000004267|iV50e6z8gV |8|18-175-189-3365|7234.04|uriously furiously final asymptotes. unusual, special pinto beans detect blithely. final foxes use +4268|Supplier#000004268|sxOBZVUf 0dHIcKtCNNOEnmav|15|25-844-621-3098|8509.79| carefully even courts. express accounts according to the bold, bold courts hang blith +4269|Supplier#000004269|V1bTs,quozHr,sLDsYldax W1t6|16|26-122-699-7118|22.77|ages sleep alongside of the ironic theodolites. ruthlessly ironic depos +4270|Supplier#000004270|REjGbhQdRpNMrFHOpI W0nhfA7aJnd69|13|23-696-420-8788|6675.31| the always ironic theodolites. pinto beans detec +4271|Supplier#000004271|jDQ3KyKBTobR86UfM7YVOf|18|28-821-408-8572|940.83|arefully carefully pending requests. special deposits above the unusual requests maintain deposits +4272|Supplier#000004272|0CUBdD9KyyE|11|21-375-837-7482|3549.32|ing theodolites. bold, ironic requests sublate carefully. final, final theodol +4273|Supplier#000004273|oI4ZKslA6m7mvRq4X|15|25-438-360-4251|9360.53| blithely ironic pinto beans. blithely ironic theodolites ar +4274|Supplier#000004274|VpVZ1PbArLtxZ1ryS,9 j5dzYgTC0Dy0 DWM|20|30-322-776-7259|9445.99|packages boost carefully across the carefully special sheaves. c +4275|Supplier#000004275|Z3aD DmnFIMXQqgX2|7|17-994-960-5487|4892.51|regular ideas. furiously ironic courts about the b +4276|Supplier#000004276|WYF6BQrFYxHYLi8eTjTSTjgH |16|26-507-556-9736|1477.23|carefully ironic warhorses. slyly ironic deposits boost quickly acc +4277|Supplier#000004277|MPjnMRh5nwI|1|11-321-241-8114|9768.10|final deposits. furiously express instructions boost fluffily around the silent, final packages. +4278|Supplier#000004278|bBddbpBxIVp Di9|3|13-334-254-5009|6475.19|ag. quickly unusual packages wake +4279|Supplier#000004279|xAqRK,rM,VQKEf1z6dsQGWMP1FpG bKQ|22|32-666-320-9388|3546.40|sits wake blithely at the furiously ironic dependencies. express accounts alongside of the +4280|Supplier#000004280|N9GaeX,haloaFXZbv8 GvkKLtmfEq|12|22-342-968-1844|4369.69|ests. quickly express excuses against the packages promise furiously according to the bli +4281|Supplier#000004281|1OwPHh Pgiyeus,iZS5eA23JDOipwk|3|13-466-621-1767|3380.49|counts cajole even, pending foxes. carefully furious dependencies are furiously around the ironic +4282|Supplier#000004282|4n7vK UT5kS2jxmr ig2NKoX3hTE6C,1Pv4kd|1|11-776-775-8792|9923.08|asymptotes sleep busily unusual Tiresias. regular, +4283|Supplier#000004283|rRw1tpxLpTFsglPnZiiV0WOPqsI7KyKk|8|18-474-720-7902|4366.67|equests cajole thinly about the furiously special asymptotes. ironi +4284|Supplier#000004284|CFwZIO2lPV|19|29-247-639-7540|7700.56|eposits. bold accounts haggle quickly for the final ideas. ironic packa +4285|Supplier#000004285|Bf3GqASNwv|20|30-791-206-1171|6372.14|. unusual deposits among the blithely ironic +4286|Supplier#000004286|OsUz6fvS3Y8mGw2rdWmjTh0nEonjaoUduz|10|20-857-469-4413|9656.61|uthlessly closely final requests. blithely even packages sublate quick +4287|Supplier#000004287|jxPJkPGiyju9JoiI7SZpsH|0|10-332-615-1441|2718.54|equests. accounts haggle carefully; fluffily dogged deposits print slyly about the slyly +4288|Supplier#000004288|yGXGVQybcGku GbzCgSunk0I1VwuJhr0i|11|21-289-951-8826|5479.43|s haggle at the blithely unusual asympto +4289|Supplier#000004289|WHUavCR2SCX6geDN5dDRgV|5|15-877-942-9592|9781.97|uests wake. ideas haggle fluff +4290|Supplier#000004290|B9mCZlnMEegzxyWCQKoCZu Vjmh,2VUoxQB7I|21|31-372-730-8796|8659.68| nag. silent asymptotes affix blithely bold, silent +4291|Supplier#000004291|lsE8N3KbWXyRK|19|29-134-838-4532|245.09|ully ironic pinto beans affix according to the furiously unusual accounts. u +4292|Supplier#000004292|HfbQVVt2syII|21|31-925-781-9911|3130.61|ounts. thinly ironic pinto beans detect carefully even accou +4293|Supplier#000004293|FCJJkQIJkWqH2T0m|17|27-692-233-6595|1097.77|ording to the pinto beans use above the carefully ironic foxes. pinto beans nag fu +4294|Supplier#000004294|6UDA5xrAlv7ProtXf|24|34-108-107-5528|1119.73|, furious asymptotes cajole quickly blithely iro +4295|Supplier#000004295|8NaPns0LW4U,0XB1r7Fq0n|21|31-599-168-3458|7233.70|ly bold requests. packages among the packages print carefully above the +4296|Supplier#000004296|lGv5AitD162GvCml5|15|25-592-414-6464|2915.23|st fluffily. ironic packages cajole quickly furiously final instructions. +4297|Supplier#000004297|KIHeBupr0uzfXq20Nk2M0Xjv|22|32-324-959-1436|207.99|ackages hang regular, ironic accounts. carefully regular d +4298|Supplier#000004298|P8E7PlS843vGQOOAYGlNyzSbt0GgFnXR80rGVhm|9|19-287-258-9572|6544.82|out the blithely blithe pinto beans. +4299|Supplier#000004299|Zts8TmM,GMLCPSoGqKrG1D412tvBACj3bv38OafK|11|21-259-114-8940|7773.77|. slyly even courts affix carefully quickly express reque +4300|Supplier#000004300|qZSf W P4g|11|21-421-201-4720|4881.16|rding to the final somas-- iron +4301|Supplier#000004301|nIUnIw9POvuLE3wthPav1ggAhpnseePXZVNSwjEv|1|11-318-701-1107|8670.57|rts-- requests haggle caref +4302|Supplier#000004302|y8J,yet1HbesuNlD61o|8|18-532-605-1462|7307.76|ring courts. idly final do +4303|Supplier#000004303|18UNNewASiYvJp3zc52pp|20|30-427-742-6158|4632.63|ts sleep carefully atop the furiously un +4304|Supplier#000004304|hQCAz59k,HLlp2CKUrcBIL|3|13-419-753-1541|630.97|ly unusual frets? quickly even sentiments nag. quickly final deposit +4305|Supplier#000004305|qNHZ7WmCzygwMPRDO9Ps|19|29-973-481-1831|9571.83|kly carefully express asymptotes. furiou +4306|Supplier#000004306|SkZkJZflW5mDg9wL fJ|10|20-911-180-1895|7767.63|uickly regular ideas. blithely express accounts along the carefully sile +4307|Supplier#000004307|3B4MszlyIqQd7kHWL,Nl5X473Q4|12|22-283-448-9282|544.08|ross the express accounts. quickly regular r +4308|Supplier#000004308|fI3i2eXnbkDB5XRO,xzt7wQPta82euOPF|23|33-520-342-6702|1256.50|regular foxes poach blithely about the slyly final r +4309|Supplier#000004309|WkrosU8iLiCgpG58VmR6liRUHlu4vbMIPfcpM|1|11-949-992-8497|7025.86|ar pinto beans sublate. slyly regular instructions haggle permanently permanent, regular theodoli +4310|Supplier#000004310|cROEuV4qJQZ2X|5|15-207-372-9865|7675.20|s are slyly special sheave +4311|Supplier#000004311|I8IjnXd7NSJRs594RxsRR0|22|32-155-440-7120|9128.97|refully. blithely unusual asymptotes haggle +4312|Supplier#000004312|qFuileVgNCS0DR0|11|21-343-959-5060|630.60|e finally ironic excuses. ironic asymptotes hang bol +4313|Supplier#000004313|X1jAOOmvD79Pn2qjmVW 4wPnM8pd|14|24-218-712-4324|3439.21|nic packages. slyly final depos +4314|Supplier#000004314|a8oKtPO4E4Vy0JZJFeh 9tOyD|19|29-878-156-8500|6729.91|omise slyly among the carefully regular requests. pending ide +4315|Supplier#000004315|sHhH7fRDqXOlVsSgKKs9yvtX|4|14-529-124-3299|5547.68|at the furiously slow theodolites wake alongside of the blithely pending dependencies. quickly iro +4316|Supplier#000004316|oxFbp0mxb65wkxW PMeG2HvYQlEx|17|27-161-639-8691|1892.88|nts wake. quickly ironic deposits are. carefully final ideas sleep among the fl +4317|Supplier#000004317|y1YhEPlqLELpjlDIreqNgCa45zu5,8|19|29-259-919-1658|-389.86|usly special pinto beans. blithely express packages according to the bold deposits might are al +4318|Supplier#000004318|TnLaWeEdA5Hz1yKKi07g6fPtvCF 0fh3Lsvi|8|18-448-117-5693|1541.19|key players boost blithely regular ideas. accounts haggle multipliers. f +4319|Supplier#000004319|g7IuprD7peH 6zBjC9|13|23-988-463-4250|6673.37| blithely final instructions. even courts among the carefu +4320|Supplier#000004320|SuttVaNiEeRVK3ZDO|23|33-370-752-6261|1421.43|riously about the pending, express requests. pending, bold co +4321|Supplier#000004321|0iWoD dOSEzWIEXaqJsK9mmvwRkfA8e,DrG2|20|30-281-415-7493|6965.04| furiously special theodolites. ironic platelets wake +4322|Supplier#000004322|nYtHA4IrK0n|21|31-903-735-8227|8397.04|. fluffily unusual packages sleep slyly ironic deposits. furiously even realms affix after the +4323|Supplier#000004323|bjfhIi7O8aupWBcSGrT2QSmJMJusT4t53,2|7|17-915-630-7352|8072.29|tions. slyly regular theodolites kindle carefully +4324|Supplier#000004324|oihY3txJ6f4v|12|22-286-705-9057|6014.97|ets. slyly regular accounts sleep ag +4325|Supplier#000004325|TaJqWrGCiBJaWMzRTT|12|22-851-405-3707|1629.40|ly alongside of the carefully special dolphins. slyly +4326|Supplier#000004326|k8FF24WDtgCNFAnScf8Ap8UfmA79 Cr|22|32-276-548-3899|4353.96| the express foxes use carefully furiously ironic packages: slyly even foxes engage furiously. th +4327|Supplier#000004327|aSEg2s hxzlSPcSLpIe9Ur3QQ Yep,IS8DWaZpME|9|19-813-118-4273|2938.08|final, even accounts according to the blithely regular notornis doubt always deposits. furio +4328|Supplier#000004328|euddbWZRcVMD3W|21|31-903-316-1844|-704.56| detect carefully even requests. never even instructions print furiously silent depo +4329|Supplier#000004329|KwMRyzn84uJyuMeKQGbzyeGAhdwMV|10|20-930-692-3265|3374.92|ly idle Tiresias. quickly final instructions cajole. f +4330|Supplier#000004330|ocO7oPwQkD87|18|28-504-820-5660|4213.80|kages use bravely. quickly bold foxes sleep furiously silent, regular packages. quickly regula +4331|Supplier#000004331|uDff,rpaiXyRnD5ZyKfIkx7f1raW,|16|26-797-748-3053|1319.60|packages use! express, pending ideas are quickly silent, even ideas. +4332|Supplier#000004332|0nenh1uRWEmG3t VAFLaPM41N0NPdT2eRvRKrun|17|27-134-141-5401|440.29| carefully ironic ideas across the deposits impress quickly above the final, specia +4333|Supplier#000004333|v EAzCeG7P687EbgPsepV2qMx2c|17|27-231-133-1020|5882.10|ions sleep closely furiously regular ideas. quickly express multipliers are finally +4334|Supplier#000004334|CxKEAJiy,Pakv6p|13|23-906-378-4436|1558.14|ngside of the blithely regular foxes. furiously regular dependen +4335|Supplier#000004335|sLgbkSGstpaB|23|33-111-821-7078|5675.39|its. regular warthogs cajole whit +4336|Supplier#000004336|H bmF6KcUBRpw,RsxymvNUS3oyCAdx iKhSLm|10|20-154-952-4754|3214.71|uffily express deposits grow slyly inside the c +4337|Supplier#000004337|YJ6OMKPhIzCDJz7GyBGS|14|24-473-518-7862|6692.01|final gifts use blithely regular, pending deposits. quickly final instructions haggle blithely +4338|Supplier#000004338|fHVztcdO93vdaB2W9p3O67V20|8|18-464-476-3509|6727.61|ending, final theodolites +4339|Supplier#000004339|1t0o3fQ1vlWu1q7SztW2W,KfdQRU5z|21|31-791-265-5920|10.68|al requests serve fluffily. quickly even dolphins +4340|Supplier#000004340|eKNZibEcZCqmfFZ7|5|15-936-199-5467|1350.22|latelets. furiously regular packages sleep accounts. blithel +4341|Supplier#000004341|ea8KZYvO7amq8A|21|31-310-408-4095|9026.06|ackages nag quickly even dolphins. slyly final foxes are carefully. instructions accord +4342|Supplier#000004342|aU7fCjZMxPLksajZE9|4|14-292-990-3385|9126.19|y bold requests sleep slyly carefully final ideas. blithely pending +4343|Supplier#000004343|GK3sbopqrQEkWLMvVBFCG|3|13-972-182-3578|8927.42|eans are fluffily. ironic dependencies aff +4344|Supplier#000004344|IY5K7HYa7wpHcgWwftQqhggicZNr|1|11-675-830-4768|1263.79|al theodolites cajole dogged multipli +4345|Supplier#000004345|p,LAXgSm x19|9|19-934-108-4550|8462.97| blithely regular instructions sleep blithely about the furiously pending packages. slyly sil +4346|Supplier#000004346|S3076LEOwo|3|13-659-999-3852|5246.64|ts. blithely ironic ideas use even platelets. blith +4347|Supplier#000004347|lTRDVw40 ,kGre7qp76vmSv0,5|11|21-432-262-9254|2287.14|y final deposits use quickly. busy, final accounts wake. requests wake carefully alongsi +4348|Supplier#000004348|s4bORUZSvT3oyBVtImdunlU6|15|25-100-602-2619|3934.13|gular accounts. carefully pending foxes are blithely. ironic accounts wa +4349|Supplier#000004349| bfX,xnzRzoxO1bFQY07pLhKeTcS|21|31-725-591-1006|8933.85|gular instructions boost regular requests: busily ironic instructions beside the dependencies are b +4350|Supplier#000004350|zNUzOPG8tpClDjNHo261t6tsoyo8uolPuG|1|11-991-829-4202|-784.44| slyly final platelets. even multipl +4351|Supplier#000004351|OrsC4YL3siq2Ew6B0OTAB5wrBq6q70vf4npk3W|0|10-677-444-3939|2897.68|ly after the express deposits-- furiously bold deposits haggle furiou +4352|Supplier#000004352|Yo FjewjVfTpLnoQ|5|15-522-365-6958|2361.49|eep slyly regular accounts. sometimes final dependencies sleep. quickly dari +4353|Supplier#000004353|b4HfPj9UX35uP2H7a6xohz7orfVX|3|13-619-496-3301|-502.19|yly furiously final excuses. unusual, ruthless +4354|Supplier#000004354|ZucRIz6o8ODOsMs75T3Z,E3XGfR|14|24-373-126-6675|5580.62|against the ironic excuses. regular theodolites lose +4355|Supplier#000004355|57lrIYbQ4 A3jGL7agpVMDeyYPmOMCIXDA7p|16|26-579-152-9709|5437.00|e to boost. regular deposits affix carefully according to the quickly even accounts. s +4356|Supplier#000004356|IHb0QT6803Hr1hmUBafgLwEFZenlW|6|16-127-384-3697|602.18|ccounts after the carefully fluffy orbit +4357|Supplier#000004357|uzdO3uspHY 53emWnBc3eaiMxexRnlN8|0|10-730-748-6216|9751.45|ar instructions. deposits maintain furi +4358|Supplier#000004358|nWiNPJVv84V7MQjLQhScmCTI80h|7|17-778-639-4159|1647.97|nal excuses. blithely silent accounts above the +4359|Supplier#000004359|C3wfYPIhGM0ijUrXO|1|11-348-761-5898|8407.10|cajole fluffily. express packages are. final requests with +4360|Supplier#000004360|w 7kM5J,fqjiqBu4SU0UPEDqspaUEm|21|31-529-726-8170|1126.14|in ideas print furiously final packages. furiously even hockey players along the furiously s +4361|Supplier#000004361|VpieNNJe36AlthVOrdKZwGZLv4resSU|11|21-793-613-6444|5058.50| wake quickly slyly final accounts. carefully ironic ideas sleep blithely after the slow deposits +4362|Supplier#000004362|fBx,Kkc,fj5URrbe 5H0mjBEU5Bv9rtv78qGCh|11|21-779-912-8841|8149.32| ironic accounts maintain quickly after the ironic deposits. bold, regular request +4363|Supplier#000004363|l4P3TdjquM8tDcE|0|10-355-720-3279|2626.48|uriously pending packages use against the regular patterns. slyly p +4364|Supplier#000004364|vmn0tOrePaZ0 VoB9U7Wag6Mlt|4|14-328-136-7973|9157.09|s. final, final deposits use according to the special, ironic hockey players. quickl +4365|Supplier#000004365|s3ZEDTVQBAGeQpWNPvEXjHr QTob|18|28-139-830-2195|271.84|structions. carefully express dependencies affix blithely slyly special acc +4366|Supplier#000004366|tsMIjxAzjpKIfBuP9LDdofvY8qOVmbR4gHKtv|10|20-946-920-5427|1771.51|unts boost furiously. ironic accounts cajole-- slyly final depos +4367|Supplier#000004367|Uyq8GDTYNJNEupawrH0Em1EjQbvtUTQKy|8|18-605-219-6851|5758.29|haggle carefully ironic ideas; slyly pending platelets eat at the deposits. blithely silent acco +4368|Supplier#000004368|oYnKb7WNX5bPZww7PQfQ|5|15-203-571-5716|-418.49|long the furiously express requests. carefully bold deposit +4369|Supplier#000004369|EMZxZ,2uHdU|4|14-842-753-3476|4334.93|lithely. regular requests wake carefully blithely final dugouts. blithely +4370|Supplier#000004370|iWqEZUI3uKQM7yZ,eHDN,BiTr,sTurvp2WaI|19|29-322-800-5692|2148.02| solve furiously. blithely ironic instructions haggle quickly alongside of the ruthless theodolites. +4371|Supplier#000004371|krCpx7vN0eJ06aMie Lh7VwTDz|3|13-840-203-5593|8796.14|dle furiously according to the packages. stealthy, pending courts ought to mold blithely alongs +4372|Supplier#000004372|FbNBgnxh4UOgflhmx2rwlk,NsibLhiS2v2P,wCBT|21|31-425-890-5322|5858.49|riously quickly regular pac +4373|Supplier#000004373|TBPWO5waPyT1GMLBRRq4WVTVDCHTU0l|3|13-911-820-1514|678.48|ely bold foxes. pinto beans wake slowly! carefully iro +4374|Supplier#000004374|wjl9sEGLUm|22|32-808-498-8611|5199.03|ns. carefully pending sentiments cajole blithely: platelets haggle along the slyly special th +4375|Supplier#000004375|Cmr952zcJJuW0xAYc0W0MA7N6vMcCjy|21|31-913-670-9251|5119.96|hes nod carefully. even, special pack +4376|Supplier#000004376|tBGL OhZ9H4XL1KDr|19|29-206-122-4201|7154.30| bold somas boost blithely. regular excuses above the r +4377|Supplier#000004377|FI7 VtABAz5ANSCziB3HrXfX Er1,9bY,g5drnb|22|32-457-129-5517|-624.41|equests about the blithely even deposits haggle furiously even do +4378|Supplier#000004378|N7opoeaydL92E5NYHLqCshFG 2o8F|4|14-709-861-1774|8427.32|slyly pending patterns. slyly permanent dep +4379|Supplier#000004379|VmRefjQS 4Gmb2K6gHO TvQE2hI3|10|20-842-501-6553|-47.54|ly pending theodolites boost fluffily regular +4380|Supplier#000004380|9rwEeXKO1RPtKXk4CVkrd0CGhVR fjeqbovCf|2|12-142-846-2401|1923.06|hockey players wake furiously above the carefully slow deposits. even accounts nag acr +4381|Supplier#000004381|0ZtvVIB4StpIeqEBeKcC,Bu626DbgO|11|21-468-647-8373|828.68|warthogs doubt. quickly ironic requests alo +4382|Supplier#000004382|oB1Kdw13dTxFZkFmg|14|24-355-250-2623|-470.06|thely slyly final asymptotes. even deposits nag over the instructions +4383|Supplier#000004383|4N74qkCDnLQhEFeNc5ZwPuu7fq5HlPDjYGGq0|23|33-319-882-5183|6551.09|ironic deposits across the carefully unusual theodolites wake furiously around the fluffily unusu +4384|Supplier#000004384|uKzetE 3cqrY9ZjQ|10|20-349-160-9300|3129.25|. closely even dependencies engage slyly. furiously even depos +4385|Supplier#000004385|DleSWUNgmzYrzkG|1|11-905-713-8552|233.11|n ideas play slyly regular deposits. slyly stealthy pains nag carefully. slyly +4386|Supplier#000004386|K0GQCM1bN,rFXIgtSBli03|14|24-286-976-7079|-212.95|arefully ruthless platelets. bold package +4387|Supplier#000004387|NNCgyaD52WCcNjxwjazihr5CfN|10|20-112-470-9141|4840.97|eodolites x-ray slyly even packages. blithely express packag +4388|Supplier#000004388|VfZ l1J,mwp4aS|3|13-309-620-1306|4141.31| ironic excuses haggle furiously final requests. flu +4389|Supplier#000004389|hPG BdDEJG21VQCF2 mYbuQiL|16|26-905-849-4909|4051.55|. blithely even packages aft +4390|Supplier#000004390|FaJZ b KdjxA06ZdUW3sdWEAddDR|0|10-361-458-5712|2471.56|ndencies wake closely special packages. carefully regular hockey players believe ideas. c +4391|Supplier#000004391|pcsiJBhSEHuFHNAxR3K c|21|31-930-718-6023|1397.96|unts nag quickly carefully express dependencies. fluffily regular de +4392|Supplier#000004392|Ytfm7E6,SxcWCsQnOATKYWhfRaX|8|18-881-234-7053|8399.47|s. dependencies integrate carefully about the slyly ironic deposits. braids +4393|Supplier#000004393|ol qpu2DWfG|6|16-873-648-3245|5013.88|unts. regular, even requests cajole carefully quickly regular theodolites. ideas are fluffily +4394|Supplier#000004394|womSMdrUmZgHk|16|26-168-154-5389|6359.20|iously regular deposits cajole. carefully express deposits doubt against t +4395|Supplier#000004395|Pxu,IsRZET0 9ToLGbn1s6aYJ60wCZQ|21|31-697-775-2800|1148.27|etect alongside of the final, regular accounts. furiously e +4396|Supplier#000004396|aMZxdgerFmYRUivN6bot9FucS|17|27-397-965-9509|2894.13|nal, thin packages nag quickly bold, even p +4397|Supplier#000004397|Bn4snYDe,bbYwt9xwsUAh1sIg3OcpcFGI|13|23-383-919-8740|8622.81|y after the pinto beans. quickly pending pinto beans +4398|Supplier#000004398|khZZ0CmLip49Zncec|21|31-691-972-3445|7692.44|s haggle fluffily regular, special accounts. fluffily ironic foxes among the quickly b +4399|Supplier#000004399|NF1ju2D6rmDc|5|15-279-100-3454|1911.91|t against the special, even ide +4400|Supplier#000004400|IOiq7lV51a3NmS bVK,4nyhZ0q1iKxM|8|18-686-104-8796|438.68|lithely even foxes. unusual accounts nod against the final foxe +4401|Supplier#000004401|YK6jsDKnht8FqNXjkHRr8X2LSF4U|11|21-116-879-6982|811.78|furiously bold requests. furiousl +4402|Supplier#000004402|acagGfDWzwmS,,WVBsszubFs3LOA8rDRS0I|21|31-756-134-3710|1501.17|cies. furiously final notornis sleep even, ironic ideas-- ironic instructions wake slyly unti +4403|Supplier#000004403|kNuMkd6GbeJpDyrfPTvzGks1bxWRPC|16|26-971-753-9655|5139.45|integrate carefully about the pending, even deposits. furiously pending instructions +4404|Supplier#000004404|nClT5NhhPdxViApjx3ahv ryCqj7R2pj|7|17-844-693-1492|4702.11|gle blithely final instructions. furiously silent ti +4405|Supplier#000004405|YzK43TC6Bi,5glkw6gQoaIpn8FCt|0|10-438-306-6509|2339.69|e carefully. bold deposits haggle quickly among the furiously bold packa +4406|Supplier#000004406|Ah0ZaLu6VwufPWUz,7kbXgYZhauEaHqGIg|3|13-445-131-7603|7118.47|regular packages sleep fur +4407|Supplier#000004407|Ce9 tPpoOZf ydCjHoLk|1|11-242-234-5325|2175.34| forges use even, pending packages. final ideas to the bold, even accounts are car +4408|Supplier#000004408|9PmB6KMkM2JHc7C,LNnj|22|32-621-849-3417|5977.89|t accounts. express attainments detect quickly. pinto beans across the slyly final +4409|Supplier#000004409|MujJ3UmLYP7XaB59nG2|24|34-668-906-3232|5501.18|nag carefully even packages. accounts use after the regular, bold pinto beans. furi +4410|Supplier#000004410|VtBiUBsSO3o0NAEnIYJ0WIefIaSVxVT|12|22-746-291-5588|1386.76|ng pinto beans. regular warhorses need to boost according to the blithe requests. furiou +4411|Supplier#000004411|W WIn1HpTu|23|33-195-172-2896|7164.67|ular ideas. slyly even excuses wake acco +4412|Supplier#000004412|I0qgeo3IuROXe9UJzYPC9gHh51|2|12-179-306-6992|8712.26|ckages after the carefully final deposits +4413|Supplier#000004413|uiswesrN E4D2SWEB iOWZuaiCm|4|14-106-184-2518|4003.99|hely. deposits poach slyly. excuses above th +4414|Supplier#000004414|DvgX74btfxm83Rxf81LTicpT9Eo vTi 0|13|23-302-983-3738|6087.28|blithely unusual pearls under the furiously final accounts haggle slyly ac +4415|Supplier#000004415|89DcPGWO1wkkTHW54sU6B4KTlM0uhBNbowQEuW|2|12-198-770-2454|6869.22| packages. theodolites play about the requests. carefully special requests sleep above the +4416|Supplier#000004416|EfIt1pkykwXHl0MPxb3g1Uo8|7|17-838-557-7867|4064.01| ironic dependencies. quickly express dependencies cajole furiously according +4417|Supplier#000004417|ULVd9moW2Bb4QSaqPmgbEGqoPR0T6TJkA|0|10-266-394-6216|2832.94|low platelets use grouches? furiously regular warhorses haggle slyly against the blit +4418|Supplier#000004418|oPd5t03PK49TWcDnfh0tepnKMievDd|19|29-495-498-4778|3251.81|ccounts boost slyly after the fluffily +4419|Supplier#000004419|caf77jtTtYpzllNVqZe1|5|15-152-489-3407|3073.08|accounts ought to sleep sly +4420|Supplier#000004420|HnfTtxqicsm3JSus2KuVLWDit|7|17-860-214-1250|2286.14| to the slyly ironic theodolites. +4421|Supplier#000004421|nGRojc6MZzQ,efToCrGDNv0xA9dp9,sM7VWo |7|17-356-246-3346|9125.21|uests are carefully. final deposits use furiously. special, sp +4422|Supplier#000004422|L6ZWIUZrNvFDviRomGMNdqZJj|10|20-616-226-2192|3543.20|regular requests. final requests integrate s +4423|Supplier#000004423|cfayNpUnJDcxt5F4BOa0cu|4|14-479-588-3965|5508.13|r hockey players use quickly carefully bold asymptotes. ironic, special foxes haggle. blithely +4424|Supplier#000004424|ZcvRtr 3XfSbgd dsnlk2YG|12|22-350-952-1893|8167.14| deposits cajole slyly. quickly regular ideas across +4425|Supplier#000004425|J8gT8YRvlHyxl|23|33-930-436-9492|5248.21|fully regular attainments sleep. final, express pearls unwind bli +4426|Supplier#000004426|hxCw2eUeF3s|13|23-867-438-2923|6722.47|g pinto beans. regular theodolites by the silent requests sleep carefully blithely expre +4427|Supplier#000004427|68zmtilvyc6Y9iQop8PNifY8o|6|16-701-355-1019|1541.13|ding, bold accounts. slyly express pinto beans +4428|Supplier#000004428|X,luVfq9zRt26X7FqHlRNS|11|21-584-883-7259|5676.37|uriously regular deposits cajole. blithely ironic requests sleep blithely regular foxes. regular +4429|Supplier#000004429|k,BQms5UhoAF1B2Asi,fLib|23|33-964-337-5038|8691.06|efully express deposits kindle after the deposits. final +4430|Supplier#000004430|yvSsKNSTL5HLXBET4luOsPNLxKzAMk|3|13-979-414-7552|-352.56|dependencies affix. accounts during the carefully express sentiments are blithel +4431|Supplier#000004431|c7LwaiKGVuOoaP4F,OjuvEgW |18|28-975-622-6049|1498.09|eas. somas sleep above the regular packages. carefull +4432|Supplier#000004432|IGGsVKCvj25EUkdNPe|11|21-706-977-4611|8321.15| slyly silent packages solve blithely blithely pending packages. al +4433|Supplier#000004433|,264VWuSf9l lOALdP cLe5coAdzMc|24|34-546-668-7007|8585.26|usual courts affix quickly furiously regular packages. carefully pending attainments are idly +4434|Supplier#000004434|3lTQ378rrVynHLSQgnjkmM02Ejpt|1|11-180-941-4852|3568.70|yly; silent accounts sleep. busy, final instructions lose along +4435|Supplier#000004435|xxh8,2sRMljSSemU8pLrpBHp|17|27-577-888-6260|5098.81|rint blithely. slyly regular accounts believe beneath the packages. slyly silent requests sle +4436|Supplier#000004436|ARi9CgTSQU8AkAN3Uu,Byfm9EyS5Ll3GTk8CJJJ|15|25-493-905-3496|3689.64|ggle alongside of the ironic, unusual requests. even requests are furiously +4437|Supplier#000004437|wuRnnRe0zTDSUhLQ5XOIHTrEyjG0qK|11|21-886-119-7265|2832.73| blithely ironic packages. carefully final accounts unwind after the blithely regula +4438|Supplier#000004438|X3YNSKLk1Bmh9OAFND7qUAdEb1I|19|29-834-445-2433|-338.23|ending foxes. regular, final dolphins b +4439|Supplier#000004439|51Ql7PjHlQxz4Cj3k9tU7fITkxrnKSSyTM|4|14-375-810-7277|242.75|uriously regular accounts nag stealthy, +4440|Supplier#000004440|k7bBHpIHLsEP0ITJDYNPLIdoO4WBU|0|10-513-706-3130|7813.19|ions will are across the final deposits. carefully +4441|Supplier#000004441|J2zFGkkuoNgPC1Ut7a|10|20-189-121-5123|7264.55| blithely bold accounts are. final, ironic fo +4442|Supplier#000004442|mn6Q7Troq9QrX6KFoKD8J8LO2|12|22-458-787-5062|4350.96|he bold asymptotes wake acros +4443|Supplier#000004443|8plN7zgSfObYpTaK|12|22-562-907-9718|7520.81|al packages wake fluffily special ideas. car +4444|Supplier#000004444|9pkHLNCjv6v5Np7NrXK3SGxCzjZ3,wFHpHgfAqg|4|14-775-539-1796|-715.80|ronic pinto beans are slyly! special foxes are care +4445|Supplier#000004445|nF3Jh5vC3mI0fRfl3ZvGLlrwoW|7|17-336-194-1455|1535.08|tions. pending, special requests are blithely. pending, final requests ab +4446|Supplier#000004446|byuSIxhvO3kNHp62He07,89hdkLkSgvOpGLu|2|12-577-715-7323|2988.85|s. carefully final accounts cajole slyly regular pinto +4447|Supplier#000004447|PogpwSZyu8k|0|10-338-947-9580|5628.13|ooze quickly along the regular, regular deposits? furiously regular packages wake. furiously sil +4448|Supplier#000004448|B,,tc9ChgqLrV0DLAaM7BEWj0cPzlNivt|18|28-528-132-9840|1054.90|en dependencies. even instructions ca +4449|Supplier#000004449|8hAeDPnMsdGI|3|13-711-559-2059|6687.61|ly. slyly careful dependencies ha +4450|Supplier#000004450|cnrtMcKzXBC1VcE0VWpXl|12|22-386-635-7767|9702.55|g foxes. foxes affix furiously express excuses. +4451|Supplier#000004451|0seuMdBqxTid ffYdqI02920lIgeNWv|6|16-209-671-8260|4890.69|nic theodolites. quickly final instructions wake furiously special instructions. +4452|Supplier#000004452|upMbwIgkpO ,In|2|12-140-151-8694|3881.08|gle; carefully bold deposits boost blithely am +4453|Supplier#000004453|Amr vJ83bnZgz |9|19-702-695-5818|8279.88|efully regular deposits. blit +4454|Supplier#000004454|ZooBrOO1fXne9 eBkoNQeN7h1yRgjSA5uJ|17|27-197-527-7677|7963.57|ke fluffily. furiously even requests wake spec +4455|Supplier#000004455|eGuuA8OB9CB3UG|8|18-472-798-8615|4856.11|arefully special deposits about the quickly fi +4456|Supplier#000004456|lIsyPje3x29urlQkP5xeo7EZj1icvC1f|16|26-107-528-7021|5713.07|ounts. blithely even instructions haggle furiously. final deposits wake fluffily after +4457|Supplier#000004457|U0Fik4venKkMZsXCL|1|11-528-153-5161|9563.11|the fluffily express foxes. blithely final asymptotes cajole after th +4458|Supplier#000004458|qX7nx,g7C,BfY TXP9fM,mFt2RTQuWmww|6|16-139-758-9332|4252.93|old packages mold blithe foxes. regular pinto beans cajole. +4459|Supplier#000004459|YAfq3TUQPwTtJSkm1 bRZwqwmnx4|13|23-964-432-2851|-954.91| accounts haggle carefully. carefully regular requests doubt blithely against th +4460|Supplier#000004460|2AqMg2cKVwauYYw8hSpKQEM0M ZuK2ZuWK3T|22|32-777-383-6892|8748.36|ing forges nag furiously even asymptotes. orbits use instructions. bold foxes dazz +4461|Supplier#000004461|HouUSURyqmrtWSk,4|24|34-759-486-8915|5809.62|ajole slyly special asymptotes. blithely regular accounts sleep q +4462|Supplier#000004462|YKHdEsd ycc71j96L49x3QJZlDgGWcCfptb,ugg|19|29-566-781-4546|4290.44|sts sleep quickly furiously +4463|Supplier#000004463|zOQtDLTLsPgF4FI8FNI3dviiJiC|14|24-425-595-4546|2494.32|he quick deposits. blithely ironic packages boost slyly about the +4464|Supplier#000004464|q4ujOwFkYY1N|24|34-684-820-7614|9977.19|lly. ironic theodolites at +4465|Supplier#000004465|jN iIdQjVDgRQID,OTnuqwq8jOKND6q4 B2A6Y|4|14-854-148-2866|713.96|nding packages detect finally? car +4466|Supplier#000004466|hs6l3USBeqgcu2NPt|1|11-575-885-9085|7507.87| final frays above the blithely perm +4467|Supplier#000004467|7BTW,9Tvb7WeewsQrIUhKsAaaU8pLxHNHpG mN1|9|19-443-184-3520|1789.37|ould have to sleep furiously. carefully special theodolites breach after t +4468|Supplier#000004468|QxE2rvpMCkB,nto33X7a0n0FFV|10|20-536-285-9136|8876.42|encies. slyly pending ideas boost blithely. +4469|Supplier#000004469|6yPIM3QyZSomoH9e9qJAl2D|4|14-974-420-9489|2062.63|en sheaves boost quickly. carefully regular +4470|Supplier#000004470|oWfjfJSKW71u|23|33-113-949-6850|3627.38|d, special orbits wake pending deposits. always final accounts integrate about the +4471|Supplier#000004471|N,TTUEjO,aX3CVT2OVCCewJ|18|28-628-863-1077|1396.15|according to the slyly regular pinto beans. furiously ironic asymptotes +4472|Supplier#000004472|IHs4IiOscGjhuviWOc67V|20|30-644-494-3536|23.72| regular requests sleep quickly along the regular ide +4473|Supplier#000004473|NlIlhLtMQr|8|18-319-538-9607|629.78|e fluffily regular foxes print fluffily according to the express packages. evenly sil +4474|Supplier#000004474|uKnr2FIJSxNNmcNacM|7|17-108-300-6090|693.47|the blithely unusual platelets boost furiously even accounts? finally regular req +4475|Supplier#000004475|fCyPeYGEI7pEhuAPIbHMml|19|29-223-770-7780|4045.69|the slyly even accounts. close +4476|Supplier#000004476|QYhizpJ2Hlsgz,waBJ3hvDy1FP|19|29-423-720-3024|2962.92|usual platelets. furiously fina +4477|Supplier#000004477|UCPt,EDnBLp7M|20|30-790-611-6861|9696.12|eve after the unusual, unusual t +4478|Supplier#000004478|zkONvYIl3m8|5|15-839-281-1312|6798.68|ke carefully about the careful ideas. bold packages use. deposits cajole quickly. bl +4479|Supplier#000004479|ukMOheLNsNsdikC8Z3V,pzIDD07|12|22-337-564-1555|6420.74|ckly blithely even requests. fluffily regular foxes sleep. slyly final requests nag. carefully +4480|Supplier#000004480|Y5IH27lNQEwL2PG6Vd0Lzpg49l2xJXNeD|21|31-991-148-7977|1473.27| across the quickly special requests ar +4481|Supplier#000004481|INYNb1pa1Q6,Jf4M2SW,seQO08muuGux1OfpFVk|17|27-779-375-3475|9201.19|uests sleep quickly. carefully special braids after th +4482|Supplier#000004482|Dqr9bYhGZoPmPnONniROzNyiQ4H5JozVNnK|9|19-865-349-1537|4553.50|sly final packages are blithely. slyly unusual packages are final, unusual accounts. unusual packag +4483|Supplier#000004483|6b9Z4 rZqrBXP09mJigTx38AZdcjB7eRCUq8|17|27-981-217-8454|1680.27| unusual excuses. slyly ironic instructions use. regularly unusual excuse +4484|Supplier#000004484|a2qRt2aGJVq8n7SDhwbJ|22|32-423-931-4014|1184.02|y carefully bold foxes. slyly final accounts nag among the carefully even +4485|Supplier#000004485|62reHN90t,nD8jpktr,KmeJ3VeCTvChr|23|33-319-815-4381|9822.30|l escapades sleep carefully among the regular, final packages. slyly regular requests nag fu +4486|Supplier#000004486|3pRN44BCE5KHVsPFfSVlZ4I1lkqL NFudA|21|31-620-794-8415|4446.31|ckages wake across the slyly final asymptotes. boldly special deposits cajole ab +4487|Supplier#000004487| nqVoLuwyj3PE,aNfvYh1QCoFhIi3DfqHATE|18|28-199-342-6204|4077.07|sly regular theodolites detect about the +4488|Supplier#000004488|4Yvuvrvpp2ME6QrFT0BmF|14|24-619-635-8206|2290.37|tegrate carefully about the blithe +4489|Supplier#000004489|51uM6mfE1EZCU|20|30-871-644-3146|3246.97|. carefully ironic deposits use slyly pending platelets. +4490|Supplier#000004490|yV,6nQyyVJgEC5fq40JcEu3k4yBxtJL|15|25-506-743-6374|9879.13|he regular foxes. blithely even requests use furiously. even, bold frays +4491|Supplier#000004491|76D4PO,rYFWZd3HKb4d1xvx5eqEnuc91UjW loy |23|33-784-210-1299|2604.53|ans wake. quickly bold courts wake carefully alongside of the fluffi +4492|Supplier#000004492|nkCFbmQfWzPF0WrGOEHAgG,mMrlPGdWVxXIWRSKN|22|32-642-591-8199|3225.80|ar dependencies integrate furiously about the furious +4493|Supplier#000004493|mYAVD3If6OnkZ5IfYjLW9yO8cN|24|34-843-652-9410|5945.37|wake. carefully even patterns solve slyl +4494|Supplier#000004494|WCQNf4k5wKj1l|0|10-731-233-7409|8818.18|lly regular asymptotes. even forges at the carefully regular asymptotes detect furiously q +4495|Supplier#000004495|BMVfijkPjiRGXfi3ks9qbijFAGg8J7LogWyjBgcS|18|28-312-659-9245|2232.83|quickly special depths. sentiments sleep unusual, ironic ideas. final, even +4496|Supplier#000004496|4iG310kgrHRzOoExb9cMKDJZGhy8|4|14-288-499-9138|4205.40|unusual accounts after the furiously final instructi +4497|Supplier#000004497|wWy3lqTQ9mHqH NiYjc2S,YcK4ycIR5N2RmkC|10|20-121-548-8399|3172.87|y around the quickly ironic packag +4498|Supplier#000004498|LklMZK6XmdLT|2|12-964-166-6801|7474.04|the pinto beans? furiously p +4499|Supplier#000004499|TGQ0 J5kioguBpMOth6V,KCk5IdBiFUSnMUF9k|4|14-993-212-9387|5219.17| bold platelets. platelets poach carefully beyond the always blithe deposits. bo +4500|Supplier#000004500|gLmZw7Vn9naI|7|17-199-768-7129|2670.33| courts sleep grouches. special, regular foxes are slyly regular pack +4501|Supplier#000004501|qGIeaf7uYt0GV|20|30-627-214-4834|9281.72|s against the carefully even requests c +4502|Supplier#000004502|oxMw XELHqKWhtw|12|22-478-458-6386|4963.65|ow slyly unusual sauternes. slyly regular packages across +4503|Supplier#000004503|XEsadZv5GN9PbM|1|11-689-225-1159|9141.11| cajole fluffily above the even, careful d +4504|Supplier#000004504|KAFOGWHoQcHdUEnK7RKVTXDIFaC7tShuUU,SZTzY|23|33-187-929-3736|6770.97|pending requests unwind accounts. express, furious platelets use sly +4505|Supplier#000004505|zTpndt7feYG7HzAYRh|8|18-475-174-3168|2645.36|sts along the accounts cajole unusual, unusual sheaves. pending, regular asymptotes sub +4506|Supplier#000004506|Md8nFbfkJvccp4XYndOA8raYFqzmXz,3|2|12-178-894-3167|7559.44|ronic requests according to the unusu +4507|Supplier#000004507|BCEriobperTXRQwdmGMvPIiJJgfXPZ3I6S0|8|18-157-881-5343|-434.97|ut the slowly bold dependencies. blithely regular deposits cajole blithely. theodolites haggl +4508|Supplier#000004508|soV5L8Y Bwu,JNNU0riER|7|17-489-932-7585|2438.21|y unusual accounts sleep carefully slyly sly accounts. special ins +4509|Supplier#000004509|j1WZbSUfYT5czrsNDQQ6|9|19-937-913-1699|2018.11| furiously. slyly bold dolphins alongside of the furiously ironic deposits a +4510|Supplier#000004510|dCqHQvJF0z8gik48C45GZVYY0bG|12|22-118-265-6408|3409.44|p carefully carefully special deposits. carefully bold requests sleep furiously carefully slow +4511|Supplier#000004511|mmkN4DJOYUIVmrmQBevFbXQPJb6a1S,so2h|15|25-583-107-7205|1491.90|requests are furiously along the carefully pending foxes +4512|Supplier#000004512|ueJ8icaozLDQqtqneSNza0r42334fnUM|11|21-744-220-9792|3269.09|ckly special sauternes. ironic excuses nag fluffily careful packages. final, idle +4513|Supplier#000004513|lauKFcAEpo5xzRS2k1Enr6N|5|15-410-158-2722|7536.56|ly ironic requests use slyly. regular dependencies wake slyly +4514|Supplier#000004514|uztivbocfE12vWkYT6ih2Tif|2|12-420-685-9694|-37.41| wake carefully across the ironic warthogs; furiously even +4515|Supplier#000004515|IKvEnE,Bfi8ah91DRQTDFfp6CxGtf0k c|14|24-845-552-3746|7158.58|uests boost blithely permanent deposits. +4516|Supplier#000004516|1RPfefnhZtmuG1cvQE,cYOo3wljLDp|6|16-689-234-9310|5236.68|ar deposits haggle about the regular ideas. ironic platelets are. sly, special +4517|Supplier#000004517|28mZOzHm U9ec4V1ejAyYD2qMfd|10|20-315-592-9023|-124.97|e furiously special pinto beans. bold sauternes are carefully express epitaphs. quickly re +4518|Supplier#000004518|vAr,embfAOX GPSvPJvGQeo0qnNsrpA|17|27-238-507-2455|9572.42|xpress asymptotes affix quickly regular accounts. blithely final requests after the carefully specia +4519|Supplier#000004519|Z3Ak,pdQgQeaf5ni|6|16-984-534-8641|6742.37|ites. final requests are. express, ironic foxes unwind slyly regular deposits. furiously ironic r +4520|Supplier#000004520|Gzt3 AP,69|0|10-306-516-3320|5981.21|across the slyly express asymptotes. final, even deposits af +4521|Supplier#000004521|myD3l0 faInlRXsP,vwKzbHN,VjcLoQjZLnLL7WY|24|34-876-469-8307|6262.06|osits according to the carefully ironic deposits wake slyly bo +4522|Supplier#000004522|xXtCKwsZDArxIBGDfzX2PgobGZsBg|3|13-691-847-9748|7453.60|eas. always blithe deposits detect slyly after the +4523|Supplier#000004523|SoBu9nw6O2tVpM|17|27-301-120-3294|2650.35|special requests. furiously re +4524|Supplier#000004524|IyJvzEpoSw7uPDKIueG8W9|9|19-259-768-7485|3487.78| warthogs haggle. requests believe furiously. slyly ironi +4525|Supplier#000004525|xQNjiof2bcjYhUA2jSqApTzKMHcGS|18|28-145-832-7821|9182.14|g the furiously ironic deposits. bold, final depths cajole carefully +4526|Supplier#000004526|idM5u6EjYXrwcM76vkb89JKLkvxV3d5DGYZWREbT|17|27-660-230-2130|3236.31|eas; ironic pinto beans cajole fur +4527|Supplier#000004527|p pVXCnxgcklWF6A1o3OHY3qW6|3|13-870-141-6447|6716.33| regular asymptotes. ironic braids +4528|Supplier#000004528|roNpvAnFYHwel1QGrgILQ|24|34-579-396-7340|8382.33|efully bold packages haggle. even packages sleep about the quic +4529|Supplier#000004529|QQ uftaLsBYd6H8IsTk|7|17-904-330-6474|2824.65|ickly slyly bold packages. slyly express foxes wake quickly +4530|Supplier#000004530|dlyR8vkFME0DB|16|26-777-914-4325|6854.00|even, express accounts are carefully express, regular foxes +4531|Supplier#000004531|rA16jlU8ajPt5BbTjk|5|15-805-355-4079|1198.67|ily. dolphins wake slyly. blithely final asymptotes by the slyly express accounts nod +4532|Supplier#000004532|6wzQLgAZY,p9BY|19|29-474-615-8869|2966.53| slyly above the regular, special braids. regula +4533|Supplier#000004533|FFkhoNBaaIQIdC8|19|29-671-575-5137|4489.14|ending packages cajole slyly furiously ironic deposits. carefully express ideas are quick +4534|Supplier#000004534|CKOK7nSpYr5KnjqJXaZktTbIiLFkbo h|6|16-798-297-4249|8980.24|es cajole furiously. blithely bold pinto beans wake account +4535|Supplier#000004535|WbG6Odl5StA,,|10|20-468-495-4445|8971.64| ideas dazzle blithely packages. even deposits cajole after the special packages. quickly +4536|Supplier#000004536|YksS84J0LnT9,c1,cR|5|15-543-666-1460|7441.41|ly about the final deposits. quickly final instructions across the quickly even d +4537|Supplier#000004537|Ac1Iq2Xcl tTCTWf,|13|23-517-402-3368|6795.52|ideas. furiously unusual deposits across the dogged pinto beans cajole slyly pendi +4538|Supplier#000004538|zVurdAlvjH9,qaN7TVPe5ex|12|22-505-386-6954|8328.48|tegrate sometimes. quickly bold theodol +4539|Supplier#000004539|6D2Ym9DGXSZ2V wkCw7XfIO|20|30-481-887-6054|-101.65|ding foxes are furiously after the carefully bold pains. boldly special accounts are qu +4540|Supplier#000004540|ca9YQJ0iLbz1S9PKNPg93xOnxHoDp|16|26-507-778-5227|5507.07|ests according to the even, silent dependencies detect slyly al +4541|Supplier#000004541|OJPZxwlH9UaQ,pw,gt9bFwj w7|24|34-792-190-3269|1382.67| packages cajole regular tithes. slyly ironi +4542|Supplier#000004542|NJSbLJDroYG2y1r3rDiKg|3|13-724-992-7449|4409.19|ously slowly silent requests. blithely pending instructi +4543|Supplier#000004543|u5lTjMJdEIdwOky3Ucx5San0iSf|9|19-673-572-2807|7300.42|n accounts do haggle. carefully ir +4544|Supplier#000004544|XRxa4w8u1Eos9g1B|19|29-909-923-9476|2154.35|wake. quick packages are along the foxes. b +4545|Supplier#000004545|E95PvYGAPAuFTklKknHvn05xWQexEGgjVl Iii|13|23-997-713-8734|5061.42|ng foxes! ironic pinto beans cajole. quickly even foxes haggle slyly. ca +4546|Supplier#000004546|PAatX8Ld8fktljR0o2Dft1|7|17-567-331-1194|2215.23|gle blithely. requests haggle. slyly regular instruct +4547|Supplier#000004547|,eGSwy2GVX6A|18|28-885-221-4058|5402.44|al packages. regular brai +4548|Supplier#000004548|cw9sLpfz3LydurPH0Mqqq|11|21-735-154-5879|2892.77|sts grow according to the quickly silent +4549|Supplier#000004549|el1dFQyFo1y8scPw04LJ8LuHfuuH,E9GPTl6NGg|12|22-238-393-7881|7754.45|regular excuses. quickly ironic accounts use express ideas. bold pearls across the f +4550|Supplier#000004550|fFCMU5U1BlLQLVQ|1|11-116-978-6838|-857.61|y even dependencies. final packages nod across the blith +4551|Supplier#000004551| ePu,AbqAQT8kZNqfBqaW6m|15|25-210-326-9058|5117.85|ges. quickly final requests haggle blithely after the even accoun +4552|Supplier#000004552|eRwxvVjYTpamQHXlldIxF,q8C|6|16-360-747-6385|4723.25|ts against the fluffily ironic packages sleep carefully of the special packages. requests +4553|Supplier#000004553|6KfjoVWk9Bk7,orN6mBZjJcS7q|4|14-788-173-5161|5656.44|lly regular platelets sleep quickly. final requests +4554|Supplier#000004554|nf,ST6fo0Seuzs43MgCBLm,|13|23-837-524-2882|3860.21|s dazzle carefully above the blithely regular requests. q +4555|Supplier#000004555|,3q9,Ejcr,PN|5|15-133-503-3928|9511.52|to beans haggle even package +4556|Supplier#000004556|8P9TFgNkp0Y1kxVXJcj2,fTP,KAGvnz|19|29-157-838-9149|6117.41|s excuses nag. bold ideas sleep furiously bold requests. carefully final theodolit +4557|Supplier#000004557|l6HoViEM EZ7fETpoO|17|27-179-479-9723|7375.80|. foxes doze across the slyly regular p +4558|Supplier#000004558|Thr0PyLNWrrzzeOgIFqOXFg5w6MVs5fzJc|18|28-888-147-2005|4060.17|pinto beans across the furiously bold packages cajole fur +4559|Supplier#000004559|TYEl2DZQ,4l1fTpBz|4|14-770-182-6947|4294.49|ns wake according to the quickly even request +4560|Supplier#000004560|W4YY84MlN5Im3NBHfZeyLJ8X9mgGof|19|29-175-506-8008|4557.12|ess packages cajole quickly ironic platelets. ev +4561|Supplier#000004561|FD8UvKGGaluDB|15|25-447-938-1887|7505.56|eep among the even, pending packages? never +4562|Supplier#000004562|mK6fViGTNklGH1,PlBWm8O|15|25-214-849-2528|6958.96|l, express instructions boost. e +4563|Supplier#000004563|D1EmBMVDK3guuyjftmL2wglpCJdttI05YyiL|2|12-805-940-5244|5245.16|r, special ideas haggle. even foxes +4564|Supplier#000004564|PaLwrQiB9R68qCiz22ZFcaF|7|17-588-364-7633|-653.75|slyly regular pinto beans across t +4565|Supplier#000004565|GgSRL9IVEnJ3zS0xzznWky1VBqWmE|13|23-874-502-4312|-663.75|osits according to the slyly regular +4566|Supplier#000004566|mAKi0qJOdVHuta0zJx3WUr4er,6QJbSrUXRFN0fN|6|16-328-958-2628|7272.00|; quickly regular requests haggle according to the blithely ironic foxes. blithely express de +4567|Supplier#000004567|RDXhhMp6b0CckZyeqVEqW5VEPoJBmEbNrjKa1iA|13|23-151-452-2399|8524.09| regular instructions doubt about the regular, bold depths. pending requests according +4568|Supplier#000004568|BtrYY,xTReh,wSJe|23|33-744-982-9995|9519.14| final courts. slyly regular requests abo +4569|Supplier#000004569| 2vqHMjpdviyiM|18|28-574-392-7943|2262.56|ly unusual accounts. regular, express ideas cajole ca +4570|Supplier#000004570|ZUwh9xolomrElngKfKQwrX9a0bgmUFqdL2V|18|28-530-319-2008|8796.87|s run blithely close asymptotes. foxes haggle blithely. quickly final foxes could have to sl +4571|Supplier#000004571|1m0rVbllS9EcE2yyAQg x|11|21-693-938-7784|7114.79|ts use blithely regular excuses. carefully express cou +4572|Supplier#000004572|UEhXFycf0WuFzmWLyAKw8G7HLuydjiyml8O1tdvh|5|15-476-159-7207|6274.69|ess dependencies along the furious, final foxes sleep quickly carefu +4573|Supplier#000004573|B1NwUSk0MfhhyceTtw|1|11-815-500-7034|-220.67|lyly even requests? regular, regular accounts eat permanently silent, regular instruct +4574|Supplier#000004574|1HvGwnVueZ5CIndc|3|13-699-599-7738|7151.45|s against the regular instructions nag abo +4575|Supplier#000004575|ye i142yCN,cbqGaaZOB|15|25-627-847-2252|6500.04|yly above the ironic platelets. ca +4576|Supplier#000004576|HaO1IBZTzOmgW7mcDYpWUt47vhw4BUW|8|18-418-401-4991|3154.31|g to the regular foxes; carefully regular pac +4577|Supplier#000004577|nDL547QPA4p7AkEq4QurW|22|32-382-840-6817|3519.47|, regular ideas hinder carefully across the special, re +4578|Supplier#000004578|Gl81NeKLP6fg|9|19-291-609-2051|2749.56| carefully ironic deposits. silent instructions across the ca +4579|Supplier#000004579|K5nhdAhx6aGpbcRNj0|6|16-946-122-1848|9756.30|ly regular dinos. regular deposi +4580|Supplier#000004580|uxQUct BmDbbJF7QZO3 ,mS7AY|0|10-765-827-9785|6205.73|final packages nag carefully according to the blithely iron +4581|Supplier#000004581|dUriHvsartZ9Yj|1|11-817-998-5780|6754.33|quickly after the carefully regu +4582|Supplier#000004582|WB0XkCSG3r,mnQ n,h9VIxjjr9ARHFvKgMDf|22|32-587-577-1351|9094.57|jole. regular accounts sleep blithely frets. final pinto beans play furiously past the +4583|Supplier#000004583|7t5SYAhtUcJyMR4pxgccY1K6CO3CDpoSk|13|23-713-629-1992|5466.36|sts. quickly regular requests at the unusual excuses thrash furiously against the stealthy, re +4584|Supplier#000004584|rc4aFmmnjUoqFocsC|1|11-370-348-9815|-942.93|counts sleep carefully silent ideas. +4585|Supplier#000004585| G7bSRD0e9e9y9ZsPwAUCkDdFMtUnx2LR|19|29-735-172-5379|1097.53|lar excuses cajole furiously. final as +4586|Supplier#000004586|IdiDj6697TbZsAr0L9r8,t|11|21-349-331-2117|8115.93|lay carefully pending pinto beans. quickly express platelets c +4587|Supplier#000004587|VwFxI9R zz9W4xjWp3kjiUaaSyg6OfQ|6|16-747-482-7688|2526.66|ct across the slyly regular excuses. slyly bold accounts grow. slyly ironic dependencies haggle s +4588|Supplier#000004588|Al7amqeLcVEZkcGjmdpNGWp0VC93q0j|3|13-176-439-9985|3611.31|fully final ideas hang furiously furiously expr +4589|Supplier#000004589|9Pl7fd3JMgKB7PLwxcU63yI7mCD463Ps2GKd|16|26-562-123-8958|6349.44| regular deposits integrate. carefully even deposits haggle slyly slyly pending i +4590|Supplier#000004590|fFt6dJwXmsW6OHdt5rxKTwPfNCNyCoZ|22|32-451-236-6213|2079.81| ironic accounts. furiously express deposits nag slyly. requests haggle blithely befo +4591|Supplier#000004591|fZx3auBseAnpUTPWcLD9r|5|15-483-963-8657|6290.11|ect blithely slyly even theodolites. furiously +4592|Supplier#000004592|6eoAjyJrWXrsoJr2HelM8zc4ZV5sW,d2je|6|16-245-427-9321|5937.53| requests. quickly ironic deposits across the s +4593|Supplier#000004593|yvhjhCRGmsENgm9SXwjQAUdNn3|1|11-519-428-6393|9095.31|ptotes? sly requests use about th +4594|Supplier#000004594|IBOTVXDTSrtq5WLsjz|24|34-759-565-6958|6890.36|sual requests boost slyly. +4595|Supplier#000004595|4ZcOVuZWis2Kg4MBnJ6mVyNV,TEFKCo,pF|9|19-908-449-6024|7245.00|special foxes cajole furiously-- fluffily pending requests kindle. blith +4596|Supplier#000004596|ZTq,wSuzJJ6qXC3vu DJ|13|23-786-524-8317|5788.66|egular waters was among the final packages. asymptotes nod fluffily blithely unusual deposi +4597|Supplier#000004597|gKuHIUE7XWqK9ZDCA,Kp0jFza4PvTq,RtFF|6|16-130-150-6625|4102.46| packages cajole. regular packages wak +4598|Supplier#000004598|2dMouY,hxVXwMYUUsZldxhh36WRaI|23|33-502-656-6262|8.77|iously final decoys haggle furiou +4599|Supplier#000004599|2l OlFEd9pU4Qv2gjjGPySzp|21|31-847-997-6293|2247.49|iously about the regular warthogs. ironic, b +4600|Supplier#000004600|iGVGwr0UYHqbt5Mg|12|22-756-980-3416|8513.63|kages haggle furiously above the carefully bold +4601|Supplier#000004601|8gjWZovIX6Ay8eMf4L RwDAaIS4J|5|15-139-205-1544|9590.05|rave deposits wake slyly against the bold, +4602|Supplier#000004602|ndt1H2FutNtnKtrrNP1v|2|12-977-596-1835|6086.74|t the blithely pending dependencies. carefu +4603|Supplier#000004603|OUzlvMUr7n,utLxmPNeYKSf3T24OXskxB5|23|33-789-255-7342|8913.96| haggle slyly above the furiously regular pinto beans. even +4604|Supplier#000004604|o4cyLBnefy7,k6cvp5|11|21-344-644-2585|6002.23|silently. slyly thin accounts p +4605|Supplier#000004605|bAI2O8hXfdMzOrlfZM0Uw3c3Xq0GiEYUPrBKdu|1|11-719-718-7122|7487.86|ans haggle slyly according to the express, express tithes. express, final ideas engage +4606|Supplier#000004606|LYP75ac0BayDRW1pUb8dTGaHwBW4FJihWPGix|7|17-938-710-2625|6507.55|onic dependencies cajole across the express, fin +4607|Supplier#000004607|1,JCa7N z8envgRg68s VnuimNFmBdM5UajUFn|23|33-119-146-5411|8423.81|equests promise furiously. careful +4608|Supplier#000004608|uRozWbS7jafTAQz4JfJemS7|19|29-293-800-6931|6988.85|to beans promise slyly. express packages wake fluffily quickly ironic req +4609|Supplier#000004609|nFb0hCtvR6ePk0g5Brq5z|17|27-238-471-9748|1699.47|ept the regular, final packages. bold dependencies cajole blithely about the furiously ir +4610|Supplier#000004610|G7HRcZ0fXIKP6bA|13|23-594-493-2109|5021.84|ld requests. even foxes m +4611|Supplier#000004611|jN1IXxhYKkoo14ZF2D6is5gY9cd2bP3NkLYfI|23|33-860-499-1392|8030.12|the idly careful ideas. thin accounts are carefully about the blithely regular gifts. e +4612|Supplier#000004612|1,9d89I1URO3QJUeUSKT|9|19-368-689-4694|1298.51|quickly bold foxes. furiously final hockey players brea +4613|Supplier#000004613|B EpzvX7P 8UriPz8PqlZ7KLPKjlCFuHJc,|23|33-670-631-9671|9064.06|nts cajole. express deposits solve quickly. fluffily regular frays are blithe +4614|Supplier#000004614|fwaCEelmSGWyr9k|8|18-716-786-1412|9694.06|ic instructions cajole furiously along the furiously even mult +4615|Supplier#000004615|VjCTCTNrr4Wlbw0pOA22PCmlRSPLCxdp87|14|24-505-841-5759|5091.56|ts engage quickly special patterns. slyly unusual accounts nag around the +4616|Supplier#000004616|9 IFT9AGzoAdA92nIcbyyNoRF2gWCLoqHT4|16|26-807-998-4996|2815.47| unusual ideas cajole across the finally regular packages. spe +4617|Supplier#000004617|KwrEbXCISH1jgJq9vLUDO1,GnoA|7|17-409-401-1510|6642.85|ly carefully express excuses. slyly regular platelets sleep carefully final packages. slyly reg +4618|Supplier#000004618|NVML8aK152LD2bR61rxd7CVY3OwLFT3yP|6|16-140-955-8584|5438.82| even packages sublate along the regular, even accounts. r +4619|Supplier#000004619|YPY8CT8q zwHdcJGvuw97ybDKoSCHw|22|32-333-528-4674|9184.55|ly regular ideas. final, express packages use fluffily accounts. quickly final instructi +4620|Supplier#000004620|z,Qym,C,goAXpPmM0L9s1Slv4|0|10-304-508-9294|-266.83|gged packages. even, even excuses cajole according to the sp +4621|Supplier#000004621|YYLVWPlth5Ve7LGwPoWC8e2Osnb8cunXrS2jW3b|5|15-865-976-9704|1294.22| final accounts sleep slyly requests. blithely ruthless requests detect to the p +4622|Supplier#000004622|tSyf58DedQKRaX 8UF6Qyd,EFDHSmTIHCXYVl|12|22-544-951-5366|2873.16|old requests are above the realms. dependencies boo +4623|Supplier#000004623|1 GzDefX4NG5ZBKiV0Wy1e6HShk8ltXMZVn3JDX1|19|29-696-973-3625|8112.42|t the accounts dazzle furiously regular sheaves. final, even accounts wake. silently even +4624|Supplier#000004624|VPAS7UP JnPeV88676P|17|27-494-991-9032|9507.70|pending requests nag carefully deposits. fluffily final requests nag blithely slyly regular pin +4625|Supplier#000004625|rqOLaXSrdmykWiyvH9oojHYpJvdWn|20|30-801-894-8169|8020.95|jole carefully. furiously pending r +4626|Supplier#000004626|hJNB,gJhi UN7HWUd5O,lO2v,sbOoM6|5|15-350-407-4023|1146.17| accounts alongside of the regular, ironic excuses cajole slyly expr +4627|Supplier#000004627|70fw74Sx3yIV,Lckmfxn9bG5qJqUc8gLjVNV|1|11-365-279-1630|547.62|ffily daring ideas haggle quietly. bold theodolites breach. carefully pen +4628|Supplier#000004628| UHB1DpXoqeaHtcPTIS8iGuOvW1|15|25-252-104-3152|5774.86|en theodolites alongside of the final courts x-ray slyly ironic requests. slyly regu +4629|Supplier#000004629|g04u2eDpI gUdAe|18|28-368-497-3533|3174.02|y final instructions. ideas above +4630|Supplier#000004630|G4w9jUoC5 b RFZGZivCWr7MbOSOnfke0SQphJr|8|18-538-452-2773|4809.70|ke carefully against the requests. ironic instructions use quickly above the slyly even theodoli +4631|Supplier#000004631|,Se21e0egM,Vodq ABbNjMf25kTmWgkDqQjA1f|17|27-251-691-3991|6584.73|thely ironic asymptotes are slyly according to +4632|Supplier#000004632|pESjh91jVTYjgolst zkE7GW8I|11|21-501-383-8312|7556.03|s the furiously special foxes. +4633|Supplier#000004633|2qnzswVOnbyY52qQtwGrNZSqZfU7z38m5JXR4OY|24|34-331-859-3951|8207.35|ep blithely according to the theodolites. final, regular accounts cajole blithely along th +4634|Supplier#000004634|ukp63mlcz4saSWQr|24|34-196-553-2719|9979.52|uriously. slyly ironic ideas affix slyly! blithely pending theodolites mainta +4635|Supplier#000004635|vkwbTz5xklMiRXZJ,UltDZ8|2|12-182-514-9614|6889.76|the furiously unusual accounts. special accounts use quickly acr +4636|Supplier#000004636|C1TCXzuHEbDa6vNFNfc4UTY6HXnBBp|2|12-986-563-8088|8164.80|ss theodolites cajole regular wart +4637|Supplier#000004637|CLqkPPUi,tCY8YoGXEXv9WTGM8rNMXZ7KLRykj|0|10-250-119-3609|9879.83|uickly. carefully ironic pinto bean +4638|Supplier#000004638|sme4G8DAkyHc9Acldxi|10|20-313-977-4259|3315.18|ages. fluffily final foxes wake quickly fluffily regular +4639|Supplier#000004639|N1QrHm9B20UErKEM6j7CPVkFIM4ur6LEORnU|2|12-299-479-7911|1270.50| regular sentiments sleep within the silent theod +4640|Supplier#000004640|XCEPGbkRufbME9JNnISD1s,GYT5m|1|11-752-437-5966|233.82| even packages doubt accounts. furiously ironic ideas haggle quickly. bli +4641|Supplier#000004641|7GGTBs43lKTN1p0lajBHHAVMYh|5|15-846-828-4921|679.79|es. ironic accounts sleep slyl +4642|Supplier#000004642|q8n,Q1cuji UxsS68haAjAxnG|2|12-563-848-5053|5112.31|d sleep slyly enticing tithes. slyly regular requests +4643|Supplier#000004643|bYPG1 gDvsOhi5RsgY2BxL iWc,L22pmAe|5|15-157-725-3051|-412.42| regular instructions integrate asymptotes? carefully silent ideas sleep caref +4644|Supplier#000004644|kcQP0AmNXMKM6bilcGwy0vKJNvqOSgVuH|2|12-644-122-5089|7408.47|ncies need to haggle bravely above the pending, even asymptotes. packages haggle fluffi +4645|Supplier#000004645|F,lDH 3YCmLaKimHIJwiwwubh9gATstzL|18|28-575-285-6669|3180.81|nstructions mold about th +4646|Supplier#000004646|3x1lY3ib1s18KUTLHv62hNuJS6Vvmfzm|1|11-545-572-2776|8865.60| requests. slyly even foxes sleep about the furiously pending packages. close, pe +4647|Supplier#000004647|uKnMpSwMhPbLExCczBr|19|29-925-745-4751|7128.07|ts. even deposits breach quietly. +4648|Supplier#000004648|vHObeHqAPZqm3ezw7Jh,Jv|14|24-544-770-6453|9508.29| bold deposits are ironic packages. blithely ironic packages +4649|Supplier#000004649|s4kO8xwlrr5KcICe|8|18-300-742-3272|6836.51|d excuses integrate according to the regular accounts. d +4650|Supplier#000004650|UZUhxJGMhAV2dev7aQ3bn4J|8|18-315-989-1584|4236.23|uickly even packages. slyly unusual packages cajole. final, pending instructions +4651|Supplier#000004651|tpDO9wvNQlI|18|28-479-983-9176|4296.53|s are stealthily. furiously brave +4652|Supplier#000004652|JBLbvJR699cR7CRLN Yrlha|15|25-839-866-7553|5717.57|ges nag furiously against +4653|Supplier#000004653|VLrYp0Z46kXnP28unR7soV9mhWk,4hpJb|15|25-257-432-3531|8196.33| the furiously final ideas. quickly bold accounts engage fluffily. fl +4654|Supplier#000004654|Wp2CarCzDh|0|10-228-513-4385|757.62|eans. pending accounts thrash carefully even dolphins-- packages sleep. +4655|Supplier#000004655|67NqBc4 t3PG3F8aO IsqWNq4kGaPowYL|3|13-406-810-5265|4295.82|ans. unusual, final foxes haggle about the express deposits. ev +4656|Supplier#000004656|O5EOVs9MGfEMxSJyvL6Ty9DDRdEs7CTx7Okwwf|3|13-245-567-7166|4923.66|telets. blithely final requests do are quickly theodolites. even +4657|Supplier#000004657|gDxp,eYee87QHUhpvGG7|21|31-886-929-3694|7984.18|ording to the final pinto beans. blithely regular requests after the deposits sleep beh +4658|Supplier#000004658|JZUO7MJ3,f3wFHTS 5dpDkfOWu8KXcgilYxPBDwM|0|10-887-916-4653|-904.63| accounts. deposits use. fluffily +4659|Supplier#000004659|YHCE9GncUg Xqvr37ltoqgFiZ3lTm|19|29-183-219-6641|6772.77|ose Tiresias. carefully final requests nag carefully fluff +4660|Supplier#000004660|1OwbGBtNhsTsTq3RU u80ygIlfnfgw1ztN8mFkI|13|23-961-983-5451|-0.92|tegrate. bold, silent pin +4661|Supplier#000004661|K0pZTfDfq86|19|29-264-237-1888|4533.14|ronic theodolites integrate final ideas. bli +4662|Supplier#000004662|t5bDsOR1SUvZs8yD1sMpFtGR506QfqzsU,NFg|11|21-742-347-3094|446.19| about the final pinto beans. carefully even sentiments kindle carefully according to t +4663|Supplier#000004663|xCapkFMdOs51DBdFwMmqoDQzi7mVo|15|25-122-559-3456|5982.01|l accounts integrate across +4664|Supplier#000004664|7T8vav85kSMKPfV4xG hc2RvSTbT|10|20-159-474-2886|8676.78|s. slyly bold requests after the furiously bold accounts sleep ab +4665|Supplier#000004665|QzSYyKSw9A wbnA,jhfFLombg93XzkonrJEDJk|17|27-771-184-4878|1747.07|aggle quickly according to the flu +4666|Supplier#000004666|WYWOwZg19CNmeGM6707e0|18|28-816-809-6362|1908.02|lar deposits detect carefully. fluffily slow requests wake carefully. blithely special requests solv +4667|Supplier#000004667|Um06UA4W7osWGE8x4|8|18-810-393-1517|1077.21|t the regular deposits. pinto beans boost never special ideas. +4668|Supplier#000004668|iI7gF3y1oKKvjVdA5Klau9w0iD|11|21-896-223-4586|7603.29| requests along the slyly regular deposits haggle carefully iro +4669|Supplier#000004669|iTG66d87I52jSF9t,yvnA|14|24-346-311-7142|-522.03|regular dolphins. quickly special tithes are carefully. bold ideas detect carefully among t +4670|Supplier#000004670|fnLEhL1yrH7XT4N|8|18-302-357-9648|2164.40|e fluffily according to the slyly even dependencies. courts are carefully a +4671|Supplier#000004671|GxMjbOOYNQa1I5r4q7MR|21|31-411-931-9792|9019.36|l ideas: blithely unusual theodolites nag after the slyly unusual acc +4672|Supplier#000004672|WESubepwEnt|0|10-875-268-9253|6394.70|ts haggle quickly furiously +4673|Supplier#000004673|2atTtfdPLk9|5|15-806-479-5890|3734.75| express instructions haggle +4674|Supplier#000004674|iRdZI,bEBIF|15|25-640-960-1813|1866.57|ording to the regular, regular tithes: pending, silent deposits +4675|Supplier#000004675|x,G9aE4O,Qbj5Y4hMEV4aJ6K6kb,dX8BM9Y3U|8|18-159-932-5118|8443.74| are even, ironic packages. furiously bold ideas sleep furiously after the carefully fi +4676|Supplier#000004676|Y5knpC3 tU4BIC2ezyRac1bGTjC qhIb85pA6TC|14|24-571-838-3209|992.92|as integrate slyly deposits. qu +4677|Supplier#000004677|s,bn 4mYd5RWFDkY88z4VdzJ |23|33-455-575-6387|-970.28| foxes. furiously even requests haggle furiously excuses. slyly final dependencies haggle blithe +4678|Supplier#000004678|V6Y4jcdvbV5WYKuymyJ6e59,BeHTEfnIy|22|32-188-313-3119|3459.50| must breach carefully. bold requests cajole. +4679|Supplier#000004679|x6LJifCYpOkV93CIzIvIalTPygUVPH7Jdm|16|26-242-857-6516|6820.19| ironic deposits wake quie +4680|Supplier#000004680|9Z9k,6Ht66AxyubwcsX0Ryig0QcTQQaqJUn|10|20-151-872-6299|5028.10|s cajole quickly among the final instructions. express patterns wake. carefully ironic accounts +4681|Supplier#000004681|qnXYjdfzSKQYH6A8N|5|15-243-148-2996|3342.98|s. carefully regular deposits cajole above +4682|Supplier#000004682|VtLISnpYihV3|19|29-319-528-9629|6427.20|iously bold dependencies. quickly pending packages detect careful +4683|Supplier#000004683|tpUsZwbbAmyGSqX5z6|8|18-108-593-7507|-90.44|ic instructions. express, regular sentiments detect. final, special theodolites across th +4684|Supplier#000004684|Wki58fvNf dzCjJD6dSxzP5i|24|34-253-819-4891|6877.29|ithely. furiously ironic ideas are furiously express deposits. even accoun +4685|Supplier#000004685|PKTo2j2iul2Mli3G7Yh5sMA9J9|17|27-515-163-1892|7682.63|blithely after the requests. ir +4686|Supplier#000004686|w4kwdV7DRwtgOyEjZy|24|34-808-762-4544|2003.30|s detect slyly after the carefully even packages. bold requests against the bli +4687|Supplier#000004687|ufYRNlPDcZOK3ODvaXsT7|2|12-361-516-1331|7384.64|eep quickly above the entic +4688|Supplier#000004688|J,JeiQCrnr94SmBTyvmYIq,Qt|4|14-967-555-6641|9650.75|ccording to the asymptotes. regularly even packag +4689|Supplier#000004689|pCDP,Tl0mnJhK|12|22-455-236-7520|3271.02| requests use carefully carefully unusual ideas. slyly permanent requests across the quickly unusual +4690|Supplier#000004690|a ghqHL5lFn6Pr|15|25-847-221-3199|7087.73|refully regular, regular deposits. slyly regular accounts unwind about the quickly ironic request +4691|Supplier#000004691|irJAzpmW2899MKTk8li8|23|33-821-399-5072|2458.75| slyly special excuses use quickly final frets. carefully i +4692|Supplier#000004692|BxHrGZnE9Ri3jgvv|13|23-807-209-1065|6124.07|haggle carefully across the c +4693|Supplier#000004693|fwDDzY8D 3df|6|16-277-475-5116|9271.27|even packages. quickly ironic accounts cajole slyly carefully regular depos +4694|Supplier#000004694|IMav7cEZ6atsJHJTsKozrW6lR|16|26-159-392-3686|5254.78|deposits are furiously about the slyly brave foxes. even pinto beans are furious +4695|Supplier#000004695|g6KTg6OgozXMozTjPx0|24|34-613-953-3572|7175.59|ackages among the bold, even requests run fluffily pending requests. blithely ironic a +4696|Supplier#000004696|J1qrxkoTf hOSu64XOflrzugGVsC7PGCl9X0|14|24-512-414-6293|5110.61|onic packages cajole slyly along the carefully special theodolites. slyly special requests u +4697|Supplier#000004697|E5HN7GuWngwcN4Xa|21|31-860-407-7060|4030.21|ly quiet platelets; silently eve +4698|Supplier#000004698|omQ3V9E4K4|16|26-206-746-3880|6777.14|ve regular, unusual instructions. deposits boost i +4699|Supplier#000004699|kJWvyn6XXK6xEaHZ5Msw4slwyDuUtplboxV|19|29-893-559-5094|7366.84|furiously express ideas. furiously idle pinto beans sleep blit +4700|Supplier#000004700|X6CQ0GMaMZ|19|29-941-483-3219|7451.98|ar accounts sleep furiously above the c +4701|Supplier#000004701|6jX4u47URzIMHf|3|13-678-533-3515|8402.33|uests. fluffily regular deposits promise +4702|Supplier#000004702|8XVcQK23akp|6|16-811-269-8946|8996.87|ickly final packages along the express plat +4703|Supplier#000004703|jOSVRYK,ju4S8alzzNsp|8|18-912-354-6265|2382.97|detect slyly along the unusual theodolites. furiously ironic packages boost slyly. slyly fin +4704|Supplier#000004704|c,AFOIbNtRT2vA769DW4baD|12|22-365-568-5640|8531.11| across the carefully final +4705|Supplier#000004705|4wKYfBk95vl0Xi0O4RbIv6AA|23|33-191-790-9624|6416.64|tes. fluffily busy deposits wake carefully regular packages. furiou +4706|Supplier#000004706|G bSXMhb0,QoBh|5|15-137-786-2210|9523.82|ously express packages serve furiously. blithely idle plat +4707|Supplier#000004707|96GsC8OoUqtFmvkqcATQ6GsKtoJ|13|23-242-680-6520|3888.35|y ironic somas use across the carefully regular foxes. furiously special deposits use slyly i +4708|Supplier#000004708|gvwZrzlG5X0fA|24|34-982-296-4584|9537.15|to the unusual, ironic grouches. quickly ironic ideas wake furious +4709|Supplier#000004709|H1jn0h7u5BU83CBFXiEo AOJ43k9Ivi|4|14-541-103-5050|7928.18|ly even accounts. carefully regular orbits nag furiously slyly ironic foxes. pending, regular foxes +4710|Supplier#000004710|Toj 73zAveh3lY8sPazxBBfJqHTu3BiNQzsh|19|29-489-471-1872|3959.58| special theodolites sleep carefully even ideas. furiously final ideas haggle. pending, ironic +4711|Supplier#000004711|bEzjp1QdQu ls2ERMxv0km vn6bu2zXlL1|3|13-416-817-3915|7695.22|onic instructions use fluffily blithely iro +4712|Supplier#000004712|zESBY8uYvFunK7|9|19-771-212-2431|4599.41|excuses. regular, pending accounts poach above the accounts. re +4713|Supplier#000004713|av82H9iI2 QXO13e5EMgmWlrss5irga|11|21-494-245-4546|3275.44|ymptotes affix slyly blithely ironic pack +4714|Supplier#000004714|IKRla2xArMmR4p3Mbn8JV8g0|21|31-544-746-3697|4356.85|r the furiously express packages are quickly enticing, regular pinto beans. spec +4715|Supplier#000004715|gk,J1INqQqcWSe2jak0gPtoJ2,4Wq|0|10-766-176-3953|7584.64|blithely ironic deposits. final packages snooze furiously ironically +4716|Supplier#000004716|bwnBuTwZAIoHF3gZHC0OQKfVCTloYLi6o37|22|32-991-620-8920|5145.18| haggle carefully. regular accounts sleep. always silent fox +4717|Supplier#000004717|H,Suh5pN230Ol,ggx0QEh3rrvzyQsq050Lat|21|31-779-145-2016|2021.82|ironic packages nag fluffily between +4718|Supplier#000004718|t1YKEJQ52LiytbQubjUkSqCrjN2WcUe7lA|18|28-733-828-2001|-236.30|nic, express foxes. ironically bold deposits at the fluffily even d +4719|Supplier#000004719|yR99LcVeVOxcix1XIiMb7UpFYblyQaEe aMg|17|27-773-971-6110|8489.12|orges. slyly final packages mold across the even, final platel +4720|Supplier#000004720|TzCxkEFVr8H,f|17|27-757-964-1364|4747.68|uriously regular multipliers use after the instructions. bold foxes nag furiously regular packages. +4721|Supplier#000004721|9ClOEYZoLDciULWl1ljA|17|27-951-525-7255|9326.27|ctions. furiously even theodolites are furiously final pack +4722|Supplier#000004722|lajXkaQUbCc|11|21-850-388-8132|5629.26|lites sleep slyly! regular, even dugouts nag around the bold asymptotes. carefu +4723|Supplier#000004723|q1P7c7ApJnEZu,p1OiwJr g|23|33-408-393-1050|9006.85|he furiously express theodolites sleep furiously abo +4724|Supplier#000004724|lljIGbk5L5dtN3mTFhSXIhi1o6GzBUyC9|18|28-181-636-3740|5150.03|onic requests haggle quickly ruthless pinto beans. platelets haggle across the final, fluffy theodol +4725|Supplier#000004725|qas7l mIPknKZ|21|31-654-353-2168|6526.12|accounts. never even packages aff +4726|Supplier#000004726|m8N5E7nFKIx6ot,I,k W2Ry51a6p1nJksAJko4M|14|24-806-459-3252|1576.31|bold packages. unusual ac +4727|Supplier#000004727|Y96a46PrmSW1dy kMLbiUpyPeR|16|26-940-708-5574|2917.60|s detect. ironic, unusual pearls wake carefully ironic the +4728|Supplier#000004728|kbpbCZVB0xVH4IBaXEI6h|9|19-625-844-5219|7116.49|furiously special deposits solve fluffily blithely final depos +4729|Supplier#000004729|rv3NUIuLN778dpyyEb695K|7|17-697-212-2208|1901.02|ions. blithely even packages dazzle blithely. quickly even pinto beans nag slyly alongside of the +4730|Supplier#000004730|CyTjPvk6AhFUGuAc|22|32-672-773-8939|4390.11|the fluffily regular accounts. furiously ironic ideas boost quickly above the slyly fin +4731|Supplier#000004731|gZw0DyFVivD3RGw9YL0|12|22-912-996-6642|5603.92|te sometimes quick, final dependencies. carefully regular accounts +4732|Supplier#000004732|O0lNJuF1HyRHPDn76GCxEmeBUzop3A3oXZu6Tdxz|7|17-631-234-3786|1079.98|as-- slyly bold instructions h +4733|Supplier#000004733|I1QE,rcg 143Fon8X8aceGxMVxrq2 s|18|28-593-521-6080|-907.81|s sleep about the dolphins. express, final packages are slowly. c +4734|Supplier#000004734|MAOjF4S,gSeBiN|0|10-179-345-4423|8280.28|regular pinto beans alongside of the slyly eve +4735|Supplier#000004735|Jjjlw4ku1VcqB,o,M|8|18-889-669-5620|3713.21|pinto beans. slyly bold asymptotes +4736|Supplier#000004736|G7VqSCrPkWThXkoKSTxzft|12|22-451-229-6907|2938.44|over the express accounts: slyly ironic requests sleep fluffily accordi +4737|Supplier#000004737|RAD8ym7Cr9aKQXMZaL3wgufvUpJgoWMt6gwrFK|23|33-610-901-1458|5366.25| regular platelets. carefully ironic +4738|Supplier#000004738|BcoxAylgpwUi5|5|15-883-567-6484|-472.81|thely special sentiments. furiously final requests integrate special, express dolphins. si +4739|Supplier#000004739|uHhDoxEOZTq7pIQKCCuz8INR|11|21-299-917-1370|4439.36|ose courts after the special, bold ideas are slyly slyly bold instructions. blithely final +4740|Supplier#000004740|yM0TXkhfjpObafbQhuWU|21|31-786-280-5667|8264.47|lyly bold instructions boost fluffily along the slyly final accounts. bold asymptotes are quickly e +4741|Supplier#000004741|DgYOWOKos9R6HBY0aU8WKTJp|24|34-842-816-1139|2060.37|final attainments. express depths sl +4742|Supplier#000004742|4LTtuEAkyVNBDcb9Du|24|34-490-878-3538|9676.47|ial foxes snooze furiously asymptotes. bold requests use q +4743|Supplier#000004743|MRK0lGAlnScFlB|8|18-467-675-5127|6455.36|structions play-- ironic packages sleep courts. boldly regular deposits det +4744|Supplier#000004744|kFOaPWhH42Vtk8uS TRqlrm33t|2|12-466-230-3207|211.02|s packages around the express courts haggle fluffily about +4745|Supplier#000004745|b71xxSn,FAwRhDXB6,|1|11-842-819-5519|2605.87|es. furiously pending requests might haggle +4746|Supplier#000004746|HrNlq N3KfDAfcfX3uho4LqI|6|16-545-107-4292|3820.39| the slyly regular ideas. regular ideas slee +4747|Supplier#000004747|SlTo5iBNO8scvHosXUOaEHFQ4C9K|24|34-218-851-7587|303.60|ts are carefully carefully final ideas. quickly close instruction +4748|Supplier#000004748|sDkHLGM6u7Wrq5Ou8qxiHtyl34euddXgWVcNj|7|17-837-145-2048|7538.07|y bold deposits sleep ironic packages; unusual dolphins nag. silent re +4749|Supplier#000004749|b8Ub1R,e7qTHsOxcDbRd8YyCfkpu|22|32-637-280-7020|3366.64|ggle at the silent theodolites. blithely quick packages snooze blithely express deposits +4750|Supplier#000004750|uPd2OHndWaXR5P2K5qJYGMl5B3uBp4gDo5veZL6|16|26-786-191-2324|8162.76|final excuses. furiously pending deposits about the bold r +4751|Supplier#000004751|QdrAF4F9ea7i3aaPY6 ll8f|15|25-656-710-3897|8621.49|ully. quickly final ideas boost slyly quickly final +4752|Supplier#000004752|r,radSvDXKi04KrAH 4 YGO0IWQo5KYNQ|8|18-700-890-4203|7481.29|ans are above the final, dogged instructions. slyly +4753|Supplier#000004753|jHk1YuGXwB0vVum|22|32-214-561-8802|6116.07|instructions cajole. slyly +4754|Supplier#000004754|txwbkNHK9qxIldQ,uz02VKpB|8|18-520-386-5634|6845.94|onic deposits. regular foxes wake blithely. regularly final theodolites use bli +4755|Supplier#000004755|H0,5c1Q9BSo81v4ysXxIyWA8I|6|16-192-740-5607|8782.52|g about the carefully final courts. fluff +4756|Supplier#000004756|8ZXo2bAqOB1Tm982semhdJvEzMtt9Bx|1|11-495-288-1993|7941.83|cuses sleep express accounts. furiously special pinto beans sleep furiously around the regu +4757|Supplier#000004757|ycr,b1aRZXr600sCi|18|28-232-869-2688|2377.20|ays are carefully dolphins! blithe +4758|Supplier#000004758|KiHe9i4FhMuM7ZjFvcG|8|18-615-389-3776|-401.21|nstructions haggle. final requests wake. blithely bold excuses lose. blithely unusual pinto beans +4759|Supplier#000004759|bxwpbOzAFn2goILzqnWso|18|28-520-218-8613|8740.72|ely regular packages. unusual excuses sleep +4760|Supplier#000004760|WUTG WXMH19FOMmWcZ|13|23-892-718-2155|9390.96|lent pinto beans sleep quickly slyly ironic requests. carefully express packages against the +4761|Supplier#000004761|vTujjUQ,ixLcwW31mN4,rrM0|4|14-916-946-2336|4018.28|ly against the quickly regular deposits. even pinto beans hag +4762|Supplier#000004762|FOYH3Br4rlwYBDSmdwf95sej6oGGPyuH8VTsDj1|1|11-235-666-1986|4300.50|oxes haggle express, regular dependencies. even excuses use carefully. carefully bold +4763|Supplier#000004763|W 7kS9LLh4ZgLpk2|21|31-335-256-6649|6550.83|g the even, regular deposits. even, silent packages use-- unusual ideas use along the asymptot +4764|Supplier#000004764|R12rv59MMOGz|9|19-122-667-8223|9875.10|its. carefully express packages wake blithely for +4765|Supplier#000004765|LCJ8Mtc6bGYDK7Kg5EKq1tudlQpkT54R8R8j|4|14-523-229-1984|895.30|g to the express ideas run quickly along the carefully even deposits. carefully daring deposit +4766|Supplier#000004766|CDZkZ2jocSawI0kS7mI6G3BcJHHduIw|11|21-921-709-8155|8180.08|e slyly across the furiously even deposits. furiously +4767|Supplier#000004767|mR0nDVqZvOIrb1i6DmxOLyTXWYyI M|2|12-834-224-9409|2682.01|fy requests against the quickly pending requests haggle alongside of the slyly ironic requests. slyl +4768|Supplier#000004768|FtgVKBvrUGC73zRo,28JZig9|7|17-237-505-5779|7517.91|olites. special, bold pinto beans are sl +4769|Supplier#000004769|Iom2IeCJ44BiWH|3|13-593-676-3451|5501.44|ess deposits use carefully across +4770|Supplier#000004770|zaW A8SdI3cq5jnxJsMk8|0|10-118-723-7583|5664.74|nst the theodolites. platelets along the carefully +4771|Supplier#000004771|iprePPIbXTe |18|28-994-196-7394|298.13| are carefully about the caref +4772|Supplier#000004772|IPyqnSZVJWuh|2|12-584-949-6906|3648.64|e stealthy deposits. blithely regular dependencies are deposits. always bold requests wake car +4773|Supplier#000004773|l bLnfPKIIlBWC8czckJ20HGUh0|1|11-927-176-6534|6669.90|unusual instructions promise slyly outside the carefully ironic ideas. regul +4774|Supplier#000004774|s3Q0WRzpHmSxgHAzXFD|5|15-839-101-6007|9052.45|ly. sometimes bold requests are among the final requ +4775|Supplier#000004775|sUP9OJdnypQogZeq|8|18-101-486-6872|1587.98|ainments unwind. deposits wake blithely pending ac +4776|Supplier#000004776|rqBhrPpxZFhTZQ8V3BjoCv8L14Wy67Jy2Y|8|18-924-758-6273|1215.77|y regular requests. special instructions us +4777|Supplier#000004777|SW3Z2kGqWSlYxqkpXJb3a9MkZFtrQ,h09|1|11-253-268-4805|4433.56|press requests. even acco +4778|Supplier#000004778|NFVxKobRQOHN B1O0U7U,|13|23-829-256-1950|9818.79|ve the slyly ironic requests nag pending +4779|Supplier#000004779|tX99QqdyiSGhHl tVjuXser,WRiH|15|25-225-279-3270|426.32|y ironic pinto beans wake about the accounts. fluffily express account +4780|Supplier#000004780|38KXwQ8CWIXIXy19 ,R0ENCJEj,yjiRTq9bbY|11|21-610-438-7867|2123.72|s cajole ideas. special accounts r +4781|Supplier#000004781|2yTiPVvJWav|0|10-862-832-4887|7347.18|instructions are furiously. quickly even instru +4782|Supplier#000004782|KcoOp,TPn3IDs2Bbc4JcdLkelSaWG vUv|16|26-453-885-3125|7172.96|regular packages nag among the silent accounts! i +4783|Supplier#000004783|s3Z36UOPsXtRBLQG AFKpOZLoLpDX|22|32-862-122-3079|5038.61|o beans must solve request +4784|Supplier#000004784|7SOr6Sl9gMh0b57BWLcv M|10|20-255-185-6528|3938.40|ular dolphins. slyly final instr +4785|Supplier#000004785|zKnM6Wq5if1ovdmAloFLN48t9 RtO,zCz |7|17-977-283-2115|3730.29|quickly pending platelets. regular, unusual foxes affix carefully after the packages. final +4786|Supplier#000004786|IUL1Wfe1hcInYn9nVpQNIfFCBnZ07UATQgx|20|30-425-720-9255|4029.60|. closely even deposits above the pending pinto beans det +4787|Supplier#000004787|8jhAajHOx17JMdlwo2Xk|24|34-207-969-7500|7462.08|ogged, pending platelets among the pending requests integrate silen +4788|Supplier#000004788|rmYoEMq8npBFGz9uM|5|15-641-694-2010|2272.06|sly sly, unusual deposits-- instructions affix f +4789|Supplier#000004789|yeZD2R4oIl|15|25-631-785-2788|3460.20|ly final requests impress quickly according to the furiously silent accounts. +4790|Supplier#000004790|hGIMs7QWh3p1PRq20onLaIKv|5|15-981-931-4002|7490.91|he grouches wake slyly deposits. caref +4791|Supplier#000004791|VclGsA7IcScLusE3|21|31-795-664-8589|2703.94|ests along the pinto beans haggle theodolites. ironic de +4792|Supplier#000004792|UUyeceVcL,KJaeb49OzQJwebcJA|5|15-286-331-7727|2021.86|y carefully final accounts? permanently regular instructions along the re +4793|Supplier#000004793|pe8Zl1MUxJumP89KKRGFCLJQyYluacyojBi|20|30-499-432-3073|338.53|ests. carefully regular instructions affix a +4794|Supplier#000004794|WqqTjtxiy3l7bJxoqJD wsfP2Gm14AYZC,2J41|9|19-808-447-7933|545.06|y special requests; regular, special excuses sleep after the special +4795|Supplier#000004795|f8Xcpzbl2lcX98nuvrFuBSxpoC7bdo81VyGWMXMr|19|29-606-558-3492|6691.78|snooze carefully. ruthless accounts play accordin +4796|Supplier#000004796|ZHE3YL7wX9nvhnKBG,M|21|31-923-800-8786|2802.51|es wake doggedly busy for +4797|Supplier#000004797|ILjbnoynNq,X03xB1cKuCqO|15|25-727-367-6267|8784.54|above the furiously special packages use along the carefully pending package +4798|Supplier#000004798|x1kpkFCop3hakg,yMhulv0Ad2|1|11-617-801-8409|8902.21|elets wake. blithely even foxes use regular, final deposits. unusual, special deposits kin +4799|Supplier#000004799|3vtM1bd2RzE6eY2LWK|18|28-505-332-9341|6435.95|arefully even foxes. slyly regular accounts boost slyly final idea +4800|Supplier#000004800|uy,dK eND3Ogaubkl,otcLPvdISomww9btn1s|0|10-172-117-6614|4102.98|r deposits are above the regular accounts. final packages detect. furiously even instruc +4801|Supplier#000004801|ijjpYDmHr,exEs1dzgra1YanXR6rYDEvxpqR|5|15-613-402-2047|8859.42|rate across the dolphins. ironic accounts haggle carefully. theodolites against the sly +4802|Supplier#000004802|wR6LjrKenZOk5w68CmmjZ6Ub|12|22-954-859-6358|2306.73|ular requests. permanent, unusual packages sleep among the slyly ironic ideas. blithely +4803|Supplier#000004803|w7bQ4PCqVCZjaAtb|1|11-856-290-9381|3193.58|ffily ironic excuses sleep above the quickly regular theodolites. fluffily pending packages sleep a +4804|Supplier#000004804|NdUhkuDe8ChG VxmXqQsZ|8|18-835-693-6240|3475.30|ng accounts cajole express deposits. blithely final p +4805|Supplier#000004805|2gKQFPhqb64TUxuNtLApozpqap2lHqpzz8KEny|24|34-670-308-2884|8301.88|furiously according to the thinly bold dependencies. pending instructions eat carefully slyly +4806|Supplier#000004806|Wa7ADCo2ZEhU9|13|23-314-796-1693|3821.72| are fluffily slyly unusual foxes. ironic, final dep +4807|Supplier#000004807|hag2uHLRxx0MDO1qSpmvS9 yP1Oo|15|25-429-747-5617|676.68|are along the quickly special dolphins. multipliers are after the fu +4808|Supplier#000004808|Y4dpiCheUoz8MjrLgTb cPQzGphr9hPoLo2i12|14|24-647-745-8200|7982.28|eep slyly final tithes. ideas among the final, bold instructions haggle into the quickly ironi +4809|Supplier#000004809|rPgJe4RN2n7YfRugQSnT|4|14-152-243-3535|189.10|sts. furiously silent ideas use furiously slyly slow +4810|Supplier#000004810|r58x9o24iqgvayYpqiLmD6fOybOh3Yn4jS|2|12-438-171-4318|8121.89|ing grouches haggle careful +4811|Supplier#000004811|8NCksH9rrGkkaSJKEpRnZvrafBJBWWxMjSx,|23|33-766-661-9013|3667.74|ctions boost regular foxes. requests ha +4812|Supplier#000004812|LaOjrGP8yH|16|26-234-576-6832|8728.06|onic foxes integrate carefully against the +4813|Supplier#000004813|bQCHDBDUfuWdiDxm,28qIs79Sgr23WBX1NqP|8|18-400-207-7577|5619.55|encies are furiously. quickly even requests boost across the furiously bold deposits. ironic +4814|Supplier#000004814|IUEBgcSJ5npstly5E7CMf,M7o2Ch|19|29-967-317-8001|-165.23|e blithely final, express ideas. blithely final instructions after the express ideas wake slyl +4815|Supplier#000004815|10APqmVAwAMIU3ikYtEZ79eVWT4VnRpVl|17|27-524-912-1189|3831.97|regular theodolites. ironic, regular frays integrate according to +4816|Supplier#000004816|eFcrfArvpKcH0dfdviLxT|13|23-353-314-4781|7829.60|ctions are fluffy, pending deposits. pla +4817|Supplier#000004817|UL ZJxywDNpE5UGHSQUsxTKMppg|24|34-537-776-9476|8096.08|ly final pains integrate blithely carefully ironic accounts. regul +4818|Supplier#000004818| pq 8W7GgiPQEHo4qMvREULChRuKqxSP9GQm2ox|4|14-584-467-9693|9793.28|ieve across the dolphins. asymptotes b +4819|Supplier#000004819|wMttOuIKQZVud4keSILt,OXr R|16|26-138-600-1708|756.35|deas. blithely express requests boost quickly slyly ironic frays. fluffily careful theod +4820|Supplier#000004820|Ja1smMw RnwhnJWsu|10|20-745-250-1724|4991.89|cajole blithely; ironic, pending forges boost accor +4821|Supplier#000004821|2JiH5BnPCNJwvERzU6ia8DLhp|6|16-809-287-1131|-274.84|kages. regular foxes slee +4822|Supplier#000004822|jco9lnR1Lh9ODQr0OaIyC|17|27-296-896-6951|9649.18|s. silent, final theodolites haggle. furiously ironic a +4823|Supplier#000004823|cOflLbPaDr6F9LhuG9TWQaRQmSfk mx|0|10-102-116-6785|9872.10|yly deposits. careful, regular requests mold slyly. pending deposits cajole blithe +4824|Supplier#000004824|cbKEO,yyzlkSf5D8P59V hUlEajLOPRcPg30|12|22-400-309-1559|5526.06|ckages sleep deposits. ironic excuses against the deposits haggle flu +4825|Supplier#000004825|CjrxyjFU9Tcgg5Lnr8KO o|24|34-121-940-1413|1547.80|old packages along the dogged pa +4826|Supplier#000004826|PnUsiXSouy0M8mqtBTnG4s|2|12-308-489-6465|7747.63| furiously regular packages haggle blithely of the busy instructions. quickly pendin +4827|Supplier#000004827|oapZgR0L1Md6IC2IZOO8UBjnuKY8u|14|24-390-759-3313|5487.47|egular, even dinos are furiously above the slyly express accounts. ironic packages haggle blithel +4828|Supplier#000004828|kk06JaDjYFDGjI90Bzbp4ScLEwad1MuSYCOBaKC|14|24-983-382-3288|1935.28|lyly ironic escapades sleep. bli +4829|Supplier#000004829|1s8 b2vOzYe1qA5HbPjyViDH1yYV9KS5N|20|30-376-754-9776|8250.84|usly final packages integrate fluffily theodolites. unusu +4830|Supplier#000004830|X,fXCVLsd,raIX Z0bT12nCk6UnfHfDm9UF5Sj7|7|17-393-421-1476|3863.49|usy requests. furiously bold accounts cajole furio +4831|Supplier#000004831|P7etrrqyCmgg5ZDxFIl7 8PanbiO0pJRCrxlGh,s|20|30-230-129-4785|9664.68| theodolites use fluffily unusual courts-- carefully unusual dependenci +4832|Supplier#000004832|M5CojlGp1 qu0ZGthJJDEzBU5vElXp|11|21-966-987-8142|6406.55| packages. fluffily ironic multipliers promise ironic deposits. +4833|Supplier#000004833|Q0YyGmPz4D194q5qIpQ,d3xNwUaf5NcyPiYuPWxp|5|15-737-550-3941|4863.81|ully express packages run. q +4834|Supplier#000004834|zNPeOQC39Vi3gmw2GchO70TmExXxcRIBo|1|11-459-399-3120|3111.60|press dependencies: slow accounts are fluffily. furiously regular pack +4835|Supplier#000004835|yAvaoHJQHGbWRhcBrGJAhyqf5PAq1jruZIo LzM|23|33-760-622-5299|629.43|ang slyly atop the instructions. slyly final accounts dazzle carefully above +4836|Supplier#000004836|cZygAtaTfslX,Adp7VVQhuDO0qYv9|3|13-897-948-4277|5744.30|ter the sometimes ironic requests. carefully special excuses use slyly sly +4837|Supplier#000004837|tYHMZS4XlJjzvj34mH2PCoj|21|31-506-485-5637|6385.26|e around the ironic, even pearls. final theodolites snooze regularly against the carefully silent +4838|Supplier#000004838|24L4lahMEi4Xe9nDFYMU|23|33-868-522-8983|-889.12|uriously close requests cajole carefully furiously special deposits. furiously fina +4839|Supplier#000004839|JxtU4NY,TdGJSAOt1,du4ujxFo0W,faXtwrWnt|23|33-450-585-1565|3550.25|oxes. regular ideas are carefully furiously ironic requests: blithely special deposits across t +4840|Supplier#000004840|VF,9SdjsKxRXQLMIGXC|20|30-743-742-4951|2979.77|requests. carefully bold instructions haggle blithely above the furio +4841|Supplier#000004841|5fI2WUviQ30OLcvqZ|1|11-147-598-6829|6909.01|accounts. quickly bold requests haggle blithely final ideas. theodolites along the blithely e +4842|Supplier#000004842|haSlvnOC8BYCvcy27P4,2pwg9CD9FdvN58TLq|2|12-471-474-9916|1790.52|gainst the theodolites. ironic, regular deposits shal +4843|Supplier#000004843|x,PnxiPitTdgtTkaF6aRVjD|17|27-509-992-9025|6259.86|heodolites. quickly unusual foxes cajole blithely. slyly final gifts wake furio +4844|Supplier#000004844|M0hof7wurIbD1C|15|25-626-755-3245|3466.38|ught to sleep carefully ironic ideas. fina +4845|Supplier#000004845|5c1Sb5a9pt|16|26-786-496-8626|1483.57|lphins. fluffily final accounts sleep even, regular grouches. express deposits boost blithely. exp +4846|Supplier#000004846|yaGqqDppYiGSkpUnHASEkrxb5xD9SkCpI2vad|24|34-701-113-2989|6380.83| express pinto beans-- re +4847|Supplier#000004847|qsHYyZLkySro7Wmc6tfZqN1ycgwq3XF1bA m|3|13-711-383-8713|1583.58|fully slyly ironic packages. pinto beans ab +4848|Supplier#000004848|SRJCKW382GEc|9|19-668-694-6038|4124.25|express packages. carefully regular accounts mold carefully according to the fluffil +4849|Supplier#000004849|7AnPSv0ispOYWXh|8|18-730-418-7910|3294.68|hely regular requests nag carefully according to the requests. carefully regular requests along the +4850|Supplier#000004850|TyfW3T dKVmzem4JDv7E6pY4KEg6m,mVU54wH|11|21-235-172-4822|8749.26|efully even deposits along the ironically ironic foxes cajole quickly slyly +4851|Supplier#000004851|Rj,x6IgLT7kBL99nqp|3|13-913-167-7389|9823.21| final deposits integrate slyly quickly ironic deposits. deposits hinder quickly +4852|Supplier#000004852|Uwdz2nnPMHBMxJeM9zCsiqo0mxjq|23|33-384-566-3301|8701.72|hely about the fluffily busy warhorses. requests cajole. quickly ironic excuses among the accou +4853|Supplier#000004853|2xAslDU1065Y,NMWe0U7zMaHaFI|8|18-794-159-5400|2874.47|ts cajole above the slyly pending pinto beans. regular deposits boost blithe +4854|Supplier#000004854|eEpp5kzlthqRD6KT, hTiW1bgShQB0aY|18|28-567-138-7891|8119.36|uffy, unusual packages. slyly ironic accounts boost furiously. quickly bol +4855|Supplier#000004855|m5F9lwXvarXG7by5pwfm5nm32Yy4AYeXRcihvXKJ|4|14-161-713-3636|5219.57|s. even, bold ideas are. even pinto beans sleep blithe +4856|Supplier#000004856|HYogcF3Jb yh1|19|29-334-870-9731|9359.61|y ironic theodolites. blithely sile +4857|Supplier#000004857|XGMR6l 7NqtVjxv|11|21-136-350-7911|5076.17|counts doze slyly fluffily special dolphins. platelets according to the sl +4858|Supplier#000004858|MzKGzMuomPZYRRxEz8tdjX5Nt5oqRdb|11|21-951-153-7767|8144.19|ns nag after the regular dependencie +4859|Supplier#000004859|d y1BEsgsuRaezea2Zt3VjBdO7aQ7T7 M|6|16-543-714-6701|4454.02|re furiously among the carefully pending foxes; permanently regular instructions sleep unusual, fin +4860|Supplier#000004860|nHZQopTKy77R|17|27-161-110-7978|1789.23|ts poach unusual, express packages-- water +4861|Supplier#000004861|antr,DZEW3|22|32-742-227-5393|6470.46|f the fluffily final ideas. slyly even instructions haggle furiously after the +4862|Supplier#000004862|Jw2TyiktrFJmTKg6EkTdR,MP13W|5|15-985-709-5896|1110.42|arefully even packages. blithely thin instructions among t +4863|Supplier#000004863|lK3WbF5Zi21qYJ3GYR,wj |1|11-202-391-2896|4706.05|the accounts. closely even requests detect even, bold deposits. regular, idle reques +4864|Supplier#000004864|gef98SlzXMpZZ k|24|34-559-157-7730|2548.54|ly regular dependencies play blithely above the even ideas. furiously bold instructions ca +4865|Supplier#000004865|VL4FFhw49fQDMYmPulRTJ|14|24-622-987-3854|8317.89|ar instructions was across +4866|Supplier#000004866|zxT7Ij4KcMeWgs7wAIwf2jhznXrpj|23|33-598-423-9515|8853.59|mptotes wake stealthily. ironic packages haggle slyly along the sl +4867|Supplier#000004867|fCUYoM0G Ctejr|2|12-621-495-5938|3378.68|special accounts against the carefully regul +4868|Supplier#000004868|7Rty18DFb24edC5m1 0W|0|10-284-701-9257|1491.03| instructions haggle according t +4869|Supplier#000004869|mzWtIp0Z1fN0n2eGwaEdBh67QxT|15|25-809-918-5052|-300.27|. slyly quick dependencies sleep furiou +4870|Supplier#000004870|ziZOGWWSFteUl7Bsm 6iBMXOVxC3twpaYwJo7Otg|16|26-527-733-6166|210.14|inal excuses should sleep blithely carefully final theodolites. regular, regular packages haggle- +4871|Supplier#000004871|,phpt6AWEnUS8t4Avb50rFfdg7O9c6nU8xxv8eC5|3|13-651-605-4159|9740.52|xes. blithely enticing requests engage carefully special asymptotes. c +4872|Supplier#000004872|rzX92i5RX1oSwuhTsF8 ZX34v|11|21-523-435-2797|4997.89|platelets wake ironically. blithely express frays detect carefully unusual deposits. deposits +4873|Supplier#000004873|kEBG2p5EM8XR4|20|30-357-280-8418|1599.46| haggle slyly stealthily even packages. instructions +4874|Supplier#000004874|B,ZfQ7ZICTQd|16|26-625-565-3242|7146.69|posits. quickly special packages affix blithely about the requests. ironic, final accounts use r +4875|Supplier#000004875|aKTNw5GBhjjiU2P|14|24-441-521-5755|3538.04|into beans. fluffily caref +4876|Supplier#000004876|LiNuGSdnExX QfQxNRZhqm8BhmzLqPmbf4vRVFI|2|12-492-478-1774|6835.93|boost slyly according to the furiously regular deposits. furiously special inst +4877|Supplier#000004877|9dhCRXvfn2P38CAgjuQ5rnfDtXHer|8|18-778-813-5892|5453.81| excuses. bold packages snooze blithely past the even requests. ironic, ir +4878|Supplier#000004878|FXT54HZZIBYw,QQ2r VQylR|2|12-520-306-8674|8268.99|e slyly slyly express dolphins; quickly regular instructions nag sly +4879|Supplier#000004879|93OCYCGYeuw96DG|15|25-870-407-6748|957.20|onic deposits detect blithely acros +4880|Supplier#000004880|TJ1OINeXbOcidR4Y|4|14-481-958-6688|3636.94| after the special, bold requests cajole slyly furiously +4881|Supplier#000004881|cq9O84kVGKZ4FuPauZWuZcuio1psUq|1|11-892-463-7553|8004.56|ins are furiously bold accounts. carefully iron +4882|Supplier#000004882|e,V Bo1KZEt|21|31-537-780-2105|640.06|uiet accounts after the ironic, pending theodolites sleep carefully ironic packages. ironic pac +4883|Supplier#000004883|58HcU6bYnOMVG1oz4i|17|27-518-661-5444|8446.51|tealthily according to the ev +4884|Supplier#000004884|42Z1uLye9nsn6aTGBNd dI8 x|3|13-314-750-4670|3608.62|unusual, final dolphins across the silent ideas should affix slowly above the blithe +4885|Supplier#000004885|e,bwHyd6HTKcbj4ScUx7GbuPloK7y6|0|10-837-773-1530|5138.82|d, unusual asymptotes. express, even deposits after the f +4886|Supplier#000004886|2H7d0mZUtQwFwz0|4|14-234-740-4050|5081.35|. quickly regular packages detect above t +4887|Supplier#000004887|,TpTOfc1nx,PLUztfC5mpK74sd1GcZfU8QM|7|17-114-984-7958|7247.36| ideas. final, pending packages wake. quickly final requests cajole. theodolites are above the re +4888|Supplier#000004888|7gvcHmNXy35L49|20|30-936-409-4815|1427.20|regular requests nag alongside of the slyly ironic +4889|Supplier#000004889|JVSA0E0p,p11R0mbk1ESg0moQZjrVbWjnqsYxKv|21|31-809-381-8399|4931.12|eep furiously furiously even requests. final foxes are fluffily even packages. carefully express +4890|Supplier#000004890|RctxkJvmWtQvwgddK46|2|12-408-322-9002|1263.90| slyly against the carefully bol +4891|Supplier#000004891|c4DxOrexw3nogQgYrEwhwdDxk1NI6bOPAwgmo8I|12|22-547-938-9375|-624.70|refully ironic packages ought to detect ironic dugouts. packages boost carefully. f +4892|Supplier#000004892|QSwl0pUwGhVMWbV8CH,jj8jsX,FAzGapT|1|11-130-125-8633|3584.41|above the slyly final pinto beans. express, special excuses haggle acros +4893|Supplier#000004893|WuZVpy6CJeEAyuNXi8dNh|0|10-102-827-4527|5158.38|ual, special requests are. blithely final foxes nag +4894|Supplier#000004894|cGDsr1eyYXpyeA3,jAc|17|27-762-154-4286|3883.32|furiously pending packages. regular theodolites cajole. carefully silent foxes do are among +4895|Supplier#000004895|TbWNU6fEkHj8VoxvOl9kDP,XwoMhs|24|34-873-271-6462|1878.94|fluffily regular packages was accounts. busily regular +4896|Supplier#000004896|MCIT7sZxkZl hB1u3nfFEefKCe,|11|21-647-261-2214|7360.02|egular deposits detect according to the iron +4897|Supplier#000004897|h,nzVUV Lvj9yKdb|7|17-738-542-1481|2920.34|accounts. blithely special Tiresias cajole blithely above the bl +4898|Supplier#000004898|gevICDmkyqeiZe|12|22-981-208-9816|9203.00|enly pending deposits about the requests believe ab +4899|Supplier#000004899|D4b3yDzDgK2WyaUdXWo0r|16|26-897-272-3576|1819.50|tect furiously along the fluffily final dependencies. slyly final ideas sleep sly +4900|Supplier#000004900|CSG,mmblpJ,mj8Z |17|27-141-214-3558|9246.89|quests wake quickly. carefully final requests serve furiously furiously final deposits. unusual pint +4901|Supplier#000004901|zjsuu 7grcQ|10|20-700-229-7679|2485.12| theodolites haggle slyly. regular pinto beans abo +4902|Supplier#000004902|YdQIzRgwNou4BRRect7We|0|10-727-814-9489|9356.77|nto beans. regular accoun +4903|Supplier#000004903|5jqh0004hsIRqM4CBf3ej|19|29-753-806-8134|1317.02| the furiously ironic requests. ironic, regular plate +4904|Supplier#000004904|hEmIpwtGll,Be|16|26-726-245-5585|1101.17|ronic requests. ironic as +4905|Supplier#000004905|4Uzp PgtwP7LE |8|18-785-170-2196|3383.39| sentiments sleep permanent instructions. slyly fin +4906|Supplier#000004906|bBg,GFEnvMyOUW UF89KaXR38lOIg|17|27-342-773-6258|3660.48|ins must have to nag above the furiously final theodolites. bravely bold accounts ar +4907|Supplier#000004907|77LkGSkqBmivob16KXbkuOKVdy |22|32-328-528-6335|7712.45|y ironic ideas. furiously idle dinos use against the carefully pending waters. packages maintain a +4908|Supplier#000004908|VPCUfG8Q9AQpQwTEqfDRuJ|2|12-879-627-9955|8147.23|ests. theodolites across the blithely pending platelets boost carefully after the fluffily regul +4909|Supplier#000004909|wouHOhyOu CyDKlNll,fGn7Y0|1|11-793-193-3798|556.83|y furiously pending deposits. furiously express ideas sleep according to +4910|Supplier#000004910|J0PPonsA6l14UiPj7b7dcBCSmD1q,C8Zv|22|32-735-545-1531|6589.93|ke blithely behind the boldly ironic requests. packages about the blithely ruthless excuses +4911|Supplier#000004911|swyExOqO zMc1Wly1ZSYn341QvRdfFg|4|14-867-277-4507|7047.29|requests are furiously. furiously +4912|Supplier#000004912|2NUOQlm1bhr1bf|11|21-127-612-6452|4876.37|efully blithely regular asymptotes. +4913|Supplier#000004913|em,yC41xEl Fst9LwEik|21|31-327-123-9150|5078.14|eposits. furiously express courts about the final deposits cajole about the final, final requests. c +4914|Supplier#000004914|M8AqpqEtejHbxbWD22SdxKY|5|15-146-186-5016|8225.93|egular dolphins sleep furiously above the slyly bol +4915|Supplier#000004915|3HwKwI7eMsaN7O1ct01o, IuLgU8jh|2|12-708-598-9707|3291.46|. foxes doze according to the silent accoun +4916|Supplier#000004916|JIWnmlNv2CBQo|15|25-478-250-8523|9506.32|efully final packages use against the slyly even theodolites. slyly regular excuses a +4917|Supplier#000004917|Rhe84NYGtbpBE4xv|17|27-143-767-1631|6773.45|sual pinto beans integrate carefully along the slyly unusual platelets. regular +4918|Supplier#000004918|mszqavCBh31Z6P|16|26-502-495-1842|379.50|s. theodolites detect care +4919|Supplier#000004919|xegp3uXfPs7yf7tph30pfHarsSazkylMQv2|5|15-433-902-1697|7209.33|the thinly regular multipliers. even, regular pinto beans nag qui +4920|Supplier#000004920|mhaiy9Dd7fcKBb|19|29-498-816-9247|212.45|lyly silent asymptotes nag slyly befo +4921|Supplier#000004921|UTT9MiyzVrXqZ9JYtjCYiUuEstt|11|21-400-799-2328|4642.64|of the furiously regular deposits print slyly carefully final foxes. ironic, regular d +4922|Supplier#000004922|mX0vXdJBLP|18|28-795-815-4757|3909.91|usual ideas may sleep slyly along the regular packages. carefully even accounts along the slyly i +4923|Supplier#000004923|u VJeVpWm2UfeajsyGzxCDvZw|24|34-911-585-4859|8927.41|to beans haggle. slyly regular theodolites integrate fluffily final +4924|Supplier#000004924|GzFHtyCVrX77xZ6iagEb hVi|15|25-244-819-9311|1049.00|inal theodolites: carefull +4925|Supplier#000004925|w3V3pkQE2hSRISfu,tGLS,5hCue2ZUWrAVh714|20|30-136-996-2100|6408.38|beans are furiously about the fluffily final deposits. unusual, +4926|Supplier#000004926|wxNks1eTvneiA PN,gWNF8|17|27-641-856-4007|2658.57|al dependencies use blithely ironic requests. quickly even packages +4927|Supplier#000004927|h5WAYrJzZMHnOnz5EgO,EvjnJPzp0FhX|9|19-636-113-7546|9634.26|refully. express deposits detect around the blithely ironic dependencies. r +4928|Supplier#000004928|IYpwXYEo2yIkLowAPbV41 oR|19|29-801-741-2301|1137.30|leep. quickly silent excuses cajole among th +4929|Supplier#000004929|dzUOBgSae7hmM6qDXc9D9T1C8zeVt139|5|15-165-696-2848|5598.98|osits integrate. furiously regular platelets sleep slyly +4930|Supplier#000004930|w0kSsrbeC1uvnXyo06Yqlf|0|10-290-814-6347|6076.07|urts. carefully express deposits cajole slyly again +4931|Supplier#000004931|4VTbOamndSM6U|14|24-625-784-4507|5455.53| regular deposits are slyly above the u +4932|Supplier#000004932|qf2dSPyr70l,eivotluIDzklU|15|25-654-894-7509|4071.63|efully pending foxes. regular asymptotes sleep c +4933|Supplier#000004933|yCGb3EhmGL 6NB7Xw|16|26-701-611-1882|9851.27|iously silent deposits are express deposits. slyly final deposits wake bold, daring depos +4934|Supplier#000004934|DmR5NR5K4ukyBFIi9VIe In0VmUWiS|4|14-337-240-6607|7366.10| requests detect blithely. fluffily regular deposits cajole carefully +4935|Supplier#000004935|eVGOcd62gpsLEm1f5G|16|26-586-543-3025|6054.66|g above the quickly regular gifts. even pea +4936|Supplier#000004936|CXvDlzbytZTd4WY Algy|9|19-801-580-8992|8758.25|dolphins boost furiously. ironic dolphins sleep furiously blithely unusual dinos. b +4937|Supplier#000004937|C7r0GICFhXvfuf3gpNekFr1kSut|2|12-254-805-9972|6919.88|ly even deposits. regular r +4938|Supplier#000004938|PAj7WnIHEW7FWr3|6|16-970-279-6216|8393.39|refully. unusual decoys are. fluffily regular deposits sleep furiously according to t +4939|Supplier#000004939|2wMwDKUfCR5f01YXzS|18|28-311-846-6124|1785.30|osits are. packages wake caref +4940|Supplier#000004940|UcTbBNn0kUtVyIPeWxG cu3qznz|2|12-983-845-9545|6431.29|furiously above the regular pinto beans. furiously blithe requests wake slyly bol +4941|Supplier#000004941|1,a6rBgCJMUdR78OYQJ4vyQtEGcNmqGvqolMYMl|22|32-769-160-7618|2781.78| wake along the special courts. qui +4942|Supplier#000004942|Q0Ao5KyTvUknEUzf8BtSD9X7Yvx35LWKbw,XIl|2|12-618-495-1420|1306.79|efully even pinto beans nod furiously fluffily express instr +4943|Supplier#000004943|x8duiqMOaVAjMxHCMbroi dMau2Kx1PgI72K8o,|0|10-869-616-1546|5141.27|yers. quickly special platelets boost blithely regular, silent asympt +4944|Supplier#000004944|,eN75B9Wo,VoklFVVnt4|11|21-660-312-1204|2254.53| wake carefully regular packages. special, final +4945|Supplier#000004945|yI71,qR3g,1x2zi4IDEKBgf0vuCdCSN|11|21-822-740-2027|5794.75|accounts sleep. accounts cajole. special asymptotes at the unusual acc +4946|Supplier#000004946|godIJZFokyU2WqnvON7vse|13|23-186-281-9873|7307.54|e pending platelets. final, regular theodolites impress c +4947|Supplier#000004947|dd0B6ar90BtBSFLfFiK|22|32-630-720-3585|928.70| final requests. closely brave packages run +4948|Supplier#000004948|371vWhovseINrPrUUsJBUOADLVqrxOLrRCf0S|8|18-726-982-1206|8531.19|ly pending packages wake furiously slyly final packages. slyly pending gifts dazz +4949|Supplier#000004949|Rcqi8k3vuqVDrHWebhehhBl0VMFMCJV2j f|7|17-991-530-7017|7774.62|nding deposits. final, ironic requests wake. furiously bold pin +4950|Supplier#000004950|yEabKdYvptXp0nj,,pGZoOKTT2lshZHX|6|16-940-472-8147|8171.59|en, express accounts. pending, even foxes across the carefully even deposits wake thinly +4951|Supplier#000004951|Q2a CbVG988HSECXRzv QaI1swqQX3|2|12-750-593-9417|2990.23|ckly ironic deposits. ruthlessly regular requests along the ir +4952|Supplier#000004952| Cg Ml158ET2vu8eeFFY|14|24-874-665-6423|6756.14|nal accounts according to the +4953|Supplier#000004953|mA7pbv8umMzz3342ZCqZGTgR1I gc|8|18-145-690-8991|6502.61|dolites nag across the furiously ironic packages. boldly final ideas sleep quickly +4954|Supplier#000004954|qE1uqzqt23hw VVD2j8G Gl0gvoxY8G0x|4|14-288-546-6291|2443.84|y about the ironic accounts. brave, ironic deposits dazzl +4955|Supplier#000004955|7vOJPagJiKxciLtSCjVNnZ5SIP V1vLcmYK3IGg|16|26-258-559-3104|8594.76|ructions mold blithely among the bold, final packages. +4956|Supplier#000004956|ibDuLQ0D8W1hK3m 7sMKFt5fw8XGzp1aU|24|34-954-986-5576|9131.89|arefully express dolphins kindle against the carefully final accounts; sentiments +4957|Supplier#000004957|jowpMl4a6x6 OYnyhaLCG2,p|9|19-426-439-1855|5712.00|r the doggedly final requests wake above the +4958|Supplier#000004958|TFktpL54vzL51GNV cGZETKO16k,|13|23-871-483-8310|2660.00|ily. quickly unusual accounts use carefully +4959|Supplier#000004959|GcAqEAZvu35CBvZFbukttW52|9|19-768-728-3181|564.12| express foxes sleep. final, regular theodolites believe. final acco +4960|Supplier#000004960|4 965jqpUhHh5|22|32-162-425-4817|8157.56|carefully regular ideas detect regula +4961|Supplier#000004961|HTHQsovnuqEaKFswci8rS|3|13-441-823-4698|9142.88|ly unusual platelets cajole carefully silent instructions. careful, final request +4962|Supplier#000004962|TsvuPcrllO2fkIC5xPsI02qzGLw09vRpnA|12|22-233-662-1715|3508.89|ornis sublate furiously according to the bold, regular pa +4963|Supplier#000004963|F5naCEAdQhhjm3IwJsla7 OF6mMbbjtH90|14|24-756-311-4817|831.27|equests are slyly. quickly express accounts are furiously. regular asymptotes are alo +4964|Supplier#000004964|OMsqenynW3u7XxruccNIC|0|10-669-638-8050|-852.96|s x-ray. regular instructions wake after the slyly +4965|Supplier#000004965|2XSH3dNMlYuJR4y|24|34-234-789-3541|6140.50|l deposits wake thin, silent +4966|Supplier#000004966|URwLiWlI42VUl|18|28-330-305-4993|-627.65|posits nag blithely final, ironic excuses. quick, s +4967|Supplier#000004967|lIvbn4iSOb4F VwG0xLQU3QHn4|6|16-766-968-2266|8990.07|uickly furiously final dependencies. express dolphins nod carefully against the +4968|Supplier#000004968|lAr865SA3gHvxbvvo|6|16-140-143-3739|9214.02|foxes integrate slyly along the slyly unusual deposits? busy deposits along the re +4969|Supplier#000004969|HdteuyoPpBVg|11|21-770-588-6089|187.50|ptotes. carefully regular requests since the slyly furious accounts are above the blithely ironic in +4970|Supplier#000004970|wytxC,bck4YitciVKox5|0|10-368-762-7098|3256.72|onic warthogs nag among the regular, ironic deposits? furiously pendin +4971|Supplier#000004971|qrJ e9wrseQQjVb83pLYiA32KSO|24|34-979-539-2048|8610.81|lar ideas. express ideas at the regular deposits cajole qui +4972|Supplier#000004972|EkqIooaxrYnvGx54pRzfM2Hwf5G6eh5epVemu|16|26-598-986-6370|3874.44|. blithely final packages wake furi +4973|Supplier#000004973|9oN52bxtLYP7Ln1V fu4W|13|23-801-973-1550|6702.85|usly pending instructions ar +4974|Supplier#000004974|u9v3n,dHDk8,8P9JuSzrZOwZCwIJqhXeIw|15|25-290-158-5290|1910.03|ld packages. requests about the blithely +4975|Supplier#000004975|GPq5PMKY6Wy|3|13-750-119-3295|1799.98|n theodolites use across the slyly unusual accounts. fluffily regular warthogs nag slyly along the +4976|Supplier#000004976|rh2cBdhFDyoI8PzP,2rgNb0LJ|22|32-854-553-8711|174.80|ns after the theodolites haggle +4977|Supplier#000004977|VYeEHmvwp0D43z|0|10-487-644-5662|9804.58|kly bold accounts affix carefully across the +4978|Supplier#000004978|Ho8kbrr7BjnPLDZWY3H2bjT51EHL3KfwqE4o|11|21-713-375-7705|3665.09|deposits. even instructions hag +4979|Supplier#000004979|OIpdjUc6s Ugwwed3NTU5W vJ9VlpBFpk892P|5|15-528-952-8885|8121.79|cial theodolites. furious +4980|Supplier#000004980| 02Q4zPvy2fluiMZSpr|12|22-144-143-7299|1969.10|ding to the theodolites. carefully regular deposits are blithely even courts. ideas according to th +4981|Supplier#000004981|w,hpC2VU3wpuO|7|17-698-852-7688|4634.99|eat furiously around the caref +4982|Supplier#000004982|LCFeh0bEqy1I4N2Jw5M|8|18-418-939-3812|7284.82|posits wake quickly slyly regular deposits. slyly steal +4983|Supplier#000004983|Q1D7GuqSwvlR4TKJzm1KdyYacBpvAi,HToOyVHic|12|22-819-215-5459|3951.48|nic instructions. slyly regular deposits sleep carefully. slyly pending dugouts +4984|Supplier#000004984|wLesh,3xCc,GrrtbzT3f0RoS4H0btIuUuR7624HP|5|15-773-360-6268|3497.99|ages. bold foxes wake carefully special hockey players. carefully special foxes det +4985|Supplier#000004985|QoMEe4xIDWcl390k,J99nELirJbX r1fB|5|15-148-929-1980|4708.84|ular deposits eat slyly to the bold, bold accounts. fluf +4986|Supplier#000004986|7O0B32PErOsvu|2|12-191-217-3733|5485.53|furiously final, regular packages. furiously special packages above the blithely pending accounts +4987|Supplier#000004987| UFx1upJ8MvOvgFjA8|3|13-936-786-7730|2316.27|ly. packages are carefully across the slyly regular accoun +4988|Supplier#000004988|jhzhG4Fi5e2qyBfXvwlhyyv9CsoyUIDTXOwpgUUF|14|24-746-787-2183|9585.13|less forges kindle. daringly pending dependencies run slyly above t +4989|Supplier#000004989|1BWGQtWWJzH2UWV8zJ06Vi80Ebes2xUCa18AOI|22|32-469-559-6242|311.07|onic Tiresias. final deposits boost according to the carefully express courts. slyly final forge +4990|Supplier#000004990|MgrvPgsfLD4Z2CkrORXppETrXWsYT|18|28-360-531-4238|1050.76|ole about the unusual theodolites. requests wake slyly ironic, final multipliers. carefully even f +4991|Supplier#000004991|9YbG9KjxgHRK2rB9mvJ7KuGE1K8ipDWVL|16|26-968-389-7594|3000.38|refully ironic packages hang. deposits doze furiously caref +4992|Supplier#000004992|zCGf7Bwq7C|18|28-665-105-3278|1322.43|ess pinto beans nag blithely pending excuses. sl +4993|Supplier#000004993|cOg,bCELki4zYO9h01DHD8Hv0OV8,1z 4,|10|20-685-878-9636|3960.62|ckages sleep blithely. slyly regular deposits are. slyly regular accounts h +4994|Supplier#000004994|K4LTyWmFoA3BBrKRWgPUWhhbcENSBzsK|20|30-729-121-6122|3181.06|ven instructions are alongside of the blithely pending courts. e +4995|Supplier#000004995|dlNt sgtcaHV5ndDdO|11|21-463-889-1360|2087.97|ges? final deposits haggle. carefully final request +4996|Supplier#000004996|YcFquiyJ0GCREXb,g2jAkmSSEwI1DdS3gpafKx8|10|20-610-448-4123|554.01|rding to the even packages wake carefully across +4997|Supplier#000004997|LeXYzsy2kv1 wB7CbeM|13|23-357-611-7051|3113.25|hely slyly express asymptotes. bold, regular pinto beans at the slyly final accounts haggle regular +4998|Supplier#000004998|Ijik8UoLj78Ek8lWrV X|4|14-298-930-6002|6334.58|d requests cajole around the pi +4999|Supplier#000004999|MZ GHfeuXCvgE|18|28-283-416-3669|5357.42|oost never. excuses among the blithely final requests wake +5000|Supplier#000005000|DeX804 w0H8FrCUvahgy ilbuzBX3NK|3|13-186-716-3689|5091.30|ake slyly quickly bold asymptotes. carefully bold requests are. final, even ideas use furiously +5001|Supplier#000005001|s iUGQbuWhvELOocTqH0sln6hRbYpu,cLiNioxLq|11|21-997-915-5565|9570.28|ages do use according to the request +5002|Supplier#000005002|jJqsKkqMIf8nOaY5LYQvNGpGkj|17|27-263-154-3728|3784.27|phins according to the slyly express pinto beans wake alongside o +5003|Supplier#000005003|dv34hDJQK,S9C6HIxdkX AbHvFe|14|24-644-394-1098|9244.34| the regular accounts. excuses are fluffily. regular asympt +5004|Supplier#000005004|qqvcCZJed1FncoUJ5mo0dIOgU7zzVHP|20|30-428-595-9706|3721.34|iously regular packages haggle slyly carefully final ideas. furiously e +5005|Supplier#000005005|K6 GI4WzmbsGEOh|21|31-985-368-1464|7595.48|ess accounts sleep quietly. express, unusual requests boos +5006|Supplier#000005006|Xvpl9rlgriF4B|1|11-843-342-5499|233.41|pendencies. regular reques +5007|Supplier#000005007|x5WnKISytOqIfLjMDpIyNvvC09|12|22-587-787-1054|4885.38|ong the pending accounts. slyly even packages are quickly along t +5008|Supplier#000005008|RxAVQWjolHk,kzy|18|28-568-926-6014|2633.84|deposits hinder above the quickly regular pinto beans. fluffily iron +5009|Supplier#000005009|rfogc0eRdxzvjpWAj,dNi|19|29-469-419-9952|871.96|ending accounts haggle furiously slyly regular pinto be +5010|Supplier#000005010|cmYBD2RaTe9vKC0uNoC4aCEu3DOA1B|22|32-840-426-3024|965.65|ake never. furiously unusual packages sleep ironic dolphins. +5011|Supplier#000005011|Ma4xYu1AdRVlyQCuYzdsS7yuIxmYiILdzId|1|11-737-696-2345|2565.79|gular requests haggle slyly against the accounts. slyly ironic dependencies sl +5012|Supplier#000005012|H2oITSxE4kqoUQgOKPKUYEM4ZWaZboiV |0|10-978-634-8748|1199.67|olites cajole blithely against the accounts. carefully busy dependencies integrate quickly dog +5013|Supplier#000005013|pA5sgFBsZ,zvG9DqKDX5LtSJRXhzCddXUxdpiKR2|17|27-950-281-5960|8720.86|d courts cajole fluffily above the slyly final ideas? blithely regula +5014|Supplier#000005014|d IZ2KkXKyE6|1|11-572-644-1147|5535.26|ts. slyly even requests nag furiously according to the ironic packages. furiously ironic depo +5015|Supplier#000005015|HOXohc8D8KJ Yx2WIOL 5EvtdnTqwXSo|3|13-358-401-5005|2096.16| furiously special packages boost +5016|Supplier#000005016|Pm76cd7RQpEoslbXa9hH6Fm,9z|5|15-955-178-1560|6076.16|al platelets after the accounts breach furiously according to the furiously bold e +5017|Supplier#000005017|RMaCRF8C,C3|8|18-252-308-7859|7035.97|nal dolphins. fluffily ironic accounts eat final requests. unusual packages are carefu +5018|Supplier#000005018|VF3jw0xQDHivHYnRt9JajvTSsxJxjnD8YY8|6|16-544-227-4448|1549.49|ests integrate slyly pending deposits. slyly dogged platelets above the blithely ironic pinto beans +5019|Supplier#000005019|4BJhH,Wczz|11|21-301-472-5208|3278.83|ual accounts print carefully silent orbits. patterns cajole carefully +5020|Supplier#000005020|8n8le,fK6fN1byNziZYGXTQLv8fTnjlyaM|22|32-966-740-2878|6256.44|ay bold, ironic theodolites. carefully regular deposits sleep always ironic ideas. regular reques +5021|Supplier#000005021|qtiNfDL dv7lsLA|0|10-185-959-4386|1005.06|ar platelets. pending accounts doze furiously ironic ideas. quickly regular +5022|Supplier#000005022|LtH4qtqST6dY,v,Bs|0|10-234-680-5990|4938.42|s nag carefully. fluffily express request +5023|Supplier#000005023|Q755r z94jtz|13|23-820-227-6379|3015.59|efully after the pending requests. always express requests are carefully. ironic excuses w +5024|Supplier#000005024|y0,QhJZwQSjhRC6 rPsa5tLcmjRy8LKjuS|19|29-535-675-8153|3976.59|y ironic deposits wake blithely above the unusual courts. +5025|Supplier#000005025|2eZ2yMMlxVDK,t9w1Ne y1CNAY|9|19-264-348-2707|7542.89|ourts against the carefully silent excuses boost blithely regular ins +5026|Supplier#000005026| QCWzn7yqZuyTPEv3xGZc,|17|27-492-930-4855|7445.31| blithely unusual deposits wake after the fluffily pending theodolites. +5027|Supplier#000005027|gnMiW6xgxZDGyTfGNHr9Y6Gf9Rc|0|10-713-481-8264|6424.30|s. theodolites along the slyly bold requests haggle final accounts. quickly regular deposits detect +5028|Supplier#000005028|CrmUuznEMBAPABSlfim|9|19-618-700-3717|6210.13|lar depths nag blithely foxes +5029|Supplier#000005029|,LffUev6QiSTJVRUwlKUB8JVYBGgE7fh |8|18-314-822-7673|3621.33|ithely final excuses. furiously stealthy foxes alongs +5030|Supplier#000005030|uVh0nT4ewaowwgjRyk8kCQ6blCU LnVqhFlvu|2|12-966-699-5374|9677.38|foxes believe among the unusual deposits. carefully unusual +5031|Supplier#000005031|8Ik mASiUsfaZ9aoSIy0sYBVs6Xedzid4eDc1|4|14-974-110-2768|-269.59|lyly regular deposits use blithely along the requests. carefully regular th +5032|Supplier#000005032|jIz N2Ybtuqj7J28jS,r1gZiw|18|28-856-977-3171|9905.18|g to the fluffily regular sentiments. slyly ironic foxes wake finally +5033|Supplier#000005033|X0TPY78Zx8uP12hWhTGGiOE8gN|22|32-513-404-7169|493.02|regularly carefully special pinto beans. silent accounts wake furiously? slyly pendin +5034|Supplier#000005034|lNTnyJDA1KVxA2aEnEdItEZFqOJPEcsB|23|33-165-557-5433|1087.52|ronic, regular instructions. bold, pend +5035|Supplier#000005035|6tCE9az7TfoPv6qAUKdw2vRtVTVefQLJ4nO1yer|11|21-165-300-2587|4964.31|ideas are carefully furiously silent theodolites. furiously bold ideas sleep ironically +5036|Supplier#000005036|Ftq,HCpxZ0|3|13-697-444-7990|9159.35| blithely. quietly final accounts +5037|Supplier#000005037|1nUPgyz73ORn1QSal|6|16-463-590-8725|5272.94|ly blithely bold asymptotes. foxes haggle qui +5038|Supplier#000005038|279zLyZkIs17Jv3vrN5,K dZ8tZAu9xZ|24|34-902-493-9828|4264.13|ironic foxes sleep blithely regular +5039|Supplier#000005039|tGYc0O9spcV2dLHSAQMBNqDLEzfe6yHKh|4|14-924-948-3112|267.92|r requests against the slyly special accounts nag bravely ironic, special pack +5040|Supplier#000005040|7OncAZTGYjs7XgHAx|0|10-223-907-3269|1928.71|rmanently stealthy, regular packages-- even, slow packages across the blithely ironi +5041|Supplier#000005041|mIu4vSDkshIwNK eeC|15|25-539-875-6090|7844.29|ounts are alongside of the regular asym +5042|Supplier#000005042|XIdas2upoZVrt8UYa9FsGgq|20|30-238-387-1758|2075.38|even multipliers wake on the express asymptotes. slyly regular accounts cajole finally. do +5043|Supplier#000005043|159m5Zd0FUN7kv58w3tyhoD2|4|14-895-422-1473|8918.11|ents against the fluffily regular foxes cajole +5044|Supplier#000005044|QQBecN3ZUfPsho0Slx|14|24-495-986-2266|9747.40|ep. foxes about the ironically regular deposits hinder along the carefully express ideas. sile +5045|Supplier#000005045|mUmOiHzG1IBdBlkipTAOeEPXwyoM 4Pev |2|12-115-703-5920|-357.75| brave ideas nag special instructions. blit +5046|Supplier#000005046|O7YXzzD9nsWNOy3D Yf|23|33-265-209-3222|2183.82|usly. carefully special accounts use deposits. pearls are fluffily blithely even accounts. furious +5047|Supplier#000005047|JsMY4WVF4VwclzwHz2juoKB4qxRvpKLJyW|19|29-350-386-7303|5510.75|ly throughout the furiously special asymptotes. mult +5048|Supplier#000005048|CS5vRYbcZu7UjNt5nU57jIpqtDh5l0Ls|12|22-997-560-9511|8256.62| about the slyly special excuses maintain fluffily slyly re +5049|Supplier#000005049|22J7yDqk9pnGoguOvx0x|11|21-500-137-4477|2063.91|ake. even, ironic pinto beans cajole furiously under the unusual packages. enticin +5050|Supplier#000005050|ibgWwWw,kD6NrOwn2Y4zoCOTEhyDV|14|24-498-493-3159|3036.00|uickly regular Customer afRecommendsu +5051|Supplier#000005051|Puejq3pV,JFX4,hUnhHbr|22|32-592-547-7141|6044.11|ts. slyly pending instructions boost furiously. slyl +5052|Supplier#000005052|XgiszGZq0pdZZHYe31dLQIMe|5|15-753-514-6917|7335.50|sly final excuses could have to are busily final packages. regular, express +5053|Supplier#000005053|FFVlqmJmxMqN|15|25-812-264-5461|8757.19|ly regular frays cajole carefully fina +5054|Supplier#000005054|KJ34,rOzK2NUiznv|1|11-250-265-4737|234.75|nag slyly final accounts. even excuses affix. slyly regula +5055|Supplier#000005055|Zykd1Z6Eehywxds0Pyqz4J2Hf1RaZR6|21|31-804-750-3159|5295.06|s sleep among the regular packages. deposits doze ironic, ironic accounts. accounts +5056|Supplier#000005056|jNR,eLOeczR3Q4xuq3aW3K|16|26-945-772-6739|2017.21|packages sleep slyly. fluffily pending accounts nod blithely-- deposi +5057|Supplier#000005057|qeAzRuVE4V8gUqsUrTLq36xPqgJSulykhWtx|8|18-724-442-1817|-850.71|g foxes according to the blithely unusual ins +5058|Supplier#000005058|LodnUlksXYBiuLbGaxyBZITONKpKOnUKb|17|27-397-133-8257|114.14|d blithe packages. carefully even deposits cajole carefully. final deposits affix furiously +5059|Supplier#000005059|yFLcezwK,TZTblG5PqA8nrw qfD|1|11-693-374-7519|2298.87|close packages. ironic, fluffy +5060|Supplier#000005060|egjgTmU5L0FZjsl,HC2Z5dk8yurOM|23|33-201-460-2475|3080.63|tions sleep platelets. blithely bold dolphins sleep steal +5061|Supplier#000005061|K7bbuYZol8GfTjEnOonzZukNi2ffx|20|30-257-188-5259|8157.92|d pinto beans. slyly even asymptotes cajole across the instructions. final forges after the +5062|Supplier#000005062|lWEM7ahnU0lkmdob5OvpqUfvITPG|2|12-577-644-6360|4216.63|deposits above the furiously +5063|Supplier#000005063| obF0Hgrj4moUr9u9BUXDrZgUhk6WoUDLS7Bt064|24|34-745-436-1269|4856.29|ions. carefully ironic waters nag across the special tithes. bold requests sleep furiously unusual +5064|Supplier#000005064|hcWgbbogD9a1V FaY402Bakl|21|31-819-896-8815|8653.21|grate carefully. quietly ironic accounts x-ray f +5065|Supplier#000005065|gv4VGfRCLuElgLNPByiuy|8|18-371-476-3271|1627.06|even, even accounts use. regular foxes boost. dugouts affi +5066|Supplier#000005066|ru iPsHLDTFqzboVUG12dKML8Cdxl1VTT|10|20-482-807-4156|9015.53|ven, express ideas. even, ironic +5067|Supplier#000005067|vIILkUb0P40EJsPkCgs mU0fUyNiPlcDiNn|19|29-498-613-2258|8057.44|. platelets about the blithely special requests wake quickly +5068|Supplier#000005068|lywBhkmjFn885,AxSnroD|6|16-491-789-4986|9481.98|blithely ironic deposits. slyly final theodolites cajole slyly about the express ex +5069|Supplier#000005069|HA5lbMpFJevFg10lH0,a2|15|25-328-522-7714|4645.07|s alongside of the unusual instructions promise blithely ironic instruc +5070|Supplier#000005070|8oIeUHboJ5hozgmr,QVk|13|23-229-247-9604|635.51| ironic accounts are carefully regular deposits. carefully regular dugouts according to th +5071|Supplier#000005071|RYDdMAh81QJMYzWDq9Pxt|12|22-628-554-8536|8960.23|s. courts cajole slyly around the +5072|Supplier#000005072| pcAH7lcrrXiT22a8Md2mWiwIaAra|19|29-400-578-9663|2616.57|ve the slyly bold multipliers. pend +5073|Supplier#000005073|YyQ5nXEFLYZ|8|18-749-295-6685|7663.56| cajole blithely carefully expr +5074|Supplier#000005074|HJ y6XJurxzcQzSJBKBy|17|27-682-574-4351|500.33|ietly regular foxes. even packag +5075|Supplier#000005075|L8ykfphBgptEtNl5DK7Syz3HiMoBpK|15|25-509-888-9880|2976.70| ironic deposits about the regular pinto be +5076|Supplier#000005076|Xl7h9ifgvIHmqxFLgWfHK4Gjav BkP|3|13-309-160-9273|-710.07|ounts cajole carefully amon +5077|Supplier#000005077|AoD06AAcIfuPu4B6YzUyFOYgVyb6|0|10-852-519-9875|9836.53|ven accounts. quickly express deposits was blithe +5078|Supplier#000005078|ML61AxsKm5wyA5nhF35X|0|10-785-180-5085|1666.74| grouches integrate carefully final requests. furiously pe +5079|Supplier#000005079|Oj0dmlwdcMrc JYwZSyfOuVpf5XEcuI,updCq|8|18-276-626-7832|4927.18|riously silent requests. quickl +5080|Supplier#000005080|,cCNkLr1lSa5sGMJCdr6W|4|14-680-489-5992|1055.14|posits across the regular ideas cajole +5081|Supplier#000005081|exVAd8XCFvfjruNqH|4|14-876-138-3641|-377.21|e blithely ironic asymptotes. bold pai +5082|Supplier#000005082|bWSoVdVj6qSqm|24|34-689-129-9336|5217.81| final dugouts affix fluffily blithely even theodolites. slyly sly accounts grow slyly un +5083|Supplier#000005083|ygFclVTpIKWAJEX6yDaN,Qeq6XUZKiKuvLRoNYq|18|28-658-224-1901|1802.75|thely express deposits. carefully final pinto beans sleep quickly requests! +5084|Supplier#000005084|3uuAO49y ROyS|18|28-691-821-6777|2753.91|packages. ironic packages cajole blithely bold pinto beans. furiously regular instru +5085|Supplier#000005085|PTibly42d9jcNqYjcWN|11|21-713-600-7449|5974.52|ar pinto beans. slyly final pinto beans above +5086|Supplier#000005086|WGw6x jIMPWWGLE2i2yLcDa5VTnYEyDO5|12|22-238-301-7360|3832.06|t furiously alongside of the f +5087|Supplier#000005087|q0c6r9wYVQx31IeGBZKfe|13|23-335-392-5204|9359.59|osits. quickly regular packages haggle among the quickly silent ins +5088|Supplier#000005088|X4UmuzzZBdYkCQR59IFULKDPYO|5|15-208-170-6484|4681.36|y. ironically bold sentiments after the unusual, iro +5089|Supplier#000005089|N6FJJ3hqOurzKReEyY1a|8|18-698-287-5739|5588.93| slyly bold asymptotes. thin packages wake blithely. ho +5090|Supplier#000005090|DgELVDIczXjsxv,z0e7Kp|24|34-735-223-4302|2187.94|ets. regular waters use slyly dependencies. ironic, close requests sle +5091|Supplier#000005091|UleyDBWl OFukWcPZXZIK i0KucFdQa4L3m|1|11-489-724-4275|8724.66|lar pinto beans are deposits. silent dependencies wake express, final excuses. boldly b +5092|Supplier#000005092|iHVSAoQ1m,,HAWMf|22|32-495-480-1228|3494.41|iously final foxes cajole. slyly pending accounts across the final courts sleep slyly aft +5093|Supplier#000005093|wubimDcO30UlP|8|18-356-823-5730|-504.70|egular pinto beans. fluffy, final foxes sleep furiously. slyly bold packages c +5094|Supplier#000005094|35COOPYtrR4fjqCH|0|10-663-141-3435|7493.34|r, even ideas use blithely about the quickly regular theodolites. instructions among the finally fin +5095|Supplier#000005095|dHS8NcFRoSmwZ|17|27-538-769-9170|-460.96|ss the regular theodolites wake among the fluffy platelets. slyly bold foxes along +5096|Supplier#000005096|L2TkpCTm9UqTEo8L1qu22B,4,2|11|21-371-681-3600|5531.40|y unusual accounts outside the +5097|Supplier#000005097|zs0ORto8VcSZbKQa5BOVtVjl3VlnwhZiThahBUm|7|17-122-172-6610|3335.77|oze ironic, ironic requests. carefully pending accounts sleep! carefu +5098|Supplier#000005098|AfuGe4l9jVsZSRUuuIAUQ7q9uREXFTT|1|11-810-329-5965|6232.88|ructions. slyly unusual packages despite the quickly pendi +5099|Supplier#000005099|kAzKSYipEE1wYMQmewdjxnl9,npiaH|12|22-251-188-1058|1012.73|egular packages haggle carefully slowly brave requests. unusual ideas about th +5100|Supplier#000005100|OfvYPs3Io,wEvvLHNaLuCX|3|13-237-801-4003|1821.87|ly. realms along the blithely regular packages boost blithely alongside of t +5101|Supplier#000005101|qMrkJAZa9T8jQxL9VLO43oNLQQB8Yux bv8Wwun|16|26-340-791-7115|5185.45|thely regular deposits cajole carefully according to the final instructions. quickly ironic +5102|Supplier#000005102|,i2pqfh6F 32nkGqSpHK6DyXQKCcEIHZjMtjbLQ0|24|34-504-244-1037|7814.88|ntegrate blithely special dependencies. foxes cajole blithely expres +5103|Supplier#000005103|DfoIXDaKULqKehaCB1RNNL1v9DFK7OYj|21|31-651-628-8911|2741.94|uriously final excuses. furiously final req +5104|Supplier#000005104|V6dIpvnddJ6BfMen9pRA4ZA5KLEcRAj3APWIB2|9|19-456-197-3940|2797.63|. carefully ironic tithes against the theodolites hagg +5105|Supplier#000005105|oTiURFggd0r,1EoqSN9,P7NcAqJuZ|12|22-456-148-7985|3804.00|p around the express ideas. regular, express dependencies are. even +5106|Supplier#000005106|oY8 g15FH2z87utclizeTsTykGpzmkFhE|4|14-250-778-8504|-247.87|accounts. packages within t +5107|Supplier#000005107|ym1,PQw1XhwsgMEBBEl6SekwvSa9xvAG,JTmKalY|5|15-924-840-5789|912.42| the decoys. blithely special fo +5108|Supplier#000005108|d3PLCdpPP3uE4GzbbAh4bWmU 7ecOifL9e1mNnzG|10|20-842-882-7047|9956.34|ronic accounts. carefully bold accounts sleep beyond +5109|Supplier#000005109|GKW7h2AyqVM FJfQsrJCwLJFht41tHhLQ3T|6|16-116-377-6604|2332.75|uickly regular requests are carefully sometimes final requests. slyly special requests u +5110|Supplier#000005110|OqU11U5FdpbHKDhiqQ3zCM7,3mVLiQ9eCFiv0Mx|6|16-915-186-9710|441.48|gular packages. pending packages alongside of the final packages sleep across the slyly ir +5111|Supplier#000005111|1hsNHdRSsNQLTdXfzsQykXCs1SCdY3mxs40Yh|4|14-399-146-5640|3837.52|y ironic deposits nag about the thinly +5112|Supplier#000005112|iK5vmw70sZ0uKyGK |2|12-368-711-9287|2269.60| the blithely regular deposits. blithely special deposits cajole fu +5113|Supplier#000005113|8Oa4 hYPK2IY2XtgGmJ3OSO42KKYhYMm0JT|0|10-257-814-5680|6190.94|riously special platelets. fluffily final foxes according to the carefully pending packages wake fu +5114|Supplier#000005114| DCuOgHcVIETSqehM|17|27-313-761-6035|185.64|osits nag slyly. deposits are furiously quickly dogged instructions. even attainme +5115|Supplier#000005115|xdh9QzWM6l299yCJRveS3oqf0JQs|24|34-674-575-7131|5012.65|ickly. blithely ruthless requests cajole. carefully special packages doubt. regular, pending pack +5116|Supplier#000005116|Cc9v,JkeDbBgx9Lm7aYjL4bGx|9|19-646-481-9129|7635.26|ts above the furiously pending foxes wake across the fi +5117|Supplier#000005117|zlBi4yswhLa1yv54Mrurx6Ldl2EWiegBiH|10|20-400-529-7218|8820.62|wind: slyly special deposits affix furiously behind the quickly ironic requests. fluffily express a +5118|Supplier#000005118|mBq1ON0bLT|1|11-539-770-3350|-797.25|en dependencies haggle fu +5119|Supplier#000005119|D8MbUqNfk1X5VYGuB|9|19-735-738-7984|7584.89|. blithely even requests are furiously after the permanentl +5120|Supplier#000005120|4ms,2ZOxHyGylzr8X0JswgHTS KuJ8fyobBbs|8|18-859-429-2143|9227.39| final requests cajole carefully careful ideas. blithely i +5121|Supplier#000005121|tSJ3v6tz8dkFGm1nBLzG5TDKWlwunrmsTbN|14|24-323-343-8567|4869.04|riously regular dependencies. quickly exp +5122|Supplier#000005122|mn ZeyFF,giB5t,S,7EKFvgO|2|12-540-857-5464|2988.55|le. unusual, even deposits boost quickly asymptote +5123|Supplier#000005123|jQvG,DczPSj8679Z4GqBj1ABBvL|4|14-664-609-3829|480.64|n, express braids use. slow packages +5124|Supplier#000005124|XMO88ZgYl1X5KbIFAYW 51GFZ12B6T71p|4|14-415-335-7489|2073.19|he furiously unusual accounts. packages use carefully across the fur +5125|Supplier#000005125|sdi55cW4MJZDi4|12|22-317-409-3573|4999.94|equests thrash. blithely even +5126|Supplier#000005126|R 6l6BFOMffZPRwYh0huGxCw3OzSGF|1|11-290-424-9612|3006.99|ckages haggle slyly. quickly final deposits wake f +5127|Supplier#000005127|7zpv255hJ3pDa7G5fNT|23|33-671-379-6897|2485.44|g to the packages-- blithely express +5128|Supplier#000005128|QI02G,a0atCr1OI8RRXMLeYB 4|16|26-151-411-5551|3796.72|, ironic accounts haggle slyly +5129|Supplier#000005129|Tg7zMb1u8XNC53k0ueUR1X|7|17-848-884-2347|9471.26|refully ironic accounts. ironic packages haggle blithely silent foxes. blithely regular ideas sleep +5130|Supplier#000005130|GhwYHR8ZBbrM28CrR,fMdeH8wgO2la|0|10-224-684-9502|492.92|, final excuses are even, ironic accounts. blithely even accounts boost furiously along the even +5131|Supplier#000005131|shQmqR2Q,x4NV|15|25-415-425-4385|-541.56|l requests nag. deposits throughout the bold depths wake carefully brave excuse +5132|Supplier#000005132|Z0t9cqKCy6ZugNI73N6EjPd1P0hwjGzH|1|11-848-121-8561|3979.10|s. regular requests along the ironic deposits dazzle slyly pending asymptotes. ironic, fina +5133|Supplier#000005133|8rsLVQ3lT7SssypCwtt|9|19-842-527-7983|5644.72|rts. furiously unusual ideas sleep blithely between the furiousl +5134|Supplier#000005134|Q861AErQN2rJGI|0|10-942-484-8952|4872.03|heodolites x-ray furiously above the requests. carefully ironic instructions boost furiously again +5135|Supplier#000005135|lmDG6dKYLGrlR 7pJKTtgx|22|32-303-648-8880|9165.36|uriously special accounts. thin warhorses nod. carefully pending dugouts imp +5136|Supplier#000005136|COFHYLZktt6ACL0C5VEvq2mlR1Bhevo|17|27-745-825-2995|7256.41|yly across the theodolites. unusual, pendi +5137|Supplier#000005137|pUbK,Ijct04c8bNNelj8VRoHcNcSOsimX S|7|17-568-560-4547|-549.28|le theodolites against the furiously regular req +5138|Supplier#000005138|DxnEEeuXwJn7xDdH1igbCtSIwiEY,H0fzlhZAP|17|27-516-189-6187|6116.88|iously special theodolites wake. requests use slyly bold orbits-- final a +5139|Supplier#000005139|uxbu7kp0 6ntQzpSLIGhcpRZJqWfZVfrS8Mnu|0|10-351-271-6252|9490.61|ssly express ideas. blithely bol +5140|Supplier#000005140|8zjhdOxG W0fTytk2FtNrR4aa0ZI,rQ M0V1eZqx|8|18-304-303-2247|6787.74|pecial accounts are slyly. carefully final +5141|Supplier#000005141|IkbWxJSDqWE70|13|23-774-675-2750|9639.46|refully even instructions nag blithely a +5142|Supplier#000005142|7fUvp, GK6DzUsj,1Uea6pRycO9PUjkiQJO|24|34-335-112-8603|566.15| special accounts sleep blithely above the final dependencies. carefully even dolphins poach slyl +5143|Supplier#000005143|TZiNN19YmRRhai6d2g9bcPJhFtb8tm9kPeWqV|8|18-278-521-4605|9254.13|nts. ironic instructions cajole furiously slyly final req +5144|Supplier#000005144|8p56d9o5NToh4y|9|19-616-280-8575|3222.23|grouches sleep furiously. even accounts wake furiously special foxes. quickly regular +5145|Supplier#000005145|nyYvIMs7p WbgO,|2|12-292-675-8158|1056.91|ers. slyly ironic asymptotes x-ray. carefully even accounts cajole ironic accounts. +5146|Supplier#000005146|91UwVfnqtjM8aRg8|23|33-134-543-8120|7379.56|nal pinto beans hinder. fl +5147|Supplier#000005147|abivKW9It0Cy6YT1uD0RP0w3|15|25-763-190-7789|1278.26|e forges cajole. quickly even requests +5148|Supplier#000005148|kT4ciVFslx9z4s79p Js825|19|29-252-617-4850|9643.55|final excuses. final ideas boost quickly furiously speci +5149|Supplier#000005149|qWunGeW,VzDslEqL,e,s8b4WxTw25uf94nYrUi1P|4|14-548-585-5934|3217.55|ages haggle slyly according to the express deposits. blithely regular packages alongside of the sl +5150|Supplier#000005150|sFYSjz4iydwo SAB2rOz1c32la,4jd5Oz2dwYT6r|8|18-206-326-3630|-655.06|ending, ironic requests. even ideas sleep carefully ironic dep +5151|Supplier#000005151|zMRPoZdcswKgAc|11|21-713-704-3070|-723.30| packages along the ironic courts nag accordi +5152|Supplier#000005152|AF2Bpt8T5lEvNnVsXidYmnYEJyGVMc|9|19-145-227-9425|7494.66|as. carefully ironic accounts among the sentiments sleep bold ideas. carefully regular a +5153|Supplier#000005153|C7yI5crVqTupS zGnZfB5AS3IUpCTicGZOhgV|23|33-227-904-9180|-237.93|egular accounts haggle furiously? +5154|Supplier#000005154|WAe fB57PNPvbMiBLb2|17|27-435-200-9120|3001.60| carefully express accounts detect slyly carefully ironic ideas. carefully unusual theo +5155|Supplier#000005155|9xgEZqXHL4kmEFh9,wgmzWx6bPSqTvwOOop|7|17-373-182-1794|-62.62|fter the fluffily ironic somas. furiously regular sheaves sleep. carefully even requests sublate fu +5156|Supplier#000005156|oz4CDE3AsHTru75lmD9JTXX2zKq|14|24-300-725-2234|1493.03|sts believe regular packages? even accounts believe. blithely pending pinto beans sleep a +5157|Supplier#000005157|9BPqSAL9F tcO|4|14-351-630-3763|312.73|lithely express, special foxes. unusual, bold accounts sleep blithely about the quickly special foxe +5158|Supplier#000005158| opkloZ,lvEdkjaxo6|3|13-268-293-4359|6964.16| sleep slyly according to +5159|Supplier#000005159|I5a6e36u3OZv3kkaUZyuu|19|29-359-857-1642|4702.50|ully blithe accounts sleep slowly regular packages. slyly ironic pinto beans above the +5160|Supplier#000005160|iYvEHZmj762YNHC4zRT QSLXdk|15|25-355-259-4155|8930.48|iresias nag furiously along the carefully ironic pinto beans. b +5161|Supplier#000005161|tMgynFzEyBoRhflBYqo|5|15-585-813-7109|8641.38|uickly ironic packages haggle quic +5162|Supplier#000005162|yboWLOJWaU55ZbBOG |8|18-588-399-9411|818.51|le. regular packages across +5163|Supplier#000005163|H1ru7Slmb6B 2oe3ek|19|29-940-730-9740|4637.54|osits nag. furiously expres +5164|Supplier#000005164|OWQm 4iCIFMHSBLiocIKzYWd5KUtYe21|23|33-734-625-1360|6176.52|y even requests are quickly bold platelets. furiously regular dolphins cajole fluffily. som +5165|Supplier#000005165|4TbLeqYIZflknnvRjqHLxFgMuET9fqe6RH|8|18-327-142-6539|-170.54|s above the slyly final dinos sleep quickly ironic platelets. +5166|Supplier#000005166|Mp6GjDPkur9B4JTgOUTaO10jA0ZkHwuJyR|4|14-945-523-4824|2706.20|requests along the quickly regular accounts sleep sile +5167|Supplier#000005167|NiEBRUM0BudGHmgb|15|25-248-319-7059|7019.68|ly special excuses x-ray quickly final requests. regular, ironic requests haggle slyly after the +5168|Supplier#000005168|qb0SFvINfiJ63eMJWohwEDCFRy|9|19-374-543-3498|2848.35|final requests along the ironic theodolites wake blithely outside the slyly ironic platelets. blit +5169|Supplier#000005169|oO6GZCtr7F|6|16-492-650-1709|9211.65| requests. boldly regular requests use blithely packages. carefu +5170|Supplier#000005170|x3ZL3yjbTL|17|27-431-747-5628|4323.24|lly regular courts poach furiously blithely regular dolphins. regular, regular courts alongside of t +5171|Supplier#000005171|qildlLehhwauS3n70fsTtrzCZRpZ|19|29-780-429-8831|3615.17|unts are fluffily according to the slyly even packages. dep +5172|Supplier#000005172|RG58pSuUcrJJjbomwgF1eJpk0AoPJI5SxZxa14|15|25-372-629-5030|5186.97|. quickly ironic instructions are blithely above the pending +5173|Supplier#000005173|LbPNUeqSAfeK4VwL5bNNRx|0|10-694-344-8752|359.53|wake blithely. blithely final instruction +5174|Supplier#000005174|ZeXBGYhZj,uAq8m5gyYWM8|6|16-284-735-1835|1559.42|blithely. doggedly bold deposits sleep bold, silent excuses. bold ideas affix. care +5175|Supplier#000005175|Hp0dm4so MR0Md9dMg2ZV8E140Utqb83y|12|22-150-780-3784|5120.85|ve the furiously express dolphins boost according to the f +5176|Supplier#000005176|FSLVwQRBSCOB7YWvfroKwLXp|11|21-324-709-3029|142.59|e of the furiously regular deposits cajole a +5177|Supplier#000005177|qES41LEoMAt6W0cJ|19|29-602-713-8599|7393.65|haggle fluffily regular packages? enticingly ironic foxes +5178|Supplier#000005178|1mMfgEZaCtsV6PpK0r33jRpF|20|30-345-637-2822|7179.85|le carefully across the packages. bold, regular accoun +5179|Supplier#000005179|nx,tP44fkLZiM,h7RDF15Al|13|23-667-711-3540|-179.89|deposits cajole. carefully bold attainments are slyly pending requests. carefully +5180|Supplier#000005180|Z91h3TTs6Fddr|16|26-683-827-8177|-540.57|uctions. furiously ironic ideas detect. excuses na +5181|Supplier#000005181|jFb5CuLfU1 jYow6irbVrmFEeVzyLu|14|24-842-241-5902|9566.41|refully silent requests. carefu +5182|Supplier#000005182|WrmiOA5X1ogypaAptJSdpEB9AaI|0|10-345-373-7701|4674.44|ve the even excuses could have to haggle slowly regular requests. fluffily silent asymptotes +5183|Supplier#000005183|DGEg4PqYMLuJbTf AmfG3zvPcQ,F|0|10-649-978-6034|-629.63| carefully ironic frets are slyly special ideas. blithely fluffy deposits main +5184|Supplier#000005184|KcBwEwspfK8QazP|18|28-717-226-1848|8676.06|al deposits sleep slyly regular, regular packages. even, pending id +5185|Supplier#000005185|GoSFICuKRjVQbJcRAQCPhIyF9Ur|19|29-952-175-5400|598.12|the furiously regular requests. furiously final requests against the +5186|Supplier#000005186| 9OWZt7iQ9zxx|10|20-801-725-4571|7615.64|regular foxes cajole quickly after the stealthily ironic requests. accounts use furiously +5187|Supplier#000005187|rgV zR GZMR0mA3ZDQS|12|22-559-252-9408|9135.11|s, ironic requests across the final packages wake alongs +5188|Supplier#000005188|Y9oC3,NpTpcLqZFcPzzIbiril5K,VpXWyS|18|28-325-390-6843|4875.65|kly ironic deposits against the furiously unusual pinto beans w +5189|Supplier#000005189|WMElza2PebRzw2v,NxPbmjinfDik7HAtBxQ4axWa|1|11-624-175-7408|4100.43|final excuses affix fluffily about th +5190|Supplier#000005190|Ho GIXxcwcCBwVqH,U7W|10|20-266-562-4864|9848.82|atelets cajole slyly among the final ideas. carefully thin accounts integrate fu +5191|Supplier#000005191|ucNlv,jALhezvmIibpSIuWu|7|17-792-220-3433|-761.79|oxes haggle quickly against the carefully express tithes +5192|Supplier#000005192|JDp4rhXiDw0kf6RH|3|13-136-480-6518|9584.92|iously regular dolphins affix fluffily special instructions. quic +5193|Supplier#000005193|Elnv5Y9NEKmdhN5Gh35k6pkvV|20|30-994-715-2763|7810.29|excuses wake furiously. blithe +5194|Supplier#000005194|Lb2k94Vp9Hq4sNptU7Eck5,o3RaaumJl 0aZL,6r|20|30-914-830-3171|2671.97|ckages. quickly pending accounts boost carefully unusual packages. requests sleep blithely accordin +5195|Supplier#000005195|Woi3b2ZaicPh ZSfu1EfXhE|3|13-622-661-2956|3717.34|eep quickly quiet packages. blithely ironic ideas cajole carefully carefully reg +5196|Supplier#000005196|EKn1rQDs252VxJ5GGcaZ8yu8DWMiaadb|3|13-862-804-8890|6283.81| special, regular accounts. deposits cajole. carefully even theodolites det +5197|Supplier#000005197|sIwLOFSUt8n138AU7ph|6|16-906-360-1957|-955.74|final ideas! fluffily regular instructions shall have to haggle carefully about the fin +5198|Supplier#000005198|Mb6qDaYTdtK53DFOdm3WDZZZGkhKcErHFhgg1P|19|29-728-923-8008|5861.18|pending pinto beans. fluffily final ideas affix +5199|Supplier#000005199|xMIKy6lAtw|21|31-599-973-3564|1842.96|nusual deposits haggle fluffily regular, ironic packages. quickly ironic ideas about the furious +5200|Supplier#000005200|ZfiCku6sEozvAiFOOZOR|4|14-673-857-2664|7650.69| fluffily pending theodolite +5201|Supplier#000005201|kMUTOmy4b l6mkn2ukSvjtpwTVl|4|14-156-913-9074|4113.73| fluffily regular packages detect quickly above the sly packages. blithely bold depos +5202|Supplier#000005202|C2PMMjKmpgC1ryNad3CI4kMVE9UaH81Uxw7J12 K|1|11-526-435-3834|2463.46| blithely close requests. blithely even requests thrash? +5203|Supplier#000005203|8uIpv041PoCb034MOB LeEzbtOgqY51aK|1|11-214-499-8004|8344.14| cajole fluffily. furiousl +5204|Supplier#000005204|sFTr9PGcYUjo5eReeIBHsjJIDjJMq,|2|12-387-945-6443|-929.58|l packages play quickly. slyly pending a +5205|Supplier#000005205|KCpbsRqeQQ36GtHLl0|11|21-308-705-4634|5447.11| the slyly ironic packages nod fluffily according to the s +5206|Supplier#000005206|Lu2gzikoW8HeNJxr3H85pGN|4|14-492-704-4627|5540.94|ts sleep carefully brave f +5207|Supplier#000005207|DW2CUhTSNXOMoLj1FUV7gR4|22|32-596-629-2272|9641.14|scapades. quickly final accounts after th +5208|Supplier#000005208|xEqnN9YbnzKb 4u3u4V06xFGhlclFx wnzO7MAN|9|19-795-571-6227|6706.06|ly final accounts haggle carefull +5209|Supplier#000005209|ARRBCxXQedFizTPvUQWJUoa7AhB|19|29-615-534-8854|791.39|r, pending accounts after the carefully permanent asymptotes wake blithely u +5210|Supplier#000005210|Gb1SnzOsnMke d5AIDzIJEooElYL|0|10-453-269-7032|-196.07|s. pending asymptotes cajole slowly. slyly +5211|Supplier#000005211|fFhp6ZiDHnO1aa9A7sfaYj6xsoKw0TfSTlehuK7m|9|19-722-866-5081|2531.14|about the fluffily bold requests +5212|Supplier#000005212|M3rXkftRA77nO, TST8KfImf7I9,jhmA9InsuF|0|10-773-259-3830|1833.63|iously ironic accounts th +5213|Supplier#000005213|3MTMXf4oNg,qXdtKbfRTGhgdGg|11|21-760-229-1261|4725.64|s. furiously unusual dependencies are furiously. pinto beans sleep across th +5214|Supplier#000005214|FpKoJu0V7zK7eiMtEgVCgdQ|1|11-240-411-9546|1784.32| the slyly bold ideas are slyly close +5215|Supplier#000005215|IFKN5IkO3h|6|16-921-861-7733|957.53|hely above the bold instructions. reque +5216|Supplier#000005216|H39m5bsz Ux,VmJ|17|27-838-914-7588|9233.82|telets. furiously regular foxes wake. quickly final a +5217|Supplier#000005217|XKWuIicCsmrN6TOiPwZiC60suCz4vb2GxAcLLEIi|23|33-238-443-5421|2886.07|ly warthogs. regular braids sleep above the fluffily even tithes! fluffily unusual acco +5218|Supplier#000005218|LeY2,jtHtxXE06Yj23mQJQfSL,zzwXXFXq9j|18|28-727-621-9068|5185.33|braids. thin instructions are blithely. slyly final accounts are blithely according +5219|Supplier#000005219| AYnWt7jJhKM|12|22-509-835-6405|5832.66|r pinto beans. silently unusual ideas detect. carefully even ideas cajole quic +5220|Supplier#000005220|YQIMhm7NYmeeAOVTOTO,|16|26-701-146-2726|6508.65|ly regular dependencies. blithely special +5221|Supplier#000005221|,nGY52Cz98m8rB5jeDuIiR,k6FV5bP2gH|5|15-985-107-7643|4859.65|gular packages was carefully above the carefully final pinto beans. slyly final theodolites +5222|Supplier#000005222|iEA8OArWlk,S5FaYjKHOXuwXEl|16|26-864-417-6650|-808.16|ter the blithely ironic requests sleep furiously across the bold, silent excuses +5223|Supplier#000005223|tiQFKToZfpl7O8N4X6C Y24fq|7|17-159-169-6397|130.07|gular excuses. furiously bold p +5224|Supplier#000005224|aqnQ6P2kmDbQIEWvf 286zsWpgG06aa8srLaqx|17|27-233-923-2924|6024.50|resias wake quickly furiously ironic foxes. bravely express +5225|Supplier#000005225|BKyNPbd025r7,3hazPOINMWH3M5c6dsuXgk2JpU|24|34-258-492-9748|7731.17|efully pending foxes are. deposits along the furiously final dugouts use evenly among the deposit +5226|Supplier#000005226|t1kGmBe6n8|23|33-562-639-9530|3871.58|g the furiously final requ +5227|Supplier#000005227|qy3Uc0MZfrkDNG13pPXi73qyxuxzBiaLD|12|22-424-805-3724|2644.29| theodolites; pending, even foxes use. fluffily final excuses wake f +5228|Supplier#000005228|ValApyRxqqCdRKHilK591RUg2Y1m6A,2q3A7S4|10|20-443-286-2665|494.31|nt, even courts. carefully regular foxes boost slyly a +5229|Supplier#000005229|feyJ4zb6toV,Z11SjX6LCH08Kaj2KwFzM4wOA7N|10|20-987-260-3596|780.93|furiously about the fluffily regular pinto beans. pending, +5230|Supplier#000005230|eb0Jbiu0Ltq,cc|13|23-614-211-3496|9.31|ites wake slowly ironic req +5231|Supplier#000005231|kKIHjkiyv3L2O eSAloxpc6eZBQvXC|10|20-691-244-6674|437.60|hely regular pinto beans are. express deposits +5232|Supplier#000005232|xlSj2gbKDJQKLXbxsuA14uguomr5SMqhfNTL|8|18-746-919-8850|4761.36|tect blithely. pinto beans cajole unusual foxes. special platelets cajole! fi +5233|Supplier#000005233|LVx2tybap3pho04RYVA,|12|22-877-708-6775|8139.64|arefully regular ideas are regular packages? final, final packages caj +5234|Supplier#000005234|XzbBu2t9BXgMBX5mBbEJPS1A0hOzeXD|16|26-203-525-1919|5004.16|ns affix slyly around the fluffily ironic dependencies. carefull +5235|Supplier#000005235|nftsPk9H2tur0nI3YHMPw4hk|20|30-346-658-5459|7842.49|ld instructions detect quickly along the fluffily even +5236|Supplier#000005236|J1CVBhwMuEbyfqgMhjAragtEwu|20|30-794-178-5398|4826.78|ts. even, final instructions cajole blithely. fluffily +5237|Supplier#000005237|1auUEZrluONpxfzvuTc1Lq93uCIeucLK|5|15-732-795-6730|7628.45|ial requests. final, final accounts detect fluffi +5238|Supplier#000005238|jmtI76 8RNG8Z2BZu|21|31-883-134-5440|4401.33|y ironic Tiresias nag blithely pending instructions. ironic, ironic requests use silent de +5239|Supplier#000005239|zMh6trjCArLcsPNeRnpE|19|29-911-233-9405|7228.36|lly special dolphins nag blithel +5240|Supplier#000005240|zcE F4crZXfDXlITkE ugRAwEkN6OIzU|21|31-564-530-9694|1121.92|tealthy accounts wake furiously. furiously regular pinto beans ha +5241|Supplier#000005241|YNh ,,HpYfRMND5TwWJrr4Oe1Hdc5lu2ra6lqx|0|10-234-545-5083|13.97|ts. blithely silent packages impress among the carefully express pinto beans. +5242|Supplier#000005242|TQPTwj5KXFplnLO8,M,x1dQvP1FH|7|17-475-160-7395|1696.74|al requests sleep quickly. blithely regular pinto beans are about the quickly final deposits. bl +5243|Supplier#000005243|YEZuqd5VnYzjHlhjEYAnHEBvNP2bv9z|8|18-429-254-1535|3087.80|lyly daring pinto beans shall impress whithout the ideas. ironic ideas na +5244|Supplier#000005244|DN ECrPEXmfa4ai6bqahAPl|20|30-630-560-5327|7599.29|furiously ironic requests? slyly ironic deposits are s +5245|Supplier#000005245|M0Bxbdoh5zB2FmYK3fV8DVr8QN4ZqVKn|9|19-499-819-2771|9389.01|e express deposits boost ab +5246|Supplier#000005246|K8sHgjaRg6Q96,MgtXDLXMysg0yBMS9tFY2ZA6o|10|20-336-563-3852|4146.09|ironic theodolites. furiously regular accounts sleep f +5247|Supplier#000005247|yDmbiJqH zqDpcUUQ1vC9mHs5ntD1NnBQCadMDHz|23|33-478-798-3015|7255.81|onic requests. quickly special attainments sleep slyly about the final asymptotes +5248|Supplier#000005248|1B7xzcmfhe,,7lhFp|2|12-222-714-7472|410.63|ual accounts according to the pending platelets affix after th +5249|Supplier#000005249|w165RaBOLOHKBYlAQO9|1|11-736-568-8463|2470.75|d theodolites. even accounts around the blithely stealthy packages haggle fluffily evenly fina +5250|Supplier#000005250|B3rqp0xbSEim4Mpy2RH J|23|33-320-228-2957|9936.22|etect about the furiously final accounts. slyly ironic pinto beans sleep inside the furiously +5251|Supplier#000005251|OAOfy3S9Q OUjL28,FVs|13|23-237-944-7853|5090.06| the regular courts boost after the furiously final reques +5252|Supplier#000005252|YBM2D2HGMLHuQ7780cIqoDOFgeiHsps|5|15-385-337-5993|-409.34|pendencies: carefully regular deposits along the final, ironic deposits are slyly a +5253|Supplier#000005253|Y3BLwN4lrUHloWSY1APa5dyDQv3mJCNcFxWj|2|12-404-400-3299|4117.09|. furiously unusual packages cajole aro +5254|Supplier#000005254|WLC8gtLs28SsRdWIAkmN|4|14-602-778-7716|3927.09|he final, final requests haggle blithely about the dependencies? pendin +5255|Supplier#000005255|iLfR,F6gzZDX3qrkx|0|10-587-625-9528|583.61| requests across the asymptot +5256|Supplier#000005256|Onc3t57VAMchm,pmoVLaU8bONni9NsuaM PzMMFz|3|13-180-538-8836|5624.79|olites breach carefully alongside of the carefully bold packages. blithe reques +5257|Supplier#000005257|f9g8SEHB7obMj3QXAjXS2vfYY22|3|13-848-650-6906|-206.07|sly ironic accounts haggle slyly dogged requests. busy courts detect blithe +5258|Supplier#000005258|Yrdz61Szt8oN0PadcEVwaF2n|14|24-587-852-3767|-512.35|ptotes boost furiously. final +5259|Supplier#000005259|MDUZfnUZhfMv5s|9|19-685-427-3631|-207.60|le fluffily above the regular, bold ideas. silently +5260|Supplier#000005260|oIi3inr8BuvL|11|21-757-882-4929|6830.39|ed to haggle along the sly +5261|Supplier#000005261|LPgJUosc4 lEJv9GmziGJbm2lU4ufrl9lzW|10|20-350-105-8148|409.23|ep quickly. bold, close waters are blithely across the stealthily +5262|Supplier#000005262|FIF9QmqWKYv,0rQdgvx9SikmUAoyG|18|28-353-676-7031|857.80|ideas. carefully ironic pinto beans sleep carefully among the f +5263|Supplier#000005263|i5GQcIdP4z6EUG5JGiJbNhlHEIOK7UkwNrxvA2H8|6|16-252-824-5951|5012.54|tect regular theodolites! express, even pinto beans are carefully. realms nag blithely ab +5264|Supplier#000005264|DFumlUmaf2JgYDrslLKwYGcZs1lCQhC|2|12-584-970-2655|5761.25|ironic ideas wake according to the ironic theodolites-- blith +5265|Supplier#000005265|lia7ERpiunMw,Ni87P4dqtUgG0Emxyy2D0A3|18|28-229-631-5492|5360.33|ckly furiously special accounts: furiously r +5266|Supplier#000005266|02jBX27NfItPhLbcYxnO6Y8mD92z9eVv3fIShKvc|15|25-412-718-8944|-980.44|e furiously regular theodolites. final, regular requests according to the fu +5267|Supplier#000005267|CuVb30nVEIorIspfE|14|24-301-579-8706|8728.10|iously regular accounts. ironi +5268|Supplier#000005268|thwtEsD6637kzSSBEQqtVKVZN|18|28-317-293-2835|6177.93|efully final decoys detect si +5269|Supplier#000005269|00WEnXEU42e9L0lmz|9|19-376-605-9501|1529.80|riously regular foxes according to the instructions cajole carefully final +5270|Supplier#000005270|hLnk3T32Fqen43k40FJMSUOnjZZt,h6F,qNwJ43|5|15-821-957-1766|2249.28|lites cajole quickly quickly final platelets. carefully regular ac +5271|Supplier#000005271|oPg1AijECJ1Q3fIPKkRdOpm2QB9g9|8|18-649-425-7033|5681.91|olites boost slyly fluffily ruthless +5272|Supplier#000005272|Quvb9Q,mTDT7jvDBfRmOWBRZP3R RAcoH5|7|17-831-726-6269|2907.40|lowly above the quickly even foxes. blithely blithe +5273|Supplier#000005273|jGX4x9bdqqor|11|21-345-660-7816|655.40|between the furiously pending accounts. blithely even accounts among +5274|Supplier#000005274|CVAHtph1pESdYsEYP,fO1AF2J|5|15-619-409-3013|2201.54|ing to the packages. ironic requests +5275|Supplier#000005275|3EMhwtef3sDkpjm|11|21-897-590-1873|6273.98|arefully unusual requests cajole furiously furiously regular packages. ironic, express pinto beans +5276|Supplier#000005276|inh0KOhRfLM7WRhdRNvJJDQx|11|21-834-496-7360|9806.52|the slyly unusual theodolites; carefully even accounts use slyly. sl +5277|Supplier#000005277|7OMBbu4xvPHasA0qzR7DSfb7Bq,LqBngqSZI5Zf|20|30-143-213-1573|3248.54|ously express deposits. even, regular pinto beans +5278|Supplier#000005278|EniIvKdYNIjcxEwhhzbSZM8rV|19|29-936-727-8831|3211.33|s? slyly bold foxes could have to engage! sheaves use t +5279|Supplier#000005279|IUwBVndhao,Y7I|17|27-182-705-1661|4806.54|ickly along the carefully even hockey players. sp +5280|Supplier#000005280|NB0MFCA5mH9UklskxDx Qi3oX h39gLySVTK|9|19-299-951-6016|8601.37|accounts integrate according to the furiously express cour +5281|Supplier#000005281|bWv9QtKGuhX5SLgnGPjfQYyJuKxnwhhuq2bq|14|24-253-579-1558|-670.56|blithely pending, ironic packages. blithe, regular requests boost along the fluffily final +5282|Supplier#000005282|NQI2D1E3kxF678qVMz18d03rhoRDMlI|9|19-544-884-7288|2761.29|nic accounts wake quietly above the blithely final instructions. even foxes doubt slyly. +5283|Supplier#000005283|5fxYXxwXy,TQX,MqDC2hxzyQ|3|13-542-389-4193|8925.27|sual packages thrash daringly regular deposits; pinto beans a +5284|Supplier#000005284|0YkvBGmuSHFbh4sAQ8X K|15|25-316-478-6289|9074.79|d requests. ironic, express request +5285|Supplier#000005285|Tt3MXwmq5RSl8b7|13|23-795-465-1559|3098.28|s haggle at the foxes. regular, special theodolites haggle regular +5286|Supplier#000005286|YOOfyQndKArEGwg2g9BRDBe7rI60c7TuuK|8|18-736-929-9371|7837.70| express deposits. carefully silent theodolites sleep blithely throug +5287|Supplier#000005287|xuHdQHi,qvGq1zD6y295Vs5T8hiDv0MDgcNy,0AM|13|23-596-742-7614|1332.83|nd the carefully final pinto beans. care +5288|Supplier#000005288|AqD0tMyqQg6hudWRPq7fkEClAwchu|16|26-212-331-6862|2818.15|slyly across the regular packages. boldly regular a +5289|Supplier#000005289|62XeOur9SnXgbdjGwb9E1aJIEBr5PA9|21|31-400-894-4035|7597.53|s. slyly final deposits kindle fluffily. dependencies detect blithely. packages must are furiously. +5290|Supplier#000005290|A4uwrH53UGhU7qiyZbfa8xeiqdGpyDvK|14|24-589-364-3552|1756.99|ggle fluffily accounts. si +5291|Supplier#000005291|coaFmwykbqVOJZi,C,Rn3,rZFpZ6CNMWEoX|13|23-808-160-2612|3666.45|ng theodolites. fluffily final instructions are carefully. carefully regular deposit +5292|Supplier#000005292|sRBWIIavn0AEKUn,tc|0|10-208-943-1119|9008.23|y according to the carefully even packages. busily unusual pla +5293|Supplier#000005293|gCER6dyZEUVR7AZsRzN6IYF 7nYIe|23|33-775-738-5118|6371.44|utside the furiously even pinto beans. fluffily regula +5294|Supplier#000005294|cs2BWiaTa28ZhyRagft BUqgV5r88J5m|10|20-355-767-9259|8877.34| pinto beans was slyly slyly iron +5295|Supplier#000005295|quPotRRrXwUrsS|17|27-364-196-7935|6482.05|ding, blithe deposits haggle above the enticingly +5296|Supplier#000005296|sk,N6IfeiTByYqUGr7n|9|19-887-791-5636|8799.12|s. quickly bold somas boost fluffily furiously express requests. blithel +5297|Supplier#000005297|nkq6 NFnEfhdy6|9|19-119-739-1447|934.67|lets. final, bold foxes are blithely at the accounts. slyly final accounts wake. furiously even noto +5298|Supplier#000005298|jtFU3iIMXr18ExQQAQRlyCtOOITSBf9eauEhK|2|12-327-575-4331|-990.16|final foxes sleep according to the furiously +5299|Supplier#000005299|m7Y2G8Pg,kl5AoMPK|7|17-904-495-9057|-752.27|. carefully close foxes x-ray. carefully even packa +5300|Supplier#000005300|gXG28YqpxU|3|13-790-538-6012|4454.86|y even theodolites boost. final foxes unwind. regular packages haggle carefully idea +5301|Supplier#000005301|cAd4Fk5waT|15|25-797-599-3277|4784.02|to beans. slyly unusual foxes cajole carefully. carefully final dolphins haggle +5302|Supplier#000005302|7zSymWFxEuf b9G3MlXSkV2T6Hxr4|0|10-723-651-2820|4081.71|into beans after the ironic courts sleep finally special accounts. accounts cajole blithely unu +5303|Supplier#000005303|i0TWTbAfLOBiFb22S|16|26-822-375-3242|3323.35|g, even theodolites. requests according to the final, express instructions s +5304|Supplier#000005304|N r0cPrm95ETtyq9a4mh|20|30-863-116-4144|6525.95|fluffily ironic accounts wake quickly under the deposits. furiously even deposits use +5305|Supplier#000005305|nq0BDIq7RkB3Nlf29acxIE|13|23-259-536-6169|9611.79| requests haggle slyly ideas. furiously express requests haggle furiously above the ironic theodo +5306|Supplier#000005306|9t8Y8 QqSIsoADPt6NLdk,TP5zyRx41oBUlgoGc9|23|33-632-514-7931|8376.52|ly final accounts sleep special, regular requests. furiously regular +5307|Supplier#000005307|Bn8K7K8FZEbZoG5L,|7|17-876-374-4719|4154.53|regular accounts. slyly final gifts c +5308|Supplier#000005308|9Ew1qiEgz2BP4DU6qfabYI9kJB60|21|31-891-878-3480|6110.79|key players engage. slyly bold asymptotes cajole. fl +5309|Supplier#000005309|kzmvKlSiri|3|13-475-801-8455|6152.09|final deposits sleep regu +5310|Supplier#000005310|C,z5yqtkKyx|16|26-876-755-4457|7148.52|l foxes. blithely unusual requests af +5311|Supplier#000005311|m2kwKAYNIe9n5ysrLXIVkPd|0|10-378-981-1830|1285.02|tions. ironic ideas above the final accounts sleep furiously quickly final accounts. iron +5312|Supplier#000005312|M7fSgSLXzpnE,EKcN39Ya |11|21-991-747-7312|2320.40|inal packages play blithely final instructions. car +5313|Supplier#000005313|bTIwc5CTKzNkrWOIYI4WlFB2kkjNRSclGjk|15|25-478-251-1194|1991.20|ar multipliers nag alongside of the regular, even the +5314|Supplier#000005314|,MOuwKmaLVA2mKYCVX4s5e2povJW|1|11-561-100-9470|2064.62|lithely idle ideas. reque +5315|Supplier#000005315|P41wx8QlVuH0KVF SsfB6q2Bz|24|34-867-473-1748|4104.02|counts affix blithely quiet theodolites. slyly ironic +5316|Supplier#000005316|9oik,Nu,ADtHG1hkjAUgh5KN1sLbFNpFqQzthMH|20|30-773-378-9147|4235.75|lithely fluffily final requests. fluffily final depths wake furiously ironic +5317|Supplier#000005317|lPOPHufNjwZaUJGVNHCC2DE FYQcKZBzHltL5|21|31-872-372-7076|7318.56|ntegrate about the slyly pen +5318|Supplier#000005318|Wp5q,sgo1rdt397icDFq5WIwLZVNWv0JqUfkv |17|27-820-230-5782|-957.52|s sleep fluffily. blithely pending ideas impr +5319|Supplier#000005319|6BU3YEG2abhHe6VgoaXUl5B9BulZKxWSPqY|20|30-658-404-7761|8799.04|ing deposits wake carefully. blithely bold requests nod quickly across the pinto beans. e +5320|Supplier#000005320|0SVtkH70cY3|9|19-157-710-7476|-427.36|ilent instructions-- ironic, pending dolphins sleep carefully ironic ideas. regular deposits boost s +5321|Supplier#000005321|uPBUzETr1ILHP1k3niRF|8|18-380-243-7261|9357.02|d the special dependencies. blithely ironic dolphins about the carefully even package +5322|Supplier#000005322|ZXCJ727TY MBYpQQPc1Q|6|16-978-250-2432|5497.89| bold dinos use bold pinto beans. slyly pending pinto beans wake. express accou +5323|Supplier#000005323|tMCkdqbDoyNo8vMIkzjBqYexoRAuv,T6 qzcu|3|13-710-468-7118|1113.75|ths. idly even theodolites along the carefully ironic depths are carefu +5324|Supplier#000005324|BH2OtHA997bh6QOs351YE |1|11-272-990-3856|5768.93|y silent dependencies poach carefully. even, special d +5325|Supplier#000005325|7YYIZmOTDpq4Ls0u5jj|9|19-252-144-8429|2395.56| permanent instructions wake slyly theodolites. final, regular packages haggle ou +5326|Supplier#000005326|gJvaakYDgaEtUOnf jSYuneO6DeM1RWlSB|8|18-562-715-6646|101.75|ideas. regular accounts use furiously. carefully regular instructions are acc +5327|Supplier#000005327|8ezizGofvsnWm1giKVGk|21|31-362-378-8693|3020.41|about the ideas boost blithely among the slyly iron +5328|Supplier#000005328|qKN,R5,XAFQamNkEbPrM7us ,5X LWn|1|11-370-937-1426|7978.98|ly ironic requests. carefully regular accounts haggle blithely blithely re +5329|Supplier#000005329|YW3bDjvPRLek4ic8WlGsqE|14|24-208-769-6460|-162.85|xpress, regular accounts hag +5330|Supplier#000005330|Yi67rc2o6A8BAhIuoZCATP8K UyNR|15|25-703-355-5851|-95.52|ly ironic requests above the carefully final packages serve quickly aroun +5331|Supplier#000005331|drSNCZY5Uu thezBqCbUbzTrpB|8|18-642-992-6796|9560.14|ag across the quickly regular deposits. ir +5332|Supplier#000005332|GDYJGl71XU4cE80rGsTtaRbOinvUG,ogq6xff1|22|32-321-949-6369|9421.93|ts cajole slyly above the blithely regular theodolites. quickly fin +5333|Supplier#000005333|pMMNZumvH8xG1tr9s|2|12-533-283-4074|7529.56|ccounts sleep carefully silent requests. p +5334|Supplier#000005334|rr5zrG4zBqvGu41NoGk,6MTdfiSwR7uxhLXD6H9|16|26-710-997-8158|3303.21|lyly regular deposits. iron +5335|Supplier#000005335| BKN QXPN9OfeK|2|12-429-977-4251|9066.05|equests. carefully unusual dependencies nag furiously. instruc +5336|Supplier#000005336|fATFQFISLgpcORhhnSYdJa|17|27-275-808-8090|-807.26| the ironic instructions maintain slyly alongside of the silent deposits-- slyly final realms haggl +5337|Supplier#000005337|z nBSOeXSXoEf|22|32-641-317-1163|4341.58|ual dolphins snooze fluffily. carefully silent ideas use. quickly silent instructions for th +5338|Supplier#000005338|I4Gs7qhuOV|22|32-634-381-9233|3894.90|y even depths nag even, express accounts. carefully regular in +5339|Supplier#000005339|HZ3RBdx2f,nrtu60,|10|20-249-930-6214|9614.31|t. regular deposits sleep. final instructions above +5340|Supplier#000005340|bTGt5CoFIRJm lhjF8ZR4|17|27-812-993-1434|6816.94|heodolites wake fluffily. carefully regular instructions boost final realms. carefully pending depos +5341|Supplier#000005341|wdlEEEwb1AESCQoms1UEL SEY9Mh1Z8G|1|11-322-276-9107|5300.56|leep about the ironic ideas; pe +5342|Supplier#000005342|0S0xlR9FDgKyYlA|1|11-646-881-6213|6604.91|inal packages. furiously regular packages about the slyly even requests are alongside of the ca +5343|Supplier#000005343|0JtugdtneLhtOMcZJ60GILnDDv9RS1|11|21-812-327-2995|5779.02|ing foxes. furiously even +5344|Supplier#000005344|z0KWYOvRQh5DkVkFwCZvBLIUPpPOv|17|27-560-765-3074|7086.90|gside of the quickly express requests. bold asymptotes integrate careful +5345|Supplier#000005345|TFPYpO6 iV4HN7x6fgg59TEnK|16|26-628-127-6169|-444.19|accounts wake slyly across the pending accou +5346|Supplier#000005346|C8aNEU8o3OTi4TWo RK0|22|32-682-989-9332|1371.85| the slyly ironic deposits. requests around the fluffily pending frays are carefully re +5347|Supplier#000005347|R UJfAVRhAKjgH|20|30-213-668-2389|9598.00|ites. packages cajole about the furiously regular accounts. blithely unusual packages are dogg +5348|Supplier#000005348|3bcel5ECu1vsgXf0G 2pMUkHN ndAmZpAp|12|22-172-249-8131|7611.48| pending requests. furiously final accounts hang carefully against the slyly spe +5349|Supplier#000005349|qbFEY2kp37p pHc,SxJvTWM1P myw,|0|10-485-956-7509|5201.40|cajole around the carefully even multipliers. furiously ironic instructions cajo +5350|Supplier#000005350|Pf2R9BM48dv6Tt|6|16-724-955-9576|975.96|d requests. regular, ironic requests among the blithely regular requests use +5351|Supplier#000005351|pAo,QIO2hoVsmTzF90g|10|20-126-846-3080|-108.05|jole: requests wake final requests. slyly fluf +5352|Supplier#000005352|PP9gHTn946hXqUF5E7idIPLkhnN|13|23-557-756-7951|7880.20|egular frays. final instructions sleep a +5353|Supplier#000005353|USvfE3GdW iEy3VNE6,SKQ3HsUYg RpLHzI6cl|4|14-238-652-1315|607.70|ons are blithely express packages. ru +5354|Supplier#000005354|o UKmH2XRXoFAh,R,|17|27-414-983-8798|3493.69|ms. regular, final Tiresias cajole slyly about the regular req +5355|Supplier#000005355|4PjyW,Ua1KgEZmsEv1tmtKoOvkziBKzSRhLN N|12|22-185-107-2267|9491.33|egular courts around the slyly even ide +5356|Supplier#000005356|dpdqrSpkO,PRgOYT9Dfx2iTUewB5G1Et|1|11-157-573-1034|7806.95|ironic accounts haggle inside the slyly ironic +5357|Supplier#000005357|Ci21sYQz0kD8F4vyUWk|13|23-717-663-7315|1429.93|into beans detect blithely across the foxes. unusual foxes nag furiously according to the s +5358|Supplier#000005358|FKVTzD1RaT1syMCClDa2VQ0CQ2|23|33-347-892-8901|6838.81| dependencies beyond the final, pending instructions wake slyly packages. carefully even excuses +5359|Supplier#000005359|QKuHYh,vZGiwu2FWEJoLDx04|23|33-429-790-6131|9938.53|uriously regular requests hag +5360|Supplier#000005360|NSPpFb,zb lAvnows2C1dk6ONz3x,8|6|16-489-531-3312|-82.72|g the requests. quickly ironic instructions use. finally final depo +5361|Supplier#000005361|qjOoazvE8Saer80Mtu iOnXB 8reT|15|25-429-333-8879|4686.04| pending ideas nag quickly according to the unu +5362|Supplier#000005362|VY0I5sfYoL|21|31-351-484-8312|7302.11|s are final, express packages; carefully regular +5363|Supplier#000005363|1gxHBI2PfQ|15|25-275-532-3176|8547.62|s use blithely slow accounts. excuses detect carefully above the +5364|Supplier#000005364|JQv2kRDiSrNNy,kXc2aWl|1|11-883-386-6571|8871.48| forges; carefully final accounts mold? iro +5365|Supplier#000005365|crrHU5sZl2 BY,oD ma5GbFI|24|34-647-376-3051|7008.02| ruthless requests. quickly special requests are f +5366|Supplier#000005366|xZ1YlzN4ielvPCH3se9|3|13-555-206-2621|4757.40|quickly final pinto beans haggle. deposits affix carefully ironic, +5367|Supplier#000005367|K66v dDhXxYH|5|15-531-938-1725|7620.61|y pending theodolites. express deposits +5368|Supplier#000005368|DQfVi W7lZxCFlt04JF XzrBeWTRfe|23|33-296-743-8157|4567.58| packages are slyly. even, final packages detect slyly. dependencies above the even, +5369|Supplier#000005369|3ar7EUFFHs1jyUlXhHtTOgo|0|10-597-638-1864|7258.51|xes hang carefully against the carefully ironic packages. blithely eve +5370|Supplier#000005370|CXjmrAqgg0M|6|16-582-221-6614|1745.32|ly on the blithely unusual foxes. furiously ironic requests boos +5371|Supplier#000005371|9iL4ZfNEziWz5NgNRoihE oY|22|32-499-859-7477|5474.69|le furiously among the enticing packages. slow dugouts a +5372|Supplier#000005372|FRt4 jmWWg2SquEIXBvOMOeIMzGWbt|5|15-326-890-9087|7078.93|ntegrate. silent, pending foxes are furiously abou +5373|Supplier#000005373|7D2MEFoNDdimC7izK8JdTAVUeq|19|29-666-138-9831|8663.63|ages. quickly ironic dependencies cajole evenly alongside of the fluffily unusual escapades +5374|Supplier#000005374|4tdBx47anUQt4wLdBNPQL7cjVh,|22|32-230-167-4170|648.33|n, regular accounts. requ +5375|Supplier#000005375|t1HqO 9KuAfjQKFfpbmSpwfaxZ52J8oGGCLN89|2|12-821-881-7903|4465.61|. even theodolites along the express, regular instructions must sleep according to the +5376|Supplier#000005376|TjFwL3P67hEvcexz5|20|30-721-836-9541|9501.01|furiously bold accounts alongside of the quickly final req +5377|Supplier#000005377|FpSTvJyMseoTimB0|11|21-214-877-8753|5472.49|of the pending, regular requests. ironic, final courts around the carefully ironic deposits w +5378|Supplier#000005378|5lP48kx9aElsaaDUtkX4AFOyXagLIowxxqn|16|26-772-307-7260|3096.20|side of the even deposits. evenly regular foxes nag according to the unus +5379|Supplier#000005379|9V7hKSGNOUxDp3gLft0I,rg0j5xzs|21|31-286-144-6185|5736.46|yly regular requests. carefully ironic courts use carefully alongsi +5380|Supplier#000005380|owZmt SBTXg57CQ9kMPM7C|5|15-834-438-9601|6747.62|e after the slyly even packages. slyly even accounts sle +5381|Supplier#000005381|EnxVlmR3hWpZM|14|24-780-658-4283|6214.92| express requests along the quickly eve +5382|Supplier#000005382|JO3p0fKZRP,MzliJUpCr6h139CaH1TrSlQ3Jk|8|18-856-463-7791|7131.81|ly furiously even deposits; final packages promise quickly. reg +5383|Supplier#000005383|CMDEPvxWiyYmldpBtYzXPr x7yY,B0oBOB1kAuT|20|30-817-692-6622|6295.49| the foxes. quickly ironic packages cajole enticingly express idea +5384|Supplier#000005384|8UfrO5ayFT0o4LEnVUceS7bX|14|24-606-992-7118|8372.64|quests haggle slyly. regularly regular accounts use evenly. quickly ironic notornis against the +5385|Supplier#000005385|SKM0Z9dd8Zm|6|16-273-318-4085|5045.64|y above the unusual, ironic excuses. ideas dazzle carefully furiously even +5386|Supplier#000005386|Ub6AAfHpWLWP|3|13-234-615-4933|1152.89|ake carefully. bold foxes among the +5387|Supplier#000005387|Wr8XJY,rbMMdS|8|18-679-129-8879|3535.05|s sleep carefully carefully even deposits. furiously final requests maintain bl +5388|Supplier#000005388|RBpponq jRQR3,cEKMd4lYj2ZNhe7hZYrR,v J|23|33-299-403-5591|647.84|es shall are beneath the furiously final requests. +5389|Supplier#000005389|9ahcfntXIusWZvO8pD6kxBqppWkSoxSf|11|21-299-169-1876|7132.09|latelets print slyly across the requests; regular deposits along the final requests det +5390|Supplier#000005390|OUz8UHJscSLiDcBk6jEZrGum|20|30-215-904-9953|64.20|oost. bold packages haggle carefully furiously regular theodolites. furiously even the +5391|Supplier#000005391|BfIsR LpIHomv77D0EU,T4x0VyZ4|18|28-756-465-8149|9925.41| special deposits are above the +5392|Supplier#000005392|80rNQXvYgc8oa6Vud3f|0|10-603-851-8821|-564.27|s sleep. unusual requests nag quickl +5393|Supplier#000005393|YlFLryTC,7xNF2l,vRNKRP66jNPtFDB6fZC|15|25-830-731-1125|7182.06|ously unusual somas. quickly regular pinto be +5394|Supplier#000005394|FxETfUU3xA7Jk8dC4lU4dxEcGgNdG69|0|10-962-292-9745|4006.02| along the final sauternes. even excuses nag fluffily according to the ironic theod +5395|Supplier#000005395|fV2eFa98uevWqH9vqUjLqvnaUxjRuhGvY6Xfd8|10|20-555-563-7932|7107.57|posits affix carefully unusual decoys. bold packa +5396|Supplier#000005396|Mq3JuBQD3gwDepGWYFhJ|5|15-910-110-8993|5981.29|enly silent pearls! silent theodolites ought to mold careful +5397|Supplier#000005397|0B5FAZjYyf5o3jAZeArJskWed|2|12-948-136-8144|7482.56|s foxes wake blithely blithely final i +5398|Supplier#000005398|2Nf6B5RqqNb,J,Sjhfc2ui3Z|17|27-138-733-4010|5377.05|bout the fluffily final packages. blithely regular theodolites sleep slyly against t +5399|Supplier#000005399|duTHo0k0yN5I100tQw,6rqM7h,4e1gF|17|27-797-520-8028|5033.53| pending, even accounts are a +5400|Supplier#000005400|EFGyKWf8UXi5bAZMOl1klG96HZJgn3|23|33-455-823-8128|6599.90| platelets haggle slyly? furiously idle deposits sleep alway +5401|Supplier#000005401|eEOlCEAaIfVexStlrgTuzwQx7vjPF6ZT dm|21|31-591-611-2096|9634.51|ding to the pending braids +5402|Supplier#000005402|OUbmvNUOfWJKa7QI2UmITdNtmWn|11|21-647-272-3176|5264.54|he enticing requests wake against the carefully spe +5403|Supplier#000005403|ofvHBqLW,HTk|9|19-375-600-5747|6183.09|arefully regular accounts cajole quickly iron +5404|Supplier#000005404|UTlIlGBbfRTemLc0GquHExfJCc aKwT7y,|0|10-232-201-7028|298.34|ost furiously final requests. enticingly regular requests hinder carefully slyly +5405|Supplier#000005405|bELF8cmaclvh7MHsX,Hlj6ncsPqO5NaM52iO|24|34-803-485-7225|7061.25|y ironic instructions kindle along the final packages. care +5406|Supplier#000005406|j7 gYF5RW8DC5UrjKC|22|32-626-152-4621|8407.04|r the blithely regular packages. slyly ironic theodoli +5407|Supplier#000005407|nVVnyfMNKSe9vYCW6GY3vMy3UmNJ|23|33-487-694-4613|197.15|s haggle blithely about the asy +5408|Supplier#000005408|8FmjKlQdz3vuG5l 3fs6zE|24|34-682-438-5340|391.51|o beans eat above the slyly regular accounts. +5409|Supplier#000005409|oM86LIMrErzimY|22|32-825-711-7980|41.97|the pinto beans. silent pinto beans are carefully. final, ironic accounts haggle blithel +5410|Supplier#000005410|I9psERm0hvL|23|33-670-449-9130|-602.83|instructions sleep carefully final, expre +5411|Supplier#000005411|f6zhj5JRnAr|1|11-541-551-7273|8153.24|es. furiously pending dependenc +5412|Supplier#000005412|m7tmUuhMSSk9wHmabFI0YEjum,XHWL|14|24-227-500-4776|2597.46|silent accounts use slyly. +5413|Supplier#000005413|Hh5byDXBmcr|20|30-736-594-6426|5528.52|r requests. furiously express packages +5414|Supplier#000005414|yDLmG4bxYA1iviZ71PkEtzcDoB0KT|22|32-204-830-9105|9837.53|y through the fluffily ruthless platelets; furiously si +5415|Supplier#000005415|9YztwN5cdOf|12|22-899-505-6203|7418.83| regular requests sleep slyly unusual foxes. blithely even requests according to +5416|Supplier#000005416|SUSrSY8WQTcUEYo|13|23-831-179-4262|1353.49|to beans about the carefully even packages use even instructions. carefully ca +5417|Supplier#000005417|umpyUJfpKS6DJe,NVwRPj6FZnvlqx7J8XanhkkRU|4|14-325-482-8276|9067.92|ter the carefully bold ideas affix furiously about the quickly regular reques +5418|Supplier#000005418|eLGAV8u15ZX2e5pJa GEg8VBKQX9Gi1|10|20-897-164-8136|8553.36|detect carefully ironic theodolites. silently regular packages against +5419|Supplier#000005419|c2j78nCvnPSImK5HbxRmqW247KLVsS|17|27-922-798-7774|6440.94|ickly special requests. r +5420|Supplier#000005420|D1k07tVPazenvdL9Y9CBqimCPcf|9|19-282-185-5705|-65.61| platelets according to the quickly regular ideas sleep even, silent pack +5421|Supplier#000005421|j7T9Xp32K5Iy642n|11|21-267-532-1889|6249.49|ts. express, even dolphins unwind furiously. ironic accounts around the blith +5422|Supplier#000005422|BMQnJ0eNFR bTEt|7|17-816-161-4257|5177.31|express requests affix quickly unusual pinto beans. quickl +5423|Supplier#000005423|eQ7MhHgC,Vg5dvnmmq 38Ht3fu|12|22-339-826-7897|3051.27|al dolphins. carefully even requests wake about the thinly ironic theodolites. packages +5424|Supplier#000005424|eJ44,Ds5P6Ljs2ohg5oEYMMbOAHN|18|28-249-524-3207|9593.18| dependencies affix blithely furiou +5425|Supplier#000005425|5ewiFdjebxHWjo|7|17-713-403-7695|3606.16| ideas. final sentiments wake slyly final courts. ironic accounts against +5426|Supplier#000005426|9Dz2OVT1q sb4BK71ljQ1XjPBYRPvO|3|13-448-787-4918|7712.02| fluffily slyly regular packages. blithely final pl +5427|Supplier#000005427|Wrx4glYCDjvSmlh|18|28-477-481-3380|920.41|e the blithely stealthy accounts: slyly even requests will maintain furiously. slyly special th +5428|Supplier#000005428|A 9bt7EzZq1uY1yQniQLTXHGUb9BE3|24|34-187-702-1052|1220.95|ffily final accounts cajole blithely. express asymptotes h +5429|Supplier#000005429|zMag7VEnYBZaYeZbsby|21|31-235-389-6713|3413.45|fily special packages. attainments are. quickly even packages a +5430|Supplier#000005430|fVZ,20XHxLY0ZvyFmGmUFu8|17|27-563-688-7196|2808.73|lithely express dolphins sleep slyly across the carefully regular de +5431|Supplier#000005431|e0c5oEacjV06Xv5HJ7u6vGM,f|24|34-875-249-9826|3273.18|theodolites. foxes are: furiousl +5432|Supplier#000005432|rYeS220yURgSOfZS3LXE,K2MgcA39,gIobzD9i5|9|19-235-314-4222|-739.72|ly regular packages use slyly fluffily final pac +5433|Supplier#000005433|ZdqCGFC7mV0WZF6p7RM1BRcV8Q0h5HsQK|16|26-300-172-7653|6462.54|thely bold platelets. slyly special instructions haggle quickly against the furiou +5434|Supplier#000005434|KfozB989PkBnQ89j83SJriSmVAx4ha|19|29-849-211-1064|6.23|. regular asymptotes nag sly +5435|Supplier#000005435|jO8nrrSl8pq0aBthAKdfiK|20|30-424-971-5752|3601.66|nal requests. special packages haggle. furiously bold pinto b +5436|Supplier#000005436|3HiusYZGYmHItExgX5VfNCdJwkW8W7R|20|30-453-280-6340|9026.80|. blithely unusual requests +5437|Supplier#000005437|pyGkLNKhQ6UVm7hg|4|14-907-539-4048|-521.34|leep. deposits against the quickly silent theodolites wake slyly idly fin +5438|Supplier#000005438|MFpEZNzGjFZN0kmhy7txpsPAA4OaKJ|16|26-355-861-2379|2728.45|the requests boost slyly along the +5439|Supplier#000005439|4m1nHac8yhuOHkpETX1b1CVrIeepXYkV|5|15-916-955-8476|6966.67|s the slyly pending dugouts haggle according to the slyly ironic accounts. +5440|Supplier#000005440|hJm0W8Dffq7XJgpv,CmUhog|11|21-818-160-7547|2928.64|eas. bold deposits hinder carefully among the bold instructions. slyly special fo +5441|Supplier#000005441|jPjn15E8lggZpguymEkBK2pSSYOshO4ZeAGzybZZ|6|16-737-279-8103|5703.42|have to integrate carefully blit +5442|Supplier#000005442|zSx6Fqr1Do4|0|10-844-454-1155|1417.39|oxes are thinly. blithely ironic instructions af +5443|Supplier#000005443|Y3SVxa7HhDryanY3rlgeNaLEz1jsDErV2|6|16-230-820-1059|5063.48|uches. silent, final deposits from the foxes cajole furiously after the requests +5444|Supplier#000005444|ttSBo mkJ68fIyI1oulQslNrq|13|23-250-479-3631|3941.73|en excuses eat quickly at the carefully final asymptotes. unusual dependencie +5445|Supplier#000005445|1mlE0Eu6 BwElxidQ3XLJ66iHBMd14OOm|9|19-641-543-4373|4563.74|le even deposits; furiously pending requests detect quickly +5446|Supplier#000005446|Nusoq0ckv9SwnJZV8Rw2dUqE,V0ylm Bon|4|14-468-853-6477|8846.35|deposits. ironic, bold ideas wake. fluffily ironic deposits must have to sleep furiously pending +5447|Supplier#000005447|tfUbLfD8A3c,WXcB,cIm9Xc|10|20-658-969-9153|2675.92|ymptotes cajole regular theodolites. regular deposits d +5448|Supplier#000005448|H6zDBulJqyBX6|13|23-795-250-6901|4110.25|losely special packages promise. quickly regular requests cajole furi +5449|Supplier#000005449|fhc8lUuZdqWUujcVaWogowEq1WVL9Y8m1efwCl3G|21|31-787-239-2170|9988.93|sts haggle fluffily. carefully bold ideas among the deposits haggle fluffily +5450|Supplier#000005450|hEt0 DNXDQ8|12|22-413-669-1577|48.48|e express accounts. furiously ironic packages a +5451|Supplier#000005451|JECYQtqXS8i13qrBf2flrHbmvuzYoVucGcMWW|23|33-215-882-1837|9795.76|jole carefully among the regular instruc +5452|Supplier#000005452|4lcA3vhI8skq9ENOusBjH4o1lEBQosW|11|21-701-262-4306|9452.69|slyly. quickly regular dependencies cajole furiously along the slyly final deposits. brave +5453|Supplier#000005453|Qt4FJJAPVEKAs8|19|29-817-942-7953|7445.80|ut the quickly unusual packages. bold accounts are blithely? furiously ironic foxes wa +5454|Supplier#000005454|NJ4sMCJou3j4PKwa|16|26-407-269-6312|3554.47|rs. regular ideas eat about the e +5455|Supplier#000005455|1pPmFeEkoes|8|18-723-651-6014|8032.13|. regular, final packages boost ironic deposits. ironic excuse +5456|Supplier#000005456|1q8Nemk YewQmca7TU|10|20-147-957-7762|2030.92|the regular deposits. regular, +5457|Supplier#000005457|3gTI SbYeckJOAGls6NlUPQUnbfhBmMXATrM|6|16-421-469-8087|9690.12| after the regular deposits! fluffily busy courts wake carefully +5458|Supplier#000005458|d39AbpvBG6mAU8 |24|34-955-330-4162|4926.44|nts. regular requests haggle alongside of the dependencies. fluffily bold packages +5459|Supplier#000005459|xaHsWujWhGN|11|21-642-305-7144|-350.78|pending excuses. instructions impress slyly after the blithely ironi +5460|Supplier#000005460|QMA17qwZCNqLKfwIVf67Ha,YU9pdn5ci3ZhTJrg|12|22-183-618-7004|9287.14|s was alongside of the ideas. slyly regular theodolites cajole carefully. deposits sleep blithely a +5461|Supplier#000005461|8jG,GyID3v9TbTOkC1SCG3V8zHin6TCgHefJd|19|29-845-837-8167|7974.97|pecial ideas sleep slyly final deposits. silent requests are against +5462|Supplier#000005462|l1pDn,uF2LneHyb6bGb3UnH|6|16-803-926-9943|9298.02| even theodolites. furiously ironic deposits haggle. pa +5463|Supplier#000005463|9LO2cBtz QudJsiywfGPcBAQ76Bd |16|26-425-984-3777|3964.13|ely express requests. deposits hang slyly regu +5464|Supplier#000005464|PL9KF5XLH468IC7HD Wb0gSh|14|24-985-855-4285|7134.07| according to the carefull +5465|Supplier#000005465|63cYZenZBRZ613Q1FaoG0,smnC5zl9|3|13-279-696-9803|340.93|the carefully even braids. regular instructions cajole furiously theodolites. pinto bea +5466|Supplier#000005466|UKqdYJokGezKzRhnJn |23|33-435-930-9900|8026.83|al packages sleep carefully. packages use carefully requ +5467|Supplier#000005467|NfJBlcUJVG8lGL|0|10-403-707-6568|1059.43|ve the pinto beans. pending foxes detect carefully. ironic accounts ha +5468|Supplier#000005468|0hLPUma0KcXFojba3LOfWg|6|16-313-737-2837|5753.09|ades wake slyly above the furiously regular ideas. fluffily speci +5469|Supplier#000005469|JvZz sITLz9qzXVPzzROMjhPWiwtIrxc7v,4|20|30-504-894-1149|170.82|es snooze. carefully express deposits boost af +5470|Supplier#000005470|dNQ2q4BhVvkvvy0HSjWx864vuPmxHKggIGkAY2Ux|19|29-743-776-5345|8978.98|ecial foxes about the express, pending Tiresias sleep furiously +5471|Supplier#000005471| 6q15XuKIPr7AgoDJX2F2q|19|29-637-173-8729|9761.80|posits. quickly even foxes +5472|Supplier#000005472|LlyLSmvY9GFvMN4QhHzMokW0k5d|21|31-595-192-5906|-278.27|ing dinos use. deposits are furiously final, final pinto beans. +5473|Supplier#000005473|0kypwT7K1ixxUY5LBcd2v8iEnO|5|15-130-672-9099|7103.32|quests. unusual pinto beans among the slyly pending deposits sleep idly slyly spe +5474|Supplier#000005474|VUqApfspXM073YGTCyWH1Xk18uPoxkNlSXb|15|25-250-807-4845|455.77|ong the furiously fluffy packages. pains across the furio +5475|Supplier#000005475|1Rgs1ZXkbAt3kG|13|23-596-337-2882|2112.13|sits nag. bold requests are ironic theodolites. deposits nag slyly special pinto beans +5476|Supplier#000005476|VFWyOOb0 q|7|17-802-655-8002|-397.91|sly bold accounts. quickly express packages wake blithely. furiously regular +5477|Supplier#000005477|fOzXmCgwtUn E2O8R8NXiDYLibNr8ThjiTJXJZ|11|21-195-969-7717|9214.58|ounts against the express ideas use slyly after the +5478|Supplier#000005478|tk1HQxRxeWlaO|3|13-377-126-4444|9440.97|ts use carefully furiously even theodolites. ironic, even de +5479|Supplier#000005479|01UuytFPeu6tQC6YsE679vJKyuTNozBjaq4By|4|14-197-899-2987|8434.37|asymptotes above the quickly ironi +5480|Supplier#000005480|p5nxeK27Qq5QrcMGlfwTQ,3sqfFIt,38J1kDM1Tf|24|34-491-455-9061|8541.08|ate fluffily across the slyly bold packa +5481|Supplier#000005481|3yFBDRhiG1LT7deOr4iojpsa|6|16-163-758-7030|9023.31|y ironic excuses sleep ironic foxes? carefully pending requests haggle carefully +5482|Supplier#000005482|VQESGLA8d WzJaQ|1|11-156-303-6195|6914.94|ideas about the express dugouts unwind furiously alongside of the warhorses. qu +5483|Supplier#000005483|bx5Qn,U6XQL|1|11-754-883-4135|2428.15|its. special deposits run. ironic dolphins cajole fur +5484|Supplier#000005484|saFdOR qW7AFY,3asPqiiAa11Mo22pCoN0BtPrKo|3|13-696-138-7821|8813.77|ever bold theodolites are furiously regular platelets. care +5485|Supplier#000005485|lPUyDH6sYA6ktvEc25b4hvYmUKrtIXVcuc6t|3|13-407-685-3623|5157.33|uriously special theodolites. excuses haggle carefully alongside of the quickly express ideas. pend +5486|Supplier#000005486|IljonN5AZJzFgZOFPkazs6fyHQSwGily|4|14-479-224-2584|8192.84| cajole furiously behind the blithely special p +5487|Supplier#000005487|nEoE4Yph7gDXNGApRdggiY4Ai|17|27-700-386-2510|8071.83|regular ideas. thin theodolites after the fluffily pending theodolites boost bold accounts. furi +5488|Supplier#000005488|LILwiXVXop3gmBCNhft7bF2nfC7dWGBE8N8A|6|16-580-434-8535|9269.10|sits boost blithely. carefully special courts above the pe +5489|Supplier#000005489|XSNO3NJWPmIPRMBou7PJodUmTF6|0|10-770-629-7558|1838.95|s. ironic pinto beans boost +5490|Supplier#000005490|gQow2jW3p7935m6dVU7gI9yTbwtwgGYjyXW Hzm|13|23-258-764-5430|6188.10|te. quickly unusual Tiresias boost blithely. busy foxes haggle s +5491|Supplier#000005491|ZE yuRg689|15|25-675-752-8437|7481.41|ffix along the furiously busy deposi +5492|Supplier#000005492|k66hEgXwo8FrHNt7Ef|17|27-305-245-2548|8101.82|ets sleep slyly ironic requests. carefully even p +5493|Supplier#000005493|mXdoSJzxQTmRF,KhCydUsjQXiE12iAb,W|19|29-168-352-1249|7402.78|yly pending instructions are carefully unu +5494|Supplier#000005494|u5ylCpj7F7mjMz4uXcDdE,n|22|32-569-869-5188|6498.51|l requests haggle according to the furiously unusual inst +5495|Supplier#000005495|czBFvGuvMHM6K1UgsnUR5rWVi|15|25-662-731-5087|-594.79|pendencies wake carefully. fluffily regular ideas poach slyly. regular deposits sleep. bol +5496|Supplier#000005496|ZBA2jz7rycLep,|11|21-547-660-9780|277.28|al packages use among the special, express platelets. slyly bold accounts boost blithely special T +5497|Supplier#000005497|jwJ,6lAvd9pY1xRNsw9sW4AD4a|16|26-882-575-9856|5237.29| foxes wake furiously. pending theodolites are furiously deposits. slyly express asymptotes sle +5498|Supplier#000005498| 9aW1wwnBJJPnCx,nox0MA48Y0zpI1IeVfYZ|13|23-774-653-1950|-816.97|jole fluffily. express, express requests nag furious +5499|Supplier#000005499|qiXCxfD290QPfh6SjTire5B1W2iy|19|29-475-835-9164|6020.12|ongside of the ideas. quickly final packages wake s +5500|Supplier#000005500|sjHZWxiy cCJFDecrNHp|12|22-790-659-5767|178.16|thely final accounts. ironic, pend +5501|Supplier#000005501|cGipjP0aSm,,nBJIlzDvJmloisH,fPZ|12|22-303-395-3383|190.22|posits. fluffily pending ideas wake. blithely bold depths wake against the instruct +5502|Supplier#000005502|X6eGK5l,5,|17|27-973-499-7535|4998.95|nts cajole fluffily. waters kindle. slyly blithe ideas cajole slyly fi +5503|Supplier#000005503|cLr3quhaIWbhK |16|26-687-835-3855|8313.52|ke among the fluffily unusual theodolites. furiously final notornis are sometimes ironically even +5504|Supplier#000005504|dRTJ42juiQY|16|26-625-847-8060|1408.70| final Tiresias wake blithely against the express, ironic pinto bea +5505|Supplier#000005505|d2sbjG43KwMPX|3|13-531-190-6523|2023.40|refully against the slyly even deposits. slyly sly instructions sleep clos +5506|Supplier#000005506|On f5ypzoWgB|3|13-542-572-5106|4288.82|furiously pending ideas wake slyly. final, final asymptotes haggle. slyly final ideas are f +5507|Supplier#000005507|v4NTIK3S qTuP|22|32-416-130-1867|8229.16|quickly brave deposits integrate carefully acro +5508|Supplier#000005508|vOoE35mRLE8atfruKcNS8r3LvJf7j4zZJjSw7Glv|2|12-924-457-8744|2550.06|fter the even accounts. dugouts affix blithely unusual pin +5509|Supplier#000005509|40BShwTmsunjcYDsaIEtBTejO7EK|11|21-392-423-4088|894.74|ingly pending foxes sublate quickly regular requests. even packages boost carefully ironic accoun +5510|Supplier#000005510|f 0PMKZD0h|24|34-332-810-7526|9817.53| warhorses. furiously regular packages boost above the quickly unusual acco +5511|Supplier#000005511|mD5fEqSyfUHW1dxltuR 7ZBTtZmY1Y|16|26-956-868-8243|1262.20|nusual asymptotes. slyly pending accounts are against the furiou +5512|Supplier#000005512|jbQhb7WAzSYp5XzjAvKSEfW|4|14-324-488-5899|3716.26|across the blithely regular requests. furiously silent foxes use furiously regu +5513|Supplier#000005513|IgESXIfvPTpwiUvo3S|2|12-142-281-4302|-854.33|regular packages. silent, unusual dependencies mold blithely across t +5514|Supplier#000005514|pLw yST2myHRwgbkIQztflbIJnKN|5|15-141-950-6425|5275.67|unts haggle after the ideas-- carefully special platelets nag quickly about the furiously unu +5515|Supplier#000005515|hwKbFCz1DM,QGL5CTE9PaJemjvDUONR9RzrR|4|14-708-611-4097|6145.83|thely pending accounts. packages wake furiously. silent dolphins boost furio +5516|Supplier#000005516|XsN99Ks9wEvcohU6jRD2MeebQFf76mD8vovuY|3|13-303-283-9409|974.02|s detect slyly stealthy excuses. even foxes are fluff +5517|Supplier#000005517|3JXFFMBtKPMRJ7sg|22|32-293-825-6891|349.82|s nag carefully blithely pending requests. pen +5518|Supplier#000005518|DZpIFpba1VZ5Qn6 BLSrxFrXfytT|0|10-917-125-9536|3310.19|ess theodolites across the slyly regular platelets sublate along the quick +5519|Supplier#000005519|cR9o8dnO1,DhgnNq34OTMeo|12|22-158-327-5924|5366.35|furiously about the carefully even foxes. pending request +5520|Supplier#000005520|wMiks0Z1elXmYBnpgfWSG4pzAY1pdMyodDO3Hyi|22|32-967-113-4478|1200.01|ounts. carefully pending packages bo +5521|Supplier#000005521|XgmOG8jUrUrcd cDpQ4T87szY YgGAE,|1|11-737-993-5777|3457.73|uickly unusual pinto beans. carefully express Tiresias along the final theodol +5522|Supplier#000005522|4ZYoGSuNUEUc3|17|27-941-491-8915|-898.35|he carefully ironic packages. th +5523|Supplier#000005523|loM,UvBG0qY86VDOroyLneGZkMx69nKO9|23|33-179-120-1158|2398.22| to the final accounts haggle q +5524|Supplier#000005524|yQUF2VQmVgsJy4dAifWzYT|11|21-709-399-8312|878.57|l, even excuses. regular, final accounts sleep among +5525|Supplier#000005525|cu,artidyzi3DfH90OuWWJCzky|14|24-971-946-3756|2010.81|leep carefully slyly special warthogs. final deposits are reg +5526|Supplier#000005526|OwCmHN8lyC,cNsDN0I3vB3Ax5UPe|22|32-845-855-9220|1574.34| wake blithely. regular asymptotes haggle. packages cajole ironically across the sly +5527|Supplier#000005527|ruPO o9up I1IWsc5nI2hhf|16|26-203-161-4959|2829.32|l pinto beans. carefully ironic instructions according to the slyly ironic +5528|Supplier#000005528|3q3AXP2IxBsMXWsQ nk3Pt6WphMWu|12|22-867-532-8065|8313.47|sts cajole slyly. final requests are +5529|Supplier#000005529|6oBPeuCY9ncw4kUjuuF5Vn Dl8BZCgHtcCN3|12|22-980-990-1520|497.87|s are. quickly ironic instructions wake above the blithely final courts. furiously ironic ideas +5530|Supplier#000005530|2f2CgHMQWVrfbCTjHZTSYwynknmTDMghQAB|9|19-903-770-1219|5655.62|oggedly ironic foxes are blithely express platelets. foxes are quickly. blith +5531|Supplier#000005531|n4kVkDr3HtHr8UrsNrlT|23|33-483-665-6523|941.43|s? blithely final waters cajole slyly. furiously i +5532|Supplier#000005532|ENMzMnfEDFcQ,AnrfCtWRJ0|1|11-484-785-7262|2526.81|y regular instructions boost again +5533|Supplier#000005533|oWOkKrUHg2OypZKVqZAGc89G1941FDBCAYMn|16|26-158-170-5358|6098.97|e. furiously regular theodolites to the theodolites grow above the carefully final foxe +5534|Supplier#000005534|zFQ1nZQPOzYN7VZEGjlhXTAzN,TaOwpDU|19|29-221-648-7213|5424.63|ong the express requests: slyly eve +5535|Supplier#000005535|2vNZzKcNWYq6JWu1ij7llEsoLtBBa |14|24-481-522-9978|1765.10|express deposits. blithely eve +5536|Supplier#000005536|Nzo9tGkpgbHT,EZ4D,77MYKl4ah1C|3|13-501-681-3157|7986.44|long the even theodolites. blithely express theodolites try to solve quickly. slyly reg +5537|Supplier#000005537|rTFxbwb34STFZu8WyEr,ONh|2|12-337-574-3631|4200.55|ions cajole. ironically even platelets sleep accounts. +5538|Supplier#000005538|TBbJ4x2cFWVrB,3fxD6OHfKVilf,c,|1|11-123-902-2596|9586.44|ular deposits boost-- ironic deposits haggle furiously slyly special theodoli +5539|Supplier#000005539|Y2eQnMeDWoJl|17|27-850-384-5675|303.89|ven requests. pending requests breach. u +5540|Supplier#000005540|4GO58KPK4IPW0GUJ7r2fip4w5Yd9yD2uJyuAq|5|15-922-885-1259|1468.45|at carefully. slyly regular packages against the pending, express accounts +5541|Supplier#000005541|Y0XWrU5rdpSD 5k8Byo3x28AxIEbysGvOx|18|28-166-498-1953|5227.04|gular, bold patterns wake quickly even, even +5542|Supplier#000005542|lM7Z7zbCFaVB7nXU1Ver|3|13-198-773-9673|2847.59| fluffily. carefully final instructions sle +5543|Supplier#000005543|NTw0R,YuX0KBzDbVhdu7oynMdAA2svLkpG|12|22-677-629-3497|7445.10|ccounts. blithely regular theodolites +5544|Supplier#000005544|nGkP19y2yjhei7gPHs2EBsWZ1DXdIPVhE|7|17-790-588-3436|2857.85|usual pinto beans use express requests. ideas wake quickly. furiously ironic deposits bo +5545|Supplier#000005545|a9vTkE8sVY|0|10-406-267-1449|3834.69|vely unusual packages haggle fluff +5546|Supplier#000005546|Jcu IfutR4vTFQUg8ngY8sSEjVtUtnUj|9|19-922-350-7889|9542.82| carefully. fluffily bold theodolites detect fluffily whithout the fluffi +5547|Supplier#000005547|lve22o11GmyI|24|34-913-837-7584|4188.21|ely. express requests about the permanently spec +5548|Supplier#000005548|C7CwN6SZ0ikFPW|21|31-470-117-2737|298.62|ess platelets about the slyly silent dinos haggle furiously regular pa +5549|Supplier#000005549|BJ3aqeHYo7psALHM12UaVYr37xlsAnd|0|10-487-269-2604|860.86|st the regular platelets. quickly bold packages against the theodolites use slyly s +5550|Supplier#000005550|c5FSo32Sb0CrRwFpK9RoCVTSPEY|11|21-738-938-7020|1402.54| ironic dolphins sleep careful +5551|Supplier#000005551|St6sF,B8BUYpOudyVydkUsSbM3r|11|21-572-794-1936|8472.63| bold ideas. express, even fo +5552|Supplier#000005552|RhZfPwwYGelewAVIxi|11|21-107-686-9785|2630.06|s are about the asymptotes? ironic packages dazzle fluff +5553|Supplier#000005553|QuSdPQkZ AOLcGgH|2|12-569-854-2981|3410.75|ly express pinto beans are. thin accoun +5554|Supplier#000005554|v IPk1I4GuXp6|17|27-355-634-8667|1526.73| are blithely according to the slyly express requests. regular, +5555|Supplier#000005555|OVDFyVFCvGZSMF|15|25-372-584-7324|5781.63|s sleep carefully ironic acc +5556|Supplier#000005556|LaxP c8bNr1Yh8lFUHyMXBoYf1Pn91nJoc4|13|23-206-751-9818|6852.59|s. final requests sleep. furiously ironic ideas cajole with the blithely special requests. unusual a +5557|Supplier#000005557|3ShyjCpd0fgHZZ,rqk1PAc n6kixym9Iej3FmHV|8|18-228-535-3302|-478.34|se carefully against the even, regular ideas. final, regular packages doubt c +5558|Supplier#000005558|HOzfI2Zn4hXuxNjS32STTh3ZW Y|5|15-289-693-5194|-462.40|ress requests. slyly silen +5559|Supplier#000005559|cMthDmitw2YdTFpfXkhbZqkiOqNOLfAz|22|32-328-469-5153|3286.38|ironic ideas haggle slyly quick pinto beans. blithely silent deposits ha +5560|Supplier#000005560|z0Jh6VLf9nyWNaYCkI4ub8Z|6|16-263-437-5752|1157.24|ainst the ideas cajole requests +5561|Supplier#000005561|ZCwI36fui PSc2KAii2Gz|12|22-448-682-4189|-292.95|permanent platelets cajole carefully regular instructions? excuses haggle furiously idly regular acc +5562|Supplier#000005562|5e0DbBnKhReDsJHH93wpQNzrKkqTk|22|32-121-244-8514|2280.44|ns are slyly bold deposits. furiously ironic instructi +5563|Supplier#000005563|tulcd9BHryy5hsned0b|11|21-484-244-9995|1376.72|re carefully regular accounts. slyly even realms against the carefully re +5564|Supplier#000005564|Y s2t94EuIwMYM9ZnelwxmQMA1WDM1Y3|1|11-413-101-9840|3404.65|d the express theodolites. ironic req +5565|Supplier#000005565|RPRbABmHFcRGvAvq|10|20-459-844-4016|8991.91|ual accounts play requests-- carefully ironic ideas alongside of the quickly ironic +5566|Supplier#000005566|4cY11YEBGD2vgPiyE3lp5dcMDpHYWdy|9|19-523-885-6705|7174.68|, pending courts outside the accounts cajo +5567|Supplier#000005567|cSvl3 Lhy2yXzASuOH2EUoY34uyfJGo7yNX3|9|19-984-730-4684|916.66|nts along the ruthlessly regular d +5568|Supplier#000005568|xTH35cEG,eNEbuQAL669CgWoyJM3N,izOYNfM|20|30-410-111-4244|4330.60|ounts are quickly ideas. final, careful platelets according to the slyly ironic frays detect after +5569|Supplier#000005569|Uxf0yd2cJI8WjqXeC9|18|28-541-359-5857|6472.34|ounts x-ray above the furiously special asymptotes. furiously unusual de +5570|Supplier#000005570|xxum0yuMEevd70XpXzA1c962KxbmrfawBt|16|26-322-913-9936|-367.62|, bold requests sleep after +5571|Supplier#000005571|OJ12o51CkXx0ci8w nfwme|6|16-518-538-5486|1325.38|ent packages. bold platelets wake blithely ironic forges. asymptotes us +5572|Supplier#000005572|o0VYozeSbEyqck|21|31-829-399-4904|9636.13| unusual accounts wake along the carefully express requests. c +5573|Supplier#000005573|9mTtG4DWSOhmjbA2gU5WSKuY8jwBl9s|6|16-800-523-3519|-433.11|nts. blithely regular pinto beans sublate bold, p +5574|Supplier#000005574|4QHqvBFxNsWoSBo9tLr6pLNhz1SZCloduHRf|11|21-700-670-3508|677.80|posits are. quickly final pac +5575|Supplier#000005575|LYg3J8w5srT9RDH7BoAqdD|21|31-341-491-2255|4754.05|ress foxes across the pinto +5576|Supplier#000005576|UXqU25sDkQRbz7YI8U|0|10-616-875-3742|6257.01| final requests. pending theodolites are enticingly; ironic requests nag around +5577|Supplier#000005577|,p,phwcYp02ArlwhKoYr8cxGlLJOonXGTTOrYnJv|24|34-657-825-9283|-355.87|eans after the even dolphins integrate across the accounts. Tiresias haggle at the fluffily unusual +5578|Supplier#000005578|y1UTYp317CbCwAzr,dnfo |20|30-178-841-4985|851.37|ct. quickly pending asymptotes cajole furiously regular dependencies. slyly permanen +5579|Supplier#000005579|ACVEMP4IwRf|21|31-699-410-9988|6864.48|sual, even requests. final foxes doze slyly r +5580|Supplier#000005580|PYke44DNzeI9O3pyTS|23|33-136-680-5730|1358.82|ic accounts. unusual, express deposits hinder slyly after the slyly final +5581|Supplier#000005581|UtL5rJkRV4x,RfVkCxNqSDppagiQrk5b86p|24|34-431-817-4279|-335.99|ular, bold requests cajole ruthlessly against the furiously ironic +5582|Supplier#000005582|BPeIaQUjfHtxPOD7nXA0TEkeH|22|32-258-813-7617|4863.59|l deposits. slyly express packages haggle. packages cajole slyly. blithely final accounts +5583|Supplier#000005583|kDAL,IrYELEzO3rdojo4C3S8a|20|30-559-824-3046|-720.70| sleep slyly. furiously regular accounts cajole +5584|Supplier#000005584|yVlen4zXsSVtrwi1|1|11-492-434-5408|7178.39|st. carefully regular deposits wake slyly. accounts sleep quickly even accounts. furiousl +5585|Supplier#000005585|nnEt0njMpEFeiyJMj6xIY2L,emUjy7h9Pm3YiGn|6|16-368-410-4285|8264.20|uthless deposits according to the even, regular foxes affix blithely +5586|Supplier#000005586|WWw,2v2kyQs5oX5F1ToNotTwyQ9CaEte5LxDz|17|27-329-481-4646|8468.41|ly unusual packages wake slyly. +5587|Supplier#000005587|Fb3AyX2CBKDQMn43xSw0Ao5|10|20-362-816-1839|8641.57| silent theodolites. final, bold escapades cajole even accounts. furiously even waters are? special +5588|Supplier#000005588|dUANNQEgY38ymw|11|21-922-199-3790|3900.27|carefully express requests about +5589|Supplier#000005589|i,apaHuHkorzIHy38JJbjt,iqWnz|7|17-256-591-4642|877.77|ously accounts. final, even requests sl +5590|Supplier#000005590|gvX4lIaMD5pPI7lZQl|24|34-604-388-6015|-382.57| the pending ideas sleep carefully slyly regular accounts. furiously unusual co +5591|Supplier#000005591| A39jVPWcKU82OI68FRPv5MLYkbnFhQeoH|24|34-727-116-4039|3115.96|al requests; unusual, ironic deposits integrate blithely among +5592|Supplier#000005592|LSStsnlgWDoomZw,6qCl9LsUvl6zICG,fsFG|1|11-794-377-7804|7862.97|lar pinto beans detect blithely according to the quickl +5593|Supplier#000005593|DLs4RI0zsa WbJez23,2RPdSVo6LfVxU|18|28-239-390-1803|7921.60|al theodolites haggle carefully. quickly even pearls use quickly. +5594|Supplier#000005594|woPTNCwd4CkGGS2NjUJHE4SbfR6yGOlN|23|33-607-658-4402|3465.74|o the busily unusual accounts. carefully regular excuses cajole fluffily along the package +5595|Supplier#000005595|ugLPijmMHr7dlj|9|19-300-161-7260|1734.63|tes. furiously regular accounts besides the packages grow above +5596|Supplier#000005596|D3,itcRfI 1essBcPzRRgArSMtHgHjNo8kc8|8|18-574-428-7441|1831.07|xes poach above the slyly bold packages. fluffily unusual ideas eat fluffily slyly +5597|Supplier#000005597|KKWKwyvONbDJrLaRpI1Q6qrVje3wOkCQEAw|1|11-119-455-3973|5832.73| according to the pinto beans. slyly ironic accounts wake slyly pending dolphins. pa +5598|Supplier#000005598|7dokuSwnwUdZQZy69vfAucaW5QvcNNcXUKrYD|2|12-473-143-3335|7649.51|y even packages. slyly special depths along +5599|Supplier#000005599|Sdpuqfm3LzU|13|23-128-489-6208|6386.10|ts integrate furiously around the blithely regular accounts. silently pending courts nag +5600|Supplier#000005600|aRzTJ,oyd7Ujd2PU1wQKiWdJan37bK|1|11-715-776-4145|2290.86| slyly furiously ironic deposits. fluffily special foxes according to the deposits haggle even, +5601|Supplier#000005601|BRZS844MqAV,waoD6IvM|8|18-325-313-4179|477.10|tealthy excuses; quickly final accounts use. furiously unusual requests about t +5602|Supplier#000005602|wj6nyIWETvfxVbnqm|12|22-762-619-6465|8570.23|lets. slyly bold requests mold above the asymptotes-- bold acc +5603|Supplier#000005603|tGkcxbasaGFwLD2ISdrtw3unm|0|10-108-469-7350|3849.69|nstructions after the final foxes sleep even requests. slyly bold excu +5604|Supplier#000005604|RBi1ab0lYokQsmhcPLIuKOlJ3GX0xZbYX|10|20-836-715-9065|8659.37|pecial asymptotes? carefully f +5605|Supplier#000005605|7Vj6Eil0mThqkM|3|13-624-871-8135|6111.79|ously careful warhorses. blithely even grouches grow slyly. regular accounts above th +5606|Supplier#000005606|gC9G8gEwEWysJWhV|9|19-520-105-8734|889.98|requests wake carefully. blithely express dolphins wake fluffily pending packages-- slyly unu +5607|Supplier#000005607|6L4hLpnMRj|19|29-351-566-4076|6833.75|ly across the quickly final platelets. express requests in place of the r +5608|Supplier#000005608|0VKQeRI S4B61buMvEevlRhO|8|18-402-586-1520|1874.15|t the carefully furious pinto beans. carefully special theodolites integra +5609|Supplier#000005609|t3UCx3ORSNVYmnFFqogsh2,WhSvFBv4y|23|33-841-813-1466|3800.99| idle accounts according to the furiously express foxes cajole +5610|Supplier#000005610|0hB1oH1SCbnuqzgH8pY3lsPjK HVLY3qa6UGWY|22|32-442-482-1719|4241.79|gular dependencies use carefully fluffily final packages. fluffily final deposits impress slyly +5611|Supplier#000005611|pSGcoAPtjG6R|24|34-290-191-3031|9785.34|eodolites. doggedly final accounts boost among the +5612|Supplier#000005612|W,,4BNEF2Gbr,e9|13|23-519-359-2038|-786.38|. special foxes use carefully. furiously even dependencies cajole furiously instructions. furio +5613|Supplier#000005613|kg4TuXHkAlfBxvXYA,KV7D5AhKz|20|30-131-528-8542|3299.08|y. blithely ironic accounts above +5614|Supplier#000005614|ujVcmyldzoO2g36U |19|29-361-914-1481|5614.79|quickly across the even packages. perma +5615|Supplier#000005615|X5z71gbDgAUvxBMdkYBWCl2|24|34-477-727-5576|7808.45|l instructions along the slyly regular accounts sleep furiously carefully express instructions. +5616|Supplier#000005616|H4,vwUV HK,Eyjzy ZMwopDB |7|17-826-297-3338|4667.58|nal platelets haggle blithely packages +5617|Supplier#000005617|JwWENFPcbxn2nk4AGFIZCWnza8hAxf8AL1XrczXM|24|34-781-296-5457|4955.01|final platelets boost carefully +5618|Supplier#000005618|4IKjb26vkJYiDM5Jp7dH|7|17-411-735-8302|6887.38|. slyly pending requests haggle after the blithely pending requests. furiously ironic sentiments b +5619|Supplier#000005619|UM950jE8b97cjbbBids4ZZ|20|30-479-741-6568|1245.79| accounts. regular foxes haggle after the ironic, brave +5620|Supplier#000005620|NJf5 Uf3BtaW0vO8yHGlM8NW5s1oGm,7uMo|9|19-237-806-1995|2999.92|ep boldly regular requests. express excuses are fluffily abou +5621|Supplier#000005621|nige3lp3nkh0KXMzjJr4Qebtcg|9|19-496-702-1610|4676.51|ress, special deposits. unusual +5622|Supplier#000005622|DdXOWOVt952GSfUoc|21|31-246-301-3598|1039.26|furiously. blithely unusual acco +5623|Supplier#000005623|zUY7Q10jg,65EIgm6|10|20-610-482-9710|7850.78|ake blithely from the slyly silent pinto beans. furiously regular ideas nag blithely. carefully +5624|Supplier#000005624|g3iLj,3ikH|1|11-833-440-6623|1052.14|yly bold requests are slyly. ironic, pending pinto beans after the quickly special deposits sl +5625|Supplier#000005625|azSX77 Kv1|14|24-977-560-7143|4330.78|sits. final instructions cajole furiously carefully regular +5626|Supplier#000005626|bJc AvkL85Qsf9ZcrpTH65Nse05m|13|23-499-299-4634|9468.87| fluffily about the blithely pending d +5627|Supplier#000005627|2EYf,Hay4UjnyDlGCio8OnH1zmjqGh5ka8vkB|20|30-584-484-2716|4426.09| blithely bold requests snooze blithely carefully even dependencies. platelets boost slyly bold acco +5628|Supplier#000005628|srWVudixc5irMTq|1|11-744-565-6737|5224.32|quickly. carefully ironic pinto beans across the quickly pending accounts are across the car +5629|Supplier#000005629|SmHC2j4zM5QuUKmnob|15|25-317-111-8884|1222.70| above the unusual deposits detect quickly quickly ironic orbits. express, unusual fo +5630|Supplier#000005630|s,XoCisPT9F8GjDHpG 60X 1ahQ6b|18|28-766-701-3392|-111.55|ly silent foxes. fluffy, ironic asymptotes are quickly above the slyly bold tith +5631|Supplier#000005631|14TVrjlzo2SJEBYCDgpMwTlvwSqC|3|13-115-858-1079|4114.18|mptotes. final deposits alongside of the theodolites engage furiously instru +5632|Supplier#000005632|j gihsmDXhBPq|13|23-651-637-8034|-944.43|ously final pinto beans. pending the +5633|Supplier#000005633|orUBy2X0OkilhW|2|12-784-316-3712|8943.21| regular deposits at the carefully even platelets cajo +5634|Supplier#000005634|6Oj0vGAbfaCdeHoQDk85eVEzc09RiC1EN76Te|1|11-193-504-6953|2614.11|ed accounts would wake. d +5635|Supplier#000005635|LXsy9xMjwREYKD|16|26-993-908-6978|-736.71| pinto beans according to the slyly +5636|Supplier#000005636|8x2GMcr6ytjya5BqROhmggj7vimfhR1Yz|16|26-700-491-9544|1989.41|ly final dependencies snooze +5637|Supplier#000005637|hSEAidPJkuyUbim9VKN|6|16-155-378-5936|4941.11|ckly. furiously regular ideas are quickly requests? slyly unusual packages across the notorni +5638|Supplier#000005638|CvEWD98Ed821nGR6xfEPnnWEV4eO ogDJ6DKgb,Z|1|11-790-276-9308|4182.82|ncies sleep among the slyly even requests? even, slow realms sleep furiously bold ideas. furiously +5639|Supplier#000005639|jQgtjUMoKE52lkrvwhvSsrEJQ|14|24-102-800-5206|8729.54|ronic requests according to the carefully final pinto beans cajole furiously accordin +5640|Supplier#000005640|gry8jIFw8CqsIV|15|25-310-531-7490|8509.74|s. furiously unusual dinos wake slyl +5641|Supplier#000005641|9Ow1wAWicIZar,S3crTLP8TGMlNhzUh|5|15-999-988-8566|-800.68|ly regular foxes. even accounts wake qu +5642|Supplier#000005642|ZwKxAv3V40tW E8P7Qwu,zlu,kPsL|3|13-238-924-2447|4926.93|al packages against the f +5643|Supplier#000005643|UQbMZV5OsBXdjIktGIh4Re|20|30-470-173-3247|7926.85| regular requests. bold packages cajole quickly. escapades cajole fluffily bold requests. iron +5644|Supplier#000005644|yUwPvIdzGnIhyHITqmtaiVEFxLe7 8e|1|11-976-815-6424|3675.69| across the accounts haggle quickly bold accounts. pe +5645|Supplier#000005645|R0XMxCzZ61LlSlf89ISRRYPKuHGrZxe|22|32-475-358-6578|-744.35|ly pending requests cajole deposits. silent pinto beans wake quickly above +5646|Supplier#000005646|OJBPqP6aCHCm0fvzoL|0|10-596-949-7843|5655.21|p furiously before the special foxes. bold theodolites are blithely. carefully +5647|Supplier#000005647|PGXvFiQiUGrtz0ja1|8|18-482-254-3271|5093.22| regular instructions are blithely. unusual, even acco +5648|Supplier#000005648|1l5fUKGh0DriO|5|15-771-332-4497|8726.04|fully express platelets? packages according to the regular packages hag +5649|Supplier#000005649|mUZyczj1OR|20|30-249-449-5666|4596.76| beans according to the furiously ironic packages affix carefull +5650|Supplier#000005650|EmtAaqWy,NSasIdmaB0oo5AA4s yID,|9|19-921-348-6677|2762.15|ggle furiously. express, bold deposits cajole since the quickly pending pinto bean +5651|Supplier#000005651|3aN TCaHK3R8iSw2tfXlXxfZZaenwbqT4NvtJQv|13|23-935-891-8618|1375.32|e about the blithely ironic deposits. bold escapades sleep furiously +5652|Supplier#000005652|fkt9h2fGMaJAPgWk9tyZdwTjau4drG|11|21-257-902-2890|1976.11|w dolphins cajole slyly. carefully pending platelets haggle carefully carefully regular deposits +5653|Supplier#000005653|pJurmYhCxauXpi1DEkr|9|19-828-165-1703|1361.60|auternes breach quickly packages. regular dependencies across the f +5654|Supplier#000005654|FZU0UZQNUsvxSmdHDReLuqRIc5jvHsk|15|25-283-748-1118|2418.48|nd the blithely express requests. realms sleep blithely. express courts cajole dari +5655|Supplier#000005655|L59qgkr191fwE|15|25-158-829-1055|400.75|eposits. slyly final instructions impress about the carefully even theodolites. unusual packages +5656|Supplier#000005656|nYY vwwWaAuXihpyn|6|16-263-334-4011|1294.72| believe slowly-- deposits boost. furiously pen +5657|Supplier#000005657|lRNgdIsnpQGn|23|33-492-144-7673|389.70|carefully about the blithely unusual accounts. +5658|Supplier#000005658|HpLVYgFCXgWG4jv cCO9MiiVgYt|2|12-883-169-4324|2691.95|r pinto beans. special deposits affix even, pending packages? ironic, +5659|Supplier#000005659|tUdkes5CdvOT3jKPRSxRhZqjUlhjgr7jDPlKXL|16|26-556-117-2599|9640.84|e final, even decoys. carefully ironic pinto beans nag slyly unusual packages. fluffily even +5660|Supplier#000005660|aUE2rZxt7mSa2s1b|1|11-379-189-4913|1673.41|sts. ironic braids sleep carefully according to the slyly even excuses. ironic pinto beans wake +5661|Supplier#000005661|pq5wuxmkIW0DyWU |21|31-833-844-2740|6995.17|lent deposits. carefully express ideas haggle quickly closely express accounts +5662|Supplier#000005662|0qAYdmfdFCuDSVXZn C77Cg1D5JdMyNzX9FnK|20|30-629-418-3163|8868.75|arly pending accounts. slyly close requests detect blithely dependencies. regular, e +5663|Supplier#000005663|dj9SqsGeG7dfIrEDLN4GtIZIS|16|26-414-536-1454|5067.61|hely unusual theodolites. furiously regular ideas are blithely silent platelets. careful +5664|Supplier#000005664|b8VWuTXRt66wF9bfrgTmNGuxf1PU0x3O9e|4|14-214-171-8987|7254.81|ts across the quickly pending pin +5665|Supplier#000005665|UQEs3xf5LpmhFLx4CIHM7JHYkGkYF|0|10-763-100-5817|5648.38|gular deposits-- blithely even packages use. furi +5666|Supplier#000005666|PimDDsyzRW uDJOuzK7rlc|12|22-602-344-5759|308.40|r packages affix quickly according to the carefully express sauternes. final packages hang qui +5667|Supplier#000005667|Dzdfnt63Mr2sgAIsGL,iKE|17|27-735-665-8431|147.73|onic dependencies among the fluffily unusual packages snooze against the final, e +5668|Supplier#000005668|9ljOdw7jprqplGgMp|21|31-728-492-8763|5831.61|bold, regular excuses. final instructions wake inside the slyly special instru +5669|Supplier#000005669|OCIG1WZEpZHP|12|22-694-387-8311|-164.35| foxes affix blithely bold +5670|Supplier#000005670|mliDT,MwBrJcAFuF,Ie0uDzgpwblYaHjY4LR|24|34-764-631-5999|7776.42|ly against the furiously ironic packages. ironic accounts wake. carefully unusual pinto be +5671|Supplier#000005671|GLzbcBZZrKT7|2|12-720-247-2328|5367.60|ickly ironic foxes. slyly ironic packages detect at the instr +5672|Supplier#000005672|LBqiISwQ8,L98tE0JS7wC0gUNAzdro|17|27-285-965-5704|214.42|ickly unusual requests was quickly slyly even sauternes. carefully final requests wake. ca +5673|Supplier#000005673|2DaN5NIk4Ugn2suV|16|26-539-840-3727|4678.84|ickly regular requests integrate requests. fluf +5674|Supplier#000005674|Ogh5,d,hT0rIFWW7|17|27-503-368-2294|7328.95|c ideas are carefully finally regular asymptotes. slyly +5675|Supplier#000005675|p6M0KpH4uNbVyySndqYgdKBIJurQnaYMXfUlXxy|5|15-997-627-7932|8777.14| bravely pains. fluffily unusual ideas detect slyly: bold requests +5676|Supplier#000005676|HInJHZisl5svSU1oKsr|21|31-411-864-6979|5466.86|quests. pending, brave deposits cajole blithely. furious asymptotes +5677|Supplier#000005677|hBD9UkHo4hVnrLmKGx|21|31-198-589-9459|7504.91|s wake quickly carefully unusual packages? blithely regular ideas among the even instru +5678|Supplier#000005678|34TkeD6GecS4lpxK8JI2V|23|33-668-572-4270|7038.35|ze furiously against the theodolites. blithely ironic foxes wake doggedly. even realms na +5679|Supplier#000005679|5gSa,3ctOY2w4RjlfPnsIa n|14|24-624-263-6114|8545.55|kages cajole finally beyond t +5680|Supplier#000005680|hWkoAtOkvn|23|33-547-203-1846|7148.26|d, even ideas sleep slyly. silent +5681|Supplier#000005681|In3MJ575HjR|8|18-620-726-3005|7829.51|ts. carefully regular deposits c +5682|Supplier#000005682| pGw5Y44A8bVHhYPluDOlMMS9tfdKmJLhZ|9|19-549-369-2500|6646.84|he daring, ironic packages sleep final, pend +5683|Supplier#000005683|5IHT1DXKzgL7m4sf1adUVT72JcvJAKm|4|14-578-354-3076|6414.90|s affix ruthlessly bold, ironic theodolites. quickly regular foxes cajole quickly behind th +5684|Supplier#000005684|0MzZnU nyISeotsC4A6CpCqMLIOr4|0|10-253-929-1245|2296.42|luffily bold foxes about the fluffily ironic decoys +5685|Supplier#000005685|3Jp7d9F7Oit18wJu,|20|30-277-816-6210|5912.36|efully ironic, regular courts. carefully regular dolphins wake fluf +5686|Supplier#000005686|f2RBKec2T1NIi7yS M|3|13-976-667-8260|4084.73|ly express accounts haggle along the slyly final deposits! final T +5687|Supplier#000005687|nVixSDsrI8YqT2iB|7|17-613-484-7952|-899.18|against the furiously ironic packages must wak +5688|Supplier#000005688|8BPZDnc5B96UHtKKP4TwOyOVAh6PQQy|19|29-299-224-7007|-567.65|s nag furiously. slyly unusual pack +5689|Supplier#000005689|HzWfteIlx2qUjb1Zp6m1X 9HiC|23|33-302-466-6800|5430.98|xes haggle fluffily slyly silent deposits. ironic dependencies use regular p +5690|Supplier#000005690|5NZzCWcgXpA8J2uAzbxsM0Gxd1pi|15|25-747-294-4235|9515.13|quickly final pinto beans haggle. final, regular platelets hag +5691|Supplier#000005691|Jr9KBVlbmCuXyXL68OXGyRhmG3I4FCUXgUp eAid|10|20-335-618-4833|925.32|unts. quickly even packages nag carefully. stealthily regular sentim +5692|Supplier#000005692|335mZ92x7onPQOtTY7VoXH|11|21-214-301-1719|6873.77|fily pending attainments. carefully final accounts across the final, silent +5693|Supplier#000005693|TB8s6K2lo6se,Yu0kABDIo gHDAg3Oxd6UsN94|1|11-716-502-9928|3528.00|yly about the deposits. final theodolites s +5694|Supplier#000005694|OMnPfXxig2BhaN3kjeDEjWL32wWxAFA fMAsSbL|22|32-631-102-8816|3073.84|ts. regular dependencies sleep furiously according to the pending pac +5695|Supplier#000005695|YOkDgHOjMdM0E8gGx6adBSLSnQ9|18|28-249-980-3939|4007.49|r instructions: furiously ir +5696|Supplier#000005696|a8mRYpNQHaBi0x5TYejuHPn,cb|2|12-622-757-1711|9959.88|ages. furiously final courts boost slyly slyly ironic accounts. unusual packages cajole. stealth +5697|Supplier#000005697|0WIsnPEvvRU,qM88|20|30-314-194-4168|6702.95|ar Tiresias are slyly regular instructions. final accounts after the ironic instruc +5698|Supplier#000005698|3XqFeCid8BHa|16|26-691-342-8783|3447.77|arefully final accounts. slyly express accounts nag fluffily +5699|Supplier#000005699|a1VSYsx3A w4,|13|23-451-776-3365|7607.66| furiously about the unusual, ironic packages. +5700|Supplier#000005700|LsNhljBonNCGLXqM2bkar|18|28-416-627-2570|7019.71|sly bold deposits mold carefully after the unusual dependencies. pending account +5701|Supplier#000005701|hRxTGOYJ0a1FyI65S5DxK32rCeUxUGjyTOgMaQ3E|9|19-590-846-6925|3299.81|ake. final requests are furiously quickly +5702|Supplier#000005702|6wC6dXrpK4,KQ0|12|22-332-520-1817|6837.43|nal pinto beans haggle blithely regular depend +5703|Supplier#000005703|980CIMdpitmPFr9udm912YBCY2gzBHrl6O,3Az44|2|12-800-405-6418|8660.64|packages are carefully. slyl +5704|Supplier#000005704|r0hgXuC3yj4sLzk12m5Vb5EshdHbDTwS2m0VMV|18|28-165-244-6997|5389.38|ular requests. final, unusual +5705|Supplier#000005705| Q2B J8,N2uF8yrux7|0|10-935-168-4106|3332.29|after the furiously even dolphins-- quickly express requests according to the care +5706|Supplier#000005706|OERtvmdpWoYTWolt0Bn7wb,|17|27-462-905-4781|2801.23|usly silent accounts. slyly express requests nag quickly inside the +5707|Supplier#000005707|flRR,Sl2Rs6jSwNHIPpko 8y9UF6T|9|19-972-519-2072|9281.05|g the furiously pending dependencies. even, even deposits cajole slyly +5708|Supplier#000005708|OsiVMcT732WkL92Mf2NfXETGs8|16|26-303-147-3139|4753.18|kages are according to the special braids. express grouches print even, even orbits. slyly ironi +5709|Supplier#000005709|s4D3HaH fHha5HIwd|23|33-731-294-1689|2676.46|s deposits affix at the silent requests. accounts sleep busily. blithely regular +5710|Supplier#000005710|, XxpudjGx53Q1N5RThZyn1CbXN6oPrfDwBS|2|12-427-727-2339|2218.67|lyly final excuses are carefully slyly even decoys. slyly ironic deposits sleep. furious +5711|Supplier#000005711|75R9OWmSouInxMjpckNMNo,kkAl1i|14|24-138-157-1385|8113.69|deas solve final, special instructions. pinto beans c +5712|Supplier#000005712|RKoQ5ijTQ6q|24|34-335-812-7636|8834.10|s wake about the final dolphins. carefully final a +5713|Supplier#000005713|SJN6CtVJPDQyQbLzDaqP |9|19-354-873-1233|5864.45|counts against the furiously express platelets haggle after the +5714|Supplier#000005714|DZKWwEVAPFzU9hp 3 ZCFMyJ|19|29-467-443-6477|1824.04|arhorses sleep around the theodolites. slyly reg +5715|Supplier#000005715|ZW R6zxZtBFRoeaZtQw8YifUBoqSv16hG|13|23-268-174-1595|8555.40|e blithely silent multipliers are +5716|Supplier#000005716|AzRV7wmScLO9Or2uJdmiD|6|16-485-932-5209|7596.31|fully unusual packages. carefully final foxes was furiously. furiously final request +5717|Supplier#000005717|tRh9 y0NJ0WuCeYZeCy94caIw|18|28-392-305-4077|3257.14| sauternes. furiously express pinto beans integrate carefully. theodolites sleep carefu +5718|Supplier#000005718|FspgdwMNadWuxrdVHWQR|5|15-337-255-8243|4660.29|ial deposits. carefully ironic packages against the slyly pending +5719|Supplier#000005719|nYekU0ih LmudR43FpcTQELA0VnPbXe|15|25-718-875-9529|3564.04|ideas haggle busily. carefully special pains ac +5720|Supplier#000005720|xcbHl1k0L3Ij4yOzMfp|20|30-964-754-8581|3862.74|s. fluffily regular reque +5721|Supplier#000005721|LXqfP897JYdfIlo dNqHuY4T|20|30-202-248-8360|8136.60|ng requests. carefully regular accounts along the +5722|Supplier#000005722|8gsTxNzb3IBTwguXYHG2Mp|9|19-364-923-8043|8897.56|ccounts sleep sometimes regular theodolites. doggedly silent foxes wake furiously final the +5723|Supplier#000005723|CmaKk13R7IAhf2ClogU|2|12-974-187-1953|5377.49|dencies. fluffily idle package +5724|Supplier#000005724|G4EV9Y56T6Gt18QVL|14|24-125-255-7661|1536.97|kages nag furiously final, even pinto beans. fluffily pending excuses na +5725|Supplier#000005725|0iiEmSAM,OBNaTLQREpJdPXZihyIkhCZ,1mn|10|20-516-236-4922|1875.84|al requests. express foxes after the ir +5726|Supplier#000005726|zyIRsW98Mz2G kVAewFLNOhQE0yPnB8R|23|33-371-634-8887|-830.09|ess deposits haggle blithely blithely special accounts +5727|Supplier#000005727|iL4rs,YxY9yXqzYqXbUwKcxK92ojStUG81|0|10-885-893-1901|5574.72|phins. carefully close waters use boldly final instructions. quickly final depo +5728|Supplier#000005728|89BeWN6 2R3wLnroAH1XvUazx8VV2rMWNWr OEXO|18|28-669-329-2958|5378.76|ng to the quickly regular excuses. silent accounts across the final requests wake furiously acc +5729|Supplier#000005729|j9TX35udXCH1lQPVNI wTsiR92|18|28-146-390-5101|790.72|ccounts. unusual Tiresias maintain quickly. +5730|Supplier#000005730|5rkb0PSews HvxkL8JaD41UpnSF2cg8H1|3|13-997-710-9302|-477.49| bold packages. bold foxes after the furio +5731|Supplier#000005731|tb 0w3GVsPR5XOl|24|34-140-915-5889|2807.97|ructions. quickly ironic pa +5732|Supplier#000005732|,0tVFQMFpshc2MoTOxRPY|11|21-894-657-6530|1474.69|quests haggle slowly furiously regular theodolites. ironic accounts haggle fluffily; quickly pend +5733|Supplier#000005733|u7or44tfqMxCogyNaJLXQ9hHuZTFcWDkkjXOcPY|15|25-521-829-3189|-10.82|ully. even deposits detect fluffily. quickly fina +5734|Supplier#000005734|1hNLPg3gwSld5nRJJHoC|0|10-571-582-3116|3056.74|encies. blithely ironic deposits wake slyly ironic, regular dugouts! carefully expres +5735|Supplier#000005735|espkK1WZ liET7zrStHGOE4|5|15-759-700-1576|2610.89| theodolites. regular dugouts affix across the fin +5736|Supplier#000005736|2dq XTYhtYWSfp|3|13-434-170-2340|207.25|lithely regular frets. blithely regular theodolite +5737|Supplier#000005737|dmEWcS32C3kx,d,B95 OmYn48|3|13-140-578-4599|5193.56|nag express deposits. quickly unusual deposits haggle bo +5738|Supplier#000005738|lnG P4MTSbdzos1iC0O4xJ9BKaSj4czTSW,dK|1|11-245-699-6773|1229.58|g fluffily theodolites. blithely +5739|Supplier#000005739|mqvJAdeXVjwtPsreLGkI|1|11-660-918-5876|5541.49|ounts. quickly regular packages serve blithely final accounts. quickly express +5740|Supplier#000005740| a80WpBbwueCb6leN ja59XwqRKL|5|15-858-545-1925|7837.02|g to the carefully even courts. instructions haggle furiou +5741|Supplier#000005741|LTTVOuHZLITkfwpaKMox|13|23-614-828-5238|897.14|sleep blithely regular depend +5742|Supplier#000005742|jr57A7w0yC5sKMj17OwGAIRdNiOVqkhvI7E8|9|19-523-715-4348|6222.09|l deposits. quickly ironic requests haggle blithely. slyly final foxes do +5743|Supplier#000005743|cbIUTLIk6VacVEiMQaLWRaGJB4SGaGIg8|21|31-390-660-6966|6746.48|ual frays use blithely idly express instructions. bold +5744|Supplier#000005744|ryzWvnrxMg8km1YMVZaIfOUgv, XVfM|12|22-916-625-1543|9109.12|cial asymptotes detect carefully among the carefully even accounts. foxes affix c +5745|Supplier#000005745|wUXBzY4t2au0ZxaBoGyuer4K jZ4O48R1v1DJFtW|6|16-123-748-1248|8031.09|ss the fluffily even pinto beans. carefully final +5746|Supplier#000005746|QAX1j3ALe VdS|5|15-632-671-9153|6538.11|ts boost quickly among the blithely regular asymptotes. furiously special pinto beans haggle furious +5747|Supplier#000005747|D0gyiOCgqwi123gjTq7cQnNzLD,e16rF 8JPO|19|29-938-591-4953|5001.39|ly even packages. theodolites x-ray quickly. carefully even packages lose furiously in +5748|Supplier#000005748|ZoCvc8ocx2|23|33-174-442-3657|5403.54|furious accounts nag. fluffily regular platelets cajole quic +5749|Supplier#000005749|X2SFFwGxttEpASPa7LZL,L|8|18-208-540-6801|-764.45|, final deposits. unusual platelets are above the +5750|Supplier#000005750|Erx3lAgu0g62iaHF9x50uMH4EgeN9hEG|19|29-344-502-5481|8841.59|gainst the pinto beans. fluffily unusual dependencies affix slyly even deposits. +5751|Supplier#000005751|yVwqlnvEUAboCZ5GUV|22|32-530-894-8442|-329.90|c foxes are blithely along the quickly dogged requests. pendin +5752|Supplier#000005752|qKui69rK6v0l,fz1T|24|34-134-879-4176|8311.79|sheaves. furiously ironic requests are alongside of the sometimes idle foxes. boldly iro +5753|Supplier#000005753|FSyU0RM1P7lA4,KYcBcsrqg4Lu|9|19-254-162-9836|5289.08|ccounts. regular, ironic excuses +5754|Supplier#000005754| sNspSyE3ne2Zi,OARwe|13|23-432-847-7108|585.83|oxes. blithely regular instructions according to th +5755|Supplier#000005755|pHKjNRUKhvNgZVI5iBPpKg|20|30-177-265-4256|-967.10|ickly after the carefully bold packages. slyly final pinto beans haggle blit +5756|Supplier#000005756|1isJPcFX4g|7|17-153-781-8644|1931.42|s run furiously alongside of the bold accounts +5757|Supplier#000005757|FBJ0hUWAem3whciziO7lndaJ|23|33-580-351-5737|9293.21|ng requests are carefully along the slyly even packages. carefully unusual deposi +5758|Supplier#000005758|9kCDpuQmHTjaOw8Lhjrt1|18|28-874-498-3667|136.41|cajole-- quick courts wake fluffily quickly regular instructions. +5759|Supplier#000005759|reK0xUkOM89gKteS5RzzbPWhqlC0OreJkBL|1|11-746-668-5794|2631.72|s the regular deposits wak +5760|Supplier#000005760|C9Q CnGG1Uqj5QDkxF3cDwgj|22|32-390-528-2239|9236.95|arefully bold ideas. quietly regular ideas about the even, even reques +5761|Supplier#000005761|qpSoYwQLWcsVIlyVeacQbbgjcEb XjCxG16Qh5d|15|25-773-316-2657|503.29|ly according to the slyly even packages. quickly regular deposits are alongside of the idly expre +5762|Supplier#000005762|KOAqAO1NZEVaQ3B8g20TukofhnNAWtQH80ck LHS|21|31-408-388-3157|8679.58| fluffily. fluffily bold dinos are slyly after the blithely express pains. final so +5763|Supplier#000005763|T1Eel5ZCTS6,s8SuJXOchvdgu8 c6OqmydwE4E|19|29-507-742-3020|4794.94|slyly bold requests print requests. foxes sleep +5764|Supplier#000005764|VoAeTqnKXYmAEVpZy|8|18-242-175-8608|533.78|ounts. blithely special hockey players cajole. furiously even depos +5765|Supplier#000005765|ZjVSXoThNZRwE|18|28-206-130-5547|4224.95|onic ideas nag carefully final accounts. slyly pending deposits nag fluffily express +5766|Supplier#000005766|1biOHW88EfLqtoUoypl6lmgAtUqkDy|13|23-973-793-6613|1690.74|al warthogs. furiously unusual requests about the slyly final pains cajole slyly final +5767|Supplier#000005767|kB4LYA9E7rU7h7P3JH|11|21-348-204-5182|1283.91|regular, ironic deposits. bli +5768|Supplier#000005768|zAAN4SgAH0EpHIWdl1Cs1YzebmAF3tJHOzIdZ|16|26-194-928-3568|5678.80|excuses wake fluffily. carefully express accounts sleep fluffily regular, d +5769|Supplier#000005769|1xL7qBXFlGNPd1|12|22-546-516-8185|5659.77|nts. carefully special deposits nod b +5770|Supplier#000005770|qbm8KfRcTTye|6|16-159-435-4344|4770.11|accounts cajole blithely-- foxes according to +5771|Supplier#000005771|wqPmXKZoucEWZM0,6s56AjHoZXi7r6skXoV U|22|32-436-418-7659|1189.39| packages. furiously unusual foxes cajole blithely. regular excuses sleep furiously. +5772|Supplier#000005772|yJVMEQMeQOmXZWvlLle|18|28-536-856-1875|1084.03|odolites. pending multipliers haggle fluffily final requests. caref +5773|Supplier#000005773|M7gZOUT WwHYkkYkDjdAfAZt9Ziw|14|24-433-658-6533|1308.49|breach above the slyly busy deposits. furiously even requests +5774|Supplier#000005774|8P9EG8Smg40oyzzFIbo9f|20|30-267-955-9017|659.29|arly ironic accounts. final, regular asymptotes wake against the final foxes. packages are. c +5775|Supplier#000005775|oRnGlwZ44M zRRA0aNLCikv3XAS4l|17|27-149-720-8879|1645.40|as boost slyly pending deposits. furiously specia +5776|Supplier#000005776|5tlx5Mg39QBtCSrKo9sF7KW 5q|6|16-713-108-8117|9180.76|ly according to the quickly ironic excuses. ironic, slow warhorses detect carefully bli +5777|Supplier#000005777|sqRdutBlc78lzXTqp0yAz4fm50bMfi9fZUF|6|16-259-814-4607|2086.96|ly blithely regular ideas. slyly final deposits sleep evenly a +5778|Supplier#000005778|n3v,v38DOorWnJt9EP|21|31-432-364-5573|5433.53| deposits believe blithely according to the f +5779|Supplier#000005779|VuGyjQNY8TbHJrVjMh28uZ 8eGedBJwq7sOK|12|22-629-392-8179|8334.78|slyly ironic packages. fluffily final pinto beans haggle furiously. slyl +5780|Supplier#000005780|H1BVHnv2ofhISQM1GdTUL|24|34-276-680-3520|6734.88|sts. furiously bold requests against the requests use q +5781|Supplier#000005781|iYF7vxZ1yn6U6QETAT UF4w95 Qy52CZr|18|28-662-305-3322|1223.78| fluffily pending ideas after +5782|Supplier#000005782|up1JCuz6MBxCbLDrrQV4gmN R|2|12-884-540-4282|7047.08| theodolites: accounts above the carefully unusual platelets are carefully blithely pend +5783|Supplier#000005783|ADa1LuDzf2iskm3z|1|11-904-393-6389|1248.71|gle blithely above the asymptotes; slow packages +5784|Supplier#000005784|j1TfApFp0Heyajtm3vOjKGAvQ6J|5|15-402-811-1863|7945.65|g accounts use regularly carefully +5785|Supplier#000005785|X7hkrkiVsCycsepTxrevBNpv6TvMIwECB|14|24-663-783-1699|558.21|e upon the quickly even foxes +5786|Supplier#000005786|BYrOEDobP2SNIpkMH39IbhwFAGEX,X3lzg|14|24-438-645-1925|7699.60|as wake quickly even depths. quickly final ex +5787|Supplier#000005787|NaJJpY1guXXNL3HF1g3gfqxA50uHD00g|23|33-503-818-5132|1301.98| alongside of the even theodolites are near the regular instructions. daringly special +5788|Supplier#000005788|fRIvHQMV0YTbMbX0kzxRhQm0AH8WG0DlyReg|10|20-651-782-2644|1249.32|sh carefully against the carefully final attainments. furiously final plate +5789|Supplier#000005789|dhdUfW6tbTWv49849nX52,8xJ4,tZ0RJn|21|31-172-402-4587|7353.08|ct across the carefully even packages. ironic requests use slyly blithely bold deposits. furiou +5790|Supplier#000005790|hjnO2Pn2HZIiUyuvvrOtK,|2|12-201-826-9391|7193.49|ng to the furiously final accounts +5791|Supplier#000005791|zub2zCV,jhHPPQqi,P2INAjE1zI n66cOEoXFG|19|29-549-251-5384|9101.00|ts. notornis detect blithely above the carefully bold requests. blithely even package +5792|Supplier#000005792|4xj ncbc1,0Sb hK1nB|2|12-104-141-7027|3660.90|nag quickly. carefully blithe accou +5793|Supplier#000005793|WVlXdhQywxw1frEcc|24|34-606-945-6044|3793.96|kages integrate slyly along the regular packages. quickly busy p +5794|Supplier#000005794|l534i KF5ArsIrOGs2WMl3S,6|13|23-493-473-6375|7089.46|iously ironic packages. courts haggle slyly across t +5795|Supplier#000005795|HXxMQyJ5aywAYA7N21g2Np,1XlhbMJtZN6PZl|2|12-184-817-2379|9653.69| requests. quietly ironic accounts integrate unusual courts. requests caj +5796|Supplier#000005796|IgQVugiNTGwZltdxqcRkiP3|9|19-524-102-8768|8612.85|yly after the regular, even packages. busily ironic fox +5797|Supplier#000005797|,o,OebwRbSDmVl9gN9fpWPCiqB UogvlSR|3|13-825-413-2694|5228.42| packages boost ironically against the slyly ironic package +5798|Supplier#000005798|N7bzE7FAYjWUjHi0oevNic0TAoloVli|13|23-559-868-4931|4060.05|s wake. quickly bold multipliers +5799|Supplier#000005799|j uJliQkuJ3vQL|20|30-658-513-7780|2317.09|sly even packages. blithely +5800|Supplier#000005800|vgZ4An2oUXmsh|0|10-397-289-9564|1351.11|kly even deposits at the carefully ironic instructions unw +5801|Supplier#000005801|ji0Lgmn1rJ3Gm3Vlh0S6AH4Dmng|13|23-108-219-2697|2020.46|se? regular requests sleep quickly. unusual, spe +5802|Supplier#000005802|1oI3h0bV0AEQtHh,2LfPimvikjSiB3FyQzYBZ|16|26-526-396-6060|9875.27|s. slyly careful instructions detect slyly among the regular asympto +5803|Supplier#000005803|,8BLtai0L4f3sq|10|20-362-331-1942|3807.72|ent pinto beans after the carefully bold ideas haggle +5804|Supplier#000005804|yfYseM0RmmsJ129,66jOm|6|16-490-936-6817|7337.18|dolphins. furiously even deposits +5805|Supplier#000005805|UELBada7O8oC4tBWnYBs3Wh,Vg3CW,kr4c,e|14|24-717-124-7710|4090.56|st. regular, regular pinto beans haggle among the blithely even dependencies. slyly final packag +5806|Supplier#000005806|jUq5LXbXpvpOvclriKO|11|21-920-603-8709|8991.50|s. final deposits x-ray. quickly unusual escapades affix dependencies! carefully ironic de +5807|Supplier#000005807|y4OEw7UCy05r4q4O3x,pOPD2S|11|21-819-445-6675|173.50| slyly unusual instructions affix quickl +5808|Supplier#000005808|s 4icF7UIthw8OZB|20|30-518-213-5617|9946.74|furiously. pending deposits run slyly above the slyly e +5809|Supplier#000005809|OS,jWuDlF0msPmJVIXAFsJVGjRs|3|13-837-191-7870|6011.16|counts hinder furiously final, pending account +5810|Supplier#000005810|qEe2y4FokSEJd|7|17-936-158-2557|3575.73|elets sleep after the final ideas. express foxes run. dug +5811|Supplier#000005811|YclYVRsXa0FyR8VhP4GFCLTYlyWz|19|29-286-414-2279|2241.41|thely ironic foxes boost permanently blith +5812|Supplier#000005812|4KOYQzG9tVYDapYQA8UreS4,T9K0mX,srrTSk|13|23-116-403-1145|5319.82|foxes haggle daringly along the +5813|Supplier#000005813|UhMr cngFTYYRJw2u6A|15|25-920-170-9164|216.08|f the special dependencies are furiously after the regular, pending accounts. ironic, pendin +5814|Supplier#000005814|wktYKjWKuPEvV|18|28-345-800-1624|-369.69|. special sentiments haggle blithely according to the acc +5815|Supplier#000005815|S6cu6cspYxHlTz2|21|31-447-356-3777|-432.19|ickly final packages use slyly according to the fluff +5816|Supplier#000005816|HEDkA6Gq,dMaCQ8plT0Sw2S98XJKklIOb3|15|25-945-284-2532|5138.28|kages nag dependencies! slyly pending platelets dazzle quickly fluffily pending pack +5817|Supplier#000005817|ugdq690ufc0e|1|11-416-479-2168|7104.56| instructions. pains was quickly. pending accounts do wake. even, pending pinto beans cajo +5818|Supplier#000005818|tD0KMqVZeLwxrGJZRU8fzKgiGyy|16|26-339-165-7108|-543.33|ay furiously. furiously even somas nod depe +5819|Supplier#000005819|qk3VnjywYdD1bGj9hksJbGueeiHn|17|27-569-186-5183|7167.48|ckly. ironic gifts haggle carefully regu +5820|Supplier#000005820|xKuv0Q0e2blJSIOCOB4|8|18-608-713-5736|899.00|thely within the furiously ironic deposits. quickly ironic acc +5821|Supplier#000005821|UOvHgHsNlljfaOcztQ5f|12|22-718-102-2301|4504.73|s. instructions are quickly. car +5822|Supplier#000005822|haVfNaqtNKyjY8H,sjr xDthZAJVXrjE|22|32-666-835-3072|6940.25|kly final requests. deposits cajole slyly. slyly pending pinto beans wake quickly. furious +5823|Supplier#000005823|DKXNhifWIqvODviyPBkNGLHu1EDtX|9|19-711-553-3710|9785.96|ly pending accounts are even, ironic requests. quickly ironic dependencies cajole furiousl +5824|Supplier#000005824|Hbo853nz6rQ5RKlDaO|19|29-320-222-4934|9759.04|leep fluffily? furiously regular deposits according to the pinto beans nag carefully above the slyly +5825|Supplier#000005825|tekQ EBsiIjK l7lNZyNddim0me77svB4VfWLier|6|16-415-870-8333|8340.19|e carefully even epitaphs. carefully unusual packages are carefu +5826|Supplier#000005826|UBUkU3Qj vj ejTvBrm1nmlFEc4ycHH2U5fvcK6|12|22-323-117-7368|9604.84|egular accounts. even, final +5827|Supplier#000005827|O,abBOMPP,r4|22|32-146-400-6420|1545.16|after the furiously even requests-- ironic, final accounts wake slyly alongside of the blith +5828|Supplier#000005828|Xt0EqDCNU6X00sNsIO7nd0ws3H|10|20-435-850-8703|7496.91|furiously about the fluffily careful idea +5829|Supplier#000005829|QMQ9k3TH9 pvKCKEXrDI0af|18|28-136-182-2731|4598.73|lyly bold requests. blithely regular accounts alongside of the daring +5830|Supplier#000005830|AFO48PCLrBgkBY3iSbpA5uvVTx ju8d oA|0|10-880-688-4405|2885.30| instructions. carefully pending packages slee +5831|Supplier#000005831|MY rnEWm8bes4Z|11|21-884-281-3474|3179.19|osits sleep fluffily final deposits. accounts are slyly furiously expre +5832|Supplier#000005832|zd9aAhG3UaqglPdUP5VErS3gmwVbCjseJEnj8g|18|28-335-357-6777|3600.96|equests: final deposits wake amon +5833|Supplier#000005833|fgLq3zjOQBJohPV6aL1Q|9|19-529-689-9783|8382.43| the furiously stealthy theodolites integrate fluffily sl +5834|Supplier#000005834|hrgnqRX qh|23|33-650-371-9232|4559.02|l packages boost ironic, unusual accou +5835|Supplier#000005835|rYoXzV3EZ77Z|21|31-985-363-2730|6417.08|es about the ironic accounts sleep above the regular accounts. slyly re +5836|Supplier#000005836|tx3SjPD2ZuWGFBRH,|3|13-140-789-4690|5871.69|quests use carefully. slyly regular accounts +5837|Supplier#000005837|rllOlAbECrWmuK7fEhd4u|23|33-255-799-2064|2587.67|ggle slyly throughout the ironic deposits. de +5838|Supplier#000005838|4SzERY e8s|0|10-697-146-8808|5905.55|to beans. slyly permanent foxes cajole alongside of the ironic foxes. even +5839|Supplier#000005839|gMn4qDf3ZMBP|9|19-434-809-9475|-350.31|xes wake fluffily final asymptotes. furiously express deposits wake furio +5840|Supplier#000005840|5056ztykbhkawZOeJMKyxyv|12|22-500-573-8261|6340.86|onic pinto beans impress furiously blithely permanent packages. slyly express requests might +5841|Supplier#000005841|I,aH9IfrWJ6cRKSKFx,QP4ryYmGaP|10|20-881-949-3700|935.86|e quickly ironic instructions sleep carefully pinto b +5842|Supplier#000005842|USS85,wP8t QZmSeKzfP7F2Nxim,nsA4MXvOeDVe|20|30-375-408-7853|7954.83|quests hang slyly furiously ironic dolphins. slyly final acco +5843|Supplier#000005843|PdrNI2F59l0otH55I5mu0S,OVnm47Ihbb7rHhV7|6|16-263-365-3876|882.39|egular requests according to the blithely ironic packages doubt about the +5844|Supplier#000005844|ZpU,qApfX1W02LNNF,wzKN|12|22-179-453-4856|259.73|ions. fluffily ironic deposits sleep blithely blithely final deposits. s +5845|Supplier#000005845|3DQwBzSByUNS|15|25-957-432-1491|7291.62|. slyly ironic foxes detect blithely. slyly final instruction +5846|Supplier#000005846|S0,rj6gU0qK9LVwAi9W ISF|2|12-589-214-1265|8289.15| unusual platelets. final platelets cajole blithely across the t +5847|Supplier#000005847|obol3bfh0zWi|11|21-530-950-2847|9796.31|equests. blithely regular deposits should have to impress. final platelets integrate fluffily +5848|Supplier#000005848|xxheHzTSroSqvL,Jt4bni7n1|20|30-144-868-2228|7719.51|y even epitaphs haggle fluffily along the blithely regular r +5849|Supplier#000005849|rCK3WlppdkFX68a5bxonrSwKCp|5|15-582-489-3907|5895.39|ructions boost blithely Customer posRecommendsdolites solve blithely unusual ideas +5850|Supplier#000005850|cn4kJKDy5otUOanVqO2ooztd8b1Ob|24|34-933-393-1499|4791.82|lowly bold deposits sleep ironic decoys. furio +5851|Supplier#000005851|4mnZHqdy I31JjlXvhpIqvp28KJ2PI2gXTYBQjm|6|16-600-916-5804|3179.49|. carefully unusual packa +5852|Supplier#000005852|gc7fELbvE,fB4XCvThrg,|11|21-944-804-9358|8611.14|the furiously final deposits. f +5853|Supplier#000005853|Tctc1i5u8J1iVjNfAeJaylgSMJR5zAMOzm|15|25-735-779-9567|9533.65|es do haggle among the slyly regular grouches. final requests dazzle quickly. +5854|Supplier#000005854|TrWjpM,jt0dnaJhGPfzM FggvMgo8CzYnr94,vM|18|28-958-693-2899|4151.31|leep. furiously ironic dependencies cajole quickly special packages. furiously bold requests use ar +5855|Supplier#000005855|zz7AA867zRvbpsZIRmoiQ5|3|13-263-716-1110|7096.08| deposits nag carefully after the final pinto beans. quickly even deposits sleep across the bl +5856|Supplier#000005856|ZZY 72oxhYlHGj4m44Sc|15|25-126-957-7207|-124.23| furiously careful ideas believe. regular requests along the dolphins sleep blit +5857|Supplier#000005857|OzpOLV67Y9uV lftTPgGQpiKfjaW9bSj9fr65Y|24|34-480-939-4166|33.10|atelets. slyly express packages sleep care +5858|Supplier#000005858|hwuebOwCH6eThXJfZ,VnW2KQ6HuBl9G43H7H4|7|17-473-346-5774|4316.38|quests boost against the furiously regular deposits. busily ev +5859|Supplier#000005859|KwJ 0xkkNHIZmRhsUDcqxq28I6BZMp|22|32-724-166-1125|9438.89|haggle carefully final requests. furiously ruthless foxes haggle furi +5860|Supplier#000005860|6zEByn0ZDjIUcOG,J41j76d2,,iYXrsk35ARd|1|11-241-464-1923|6483.10|es sleep above the carefully final +5861|Supplier#000005861|rH5v IFhgwje fiI1A|14|24-710-401-5973|4482.97|s. ironic, bold asymptotes boost blithel +5862|Supplier#000005862|iN7KKjLyA,KcU9s7|14|24-893-868-7265|4329.27|ccording to the silent, final platelets sleep carefully across the pending foxes. special, special +5863|Supplier#000005863|qTTgI7ay9JmcaE9UQIOniUxbe|14|24-169-439-5401|6103.04|sual pinto beans. fluffily silent foxes use aft +5864|Supplier#000005864|WCNfBPZeSXh3h,c|22|32-454-883-3821|7852.45|usly unusual pinto beans. brave ideas sleep carefully quickly ironi +5865|Supplier#000005865|HNvx9LdPzWRG4AVvzjMtGr|23|33-721-399-3903|7215.90| wake according to the foxes. ironic, final ideas nag pendin +5866|Supplier#000005866|BdqfBk1CHo q8WFqQPueYlM4155VUbb|24|34-582-696-4179|663.68|! slyly unusual requests wake around the regular, careful accounts. quickly even requests use busi +5867|Supplier#000005867|Xr,svOB0nR0|0|10-333-247-2253|9379.98|. carefully final pinto beans was furiously to the regular e +5868|Supplier#000005868|8LvN8CjlL4eIeGL6o4zDzvi|16|26-184-738-3178|6407.27|ickly bold requests. quickly pending excuses use afte +5869|Supplier#000005869|,UjeDDuvd6xwv513Uhl9o|24|34-551-419-5228|6020.16|nts. furiously regular accounts nag furiously alo +5870|Supplier#000005870|woKRQpz8lVkG3U|5|15-286-734-3144|838.72|even, ironic ideas: even +5871|Supplier#000005871|cD8wU,pMUASUttBQy2o|2|12-485-679-2523|3374.15|r requests. final, final foxes haggle even pinto beans. fluffily unusual frets sle +5872|Supplier#000005872|4BAYNI9JrbipNPswV9uFmYwYkP0aqBUvoWuA|2|12-313-676-7687|2354.96|hely ironic gifts against the bold, +5873|Supplier#000005873|fymmowXPSlMtTLSZtgX5GdH8jbWTifvuSy|9|19-458-725-2451|-239.39| slyly theodolites. pending hockey pl +5874|Supplier#000005874| gcWcaQfgcXFeIZCHGCnWye4RAvP7glbfF|19|29-949-430-7350|561.84|r the requests cajole regular +5875|Supplier#000005875|lK,sYiGzB94hSyHy9xvSZFbVQNCZe2LXZuGbS|3|13-189-850-2992|2835.61|slyly ironic deposits: ironic, final pinto beans hind +5876|Supplier#000005876|gao cf6F4AVlabFh2X8pEJQdVQ1E|11|21-765-707-6735|3995.56|s. slyly special pinto beans doze blithely. final accounts across the quickly even pinto beans c +5877|Supplier#000005877|EyqOHClZZMJkq grnOX9 4alZx6P7B2fq|13|23-419-288-6451|7447.86|lar pinto beans breach carefully final pinto +5878|Supplier#000005878|A,IadYxfPUwdtq64l0IdYyn|8|18-848-904-5029|5204.60| haggle carefully: furiously final asymptotes sleep after the slyly regular orbits. evenly speci +5879|Supplier#000005879|rtkHSrcVLnC|18|28-167-409-9977|-220.76|ounts. regular, pending frays was accounts. quickly regular accounts wake. f +5880|Supplier#000005880|,kWeCiR0Z0FKHgtlSl|11|21-288-277-3094|5543.38|inal, final requests. slyly final platelets cajole fluffily unusual tithes. carefull +5881|Supplier#000005881|nzSpOTf beWk3s,wdEIvhcnxbIokhBmpw0e|14|24-679-231-1719|6562.25| sleep furiously upon the quickly regular p +5882|Supplier#000005882|tyhkI34H9CzUnixln82wW6zy|17|27-234-815-5699|3684.37|packages affix courts. bold, ruth +5883|Supplier#000005883|eDFByWP 4o3w4zpExTN|1|11-595-897-4629|2135.52| up the furiously regular pearls. regular pinto beans cajole alongside of the deposits-- carefully s +5884|Supplier#000005884|ePTyjlInRJD20c8AwIu|1|11-137-953-9158|7598.05|es use quickly quickly ironic asymptotes. slyly pending pinto beans across the care +5885|Supplier#000005885|OCHwRDJ6B4ddVLNuNu5aGQloABT0frKScM35s|9|19-371-647-9423|-568.75|inally bold requests nag fluffily boldly +5886|Supplier#000005886|j0lxko ,iX2jE|23|33-484-154-9190|8995.48|tructions. carefully regular asymptotes use. carefully final foxes haggle fluffily e +5887|Supplier#000005887|ezR5v1JpTU4X2|1|11-632-582-5967|1099.83|. quickly pending accounts use accounts. furiously final foxes de +5888|Supplier#000005888|vinz89pQTZV4rd,rmz1uL6kFErUS5CnJ7qS65t|4|14-227-190-2807|2741.36|gs. final foxes affix above +5889|Supplier#000005889|a mHHvmLpO3dwDLkK1xs6xHfAuPzOXsDA38,qq|1|11-510-132-4502|4051.72|y express requests grow blithely along the +5890|Supplier#000005890|uvY58i9fnlwZca90lKl|10|20-594-416-4451|5268.70|ily final requests are requests. furiously regula +5891|Supplier#000005891|xsn9EWGZGqq6,qjpVELjvwzeev|15|25-412-292-5694|-865.02|s haggle furiously final foxes. furiously pending accounts according to the final dependencie +5892|Supplier#000005892|Xm60l3bJ5dpolwpYVPeVmtAGb5hWUy4UZVp1T|4|14-685-978-4271|1497.24|egular deposits sleep carefully. ideas across the carefull +5893|Supplier#000005893|e,hz2aPH8,lHPL5l7ztoo7mLSJ|24|34-871-533-8502|5029.68|the carefully express orbits; furiously special theodolites are furiously across t +5894|Supplier#000005894|x,96zxP3FAMchh8yAIWJq|0|10-621-434-6822|7902.62|across the busy grouches. bold pack +5895|Supplier#000005895|03OgeZendt7MogwRw5oKwvMfuh76e0dWpCQDY9|18|28-778-306-3099|3380.42|usly. carefully regular f +5896|Supplier#000005896|NwV,QYlRuz|20|30-167-592-8264|8687.00|deposits use slyly furiously bold packages; furious +5897|Supplier#000005897|0vftZyNjdMowV7u8UalLK3|10|20-318-235-6430|8551.49|egular requests. regular platelets affix across the furiously regular pinto bean +5898|Supplier#000005898|guNPgZq NmfW VUGj,R5D63GGo5ZWQH9OHnQGMH|15|25-952-897-6515|7907.18|e regular accounts. blithely unusual foxes are. slyl +5899|Supplier#000005899|SyLaUd8giTOVn919BfD4o8An |8|18-391-485-7499|3087.61|re slyly after the ideas. pending gifts at the furiously express accounts are whithout the slyly sl +5900|Supplier#000005900|bx3tHCCW62OOsiFfBzbjc|9|19-170-943-2800|4464.67|hely even excuses. slyly pending braids sleep quic +5901|Supplier#000005901|vfCFcAHhCa2WfogeAjYfZshxHQQdJ|6|16-840-687-6900|4742.57|onic pains are blithely according to the quickly ironic instruc +5902|Supplier#000005902|rb4HvSpgYH|4|14-930-257-5773|346.84| slyly among the furiousl +5903|Supplier#000005903|bp2SIwNvlgqc3fYbVJfWYkJCw|12|22-746-944-3532|8246.32| across the carefully bold packages. slyly pending +5904|Supplier#000005904|qR3xFuR1VfDt5XfudzZGdoAgS0HR7 xS|22|32-950-749-3092|4943.93|fily special foxes. furiously final +5905|Supplier#000005905|JoY1zCsAvnnKT|17|27-566-885-3848|7718.48|final, busy instructions. carefully regular requests about the carefully express packages +5906|Supplier#000005906|wDWm96T46A|20|30-933-581-1687|-420.50| requests wake carefully according to the packages. carefully bold platelets cajole slyly. sly +5907|Supplier#000005907|2,sYAOLG6IkVq|4|14-143-880-9883|3514.79|y permanent packages alongside of the slyly bold foxes was a +5908|Supplier#000005908|DMVWvjVV7LIKw,KBMeLfTbZS2|19|29-783-528-4431|1114.89|ecial pinto beans. even a +5909|Supplier#000005909|nNWQMGlzCaHxeMolpKtCuCz9ODpfDi0Se 5PA9V|21|31-222-156-7882|2983.72|es affix fluffily around the request +5910|Supplier#000005910|KCTkK928xl2KB4J9e6oWIgE9Q|17|27-105-278-6408|7291.51| about the carefully final excuses are blithely across the regula +5911|Supplier#000005911|8oYfVNd8,PAGo6JyppW|5|15-671-778-4965|6418.84|. final asymptotes serve fluffily among the furiously +5912|Supplier#000005912|lIIdxmznuo|4|14-122-400-7610|2553.40| even ideas. pending, final packages haggle furiously final excuses. careful +5913|Supplier#000005913|1zXV2kudN5trfL48z|13|23-529-455-7751|1937.65|ound the even, special requests. slyly ironic instructions cajole above the blithely ev +5914|Supplier#000005914|SAAh9XqQBEDkP|24|34-768-936-4723|3980.11|ggle fluffily. regular deposits ab +5915|Supplier#000005915|Trf9ItDwPlRYrAglba|9|19-254-653-6456|741.27|e bold, regular sentiments haggle across the ideas. quickly ironic requests bes +5916|Supplier#000005916|eTiCaIOAelGmGFFsaevI6pm|0|10-514-995-4456|4407.93|nts. blithely pending accounts wake quickly against t +5917|Supplier#000005917|Em39ZTxHfW5HMRmmYFNaYGfK05rr0SC|10|20-534-254-1886|8123.98|ges. regular instructions are care +5918|Supplier#000005918|Qq 0NBS0sOp27D1iO2gcweAf9b51UqhXMaO3V81|4|14-778-241-3496|5837.94|thely regular requests are carefully across the special packages. stealthy fox +5919|Supplier#000005919|9XqTzwy FQ0EtxZGTSJ5GH1mozW|10|20-739-829-1295|7292.72|ounts haggle furiously quickly regular acc +5920|Supplier#000005920| hMa0,1UhcitRnJP|20|30-644-157-6326|4861.28|o beans sleep requests-- furiously careful excuses wake carefully. theodolites sleep blithely fu +5921|Supplier#000005921|mT5AWmgfG9YhbOUttnn|17|27-238-166-1732|4466.80|e atop the furiously regular theodolites: carefully re +5922|Supplier#000005922|2c,Z49DJ3Ug5K92TI|5|15-440-529-2724|7505.58|lly carefully special tithes. sometimes pending accounts poach ab +5923|Supplier#000005923|gYhnZqMGHjF4TqWV cWenjHzrgNe|5|15-151-950-6154|7372.79|theodolites wake carefully regular packages. instructions of the requests +5924|Supplier#000005924|EsAX2Kl2ePben8iCc9airPaQAuA9HIr0c9A4T,K|18|28-596-222-5796|8568.78|arefully regular pinto beans use packages. carefully final packages acco +5925|Supplier#000005925|hGNSrgOZsokq1LycZz2MlvtPn4JaHhsY4OA1bvr|8|18-178-921-8905|1488.87|quests. furiously final deposits boost fluffily. requests print regular packages. fluffily ironic +5926|Supplier#000005926|c3e76 dAVh6792K|21|31-569-743-5242|7379.27|ly ironic accounts. carefully ironic deposits breach bravely furiously silent depe +5927|Supplier#000005927|wnr9EDASMFzXRXdmxi1EhEMBhcgaSt,QtA0,I7|22|32-621-372-9004|7405.21|mptotes wake slyly regular theodolites. express grouches wake blithely. bli +5928|Supplier#000005928|Vjan53HKk5tfmSaGrNMcPcYNB,SJAf5uJTYqGq|16|26-894-855-2305|3428.99|mas mold. blithely ironic accounts sleep blithely carefully pending requests. slyly even tithes nag. +5929|Supplier#000005929|1Fu0ndlDNZt|14|24-564-377-1415|97.32|fully. slyly ironic excuses cajole? slyly ironic theodolites hinder. special +5930|Supplier#000005930|MXIG6ZJHK27uwcXuT2zTHWT9WTl|9|19-247-536-8083|5592.19|er the ironic deposits. final theodolites are carefully. fluffily +5931|Supplier#000005931|TC6RpSkg5fSfi2Ika06OqHhkIxUzNCtnkMeo|16|26-567-847-1082|1952.93|telets boost slyly regular theodolites. ironic deposits are carefully furi +5932|Supplier#000005932|uSiAbsTwYJ7u46sCRY,YnPdcE|17|27-523-667-3830|7871.05|e slyly express Tiresias boost furiously final ideas. accounts wake slyly final pint +5933|Supplier#000005933| 0Y6Vzus,SVDpfR203m6EjUI7eYg A0JUwmUHt|23|33-372-776-8413|714.05|r sentiments. regular, regular requests aft +5934|Supplier#000005934|3M,bbvuCmW4iPoL7QL95z89R6M|19|29-187-727-8600|8958.83|ithely silent theodolites above the sp +5935|Supplier#000005935|vB3BeyMIWG JEMeF7,4h2|24|34-184-639-7716|1106.06|ccording to the platelets sleep in +5936|Supplier#000005936|jXaNZ6vwnEWJ2ksLZJpjtgt0bY2a3AU|22|32-644-251-7916|8569.52|. regular foxes nag carefully atop the regular, silent deposits. quickly regular packages +5937|Supplier#000005937| kYSmhyv81ptthtH5efk|16|26-965-921-5958|3045.19|r dependencies sublate carefully after the daringly regul +5938|Supplier#000005938|ZRguRXXgt5uTLe8 oU6tnew7ZSNov NdhDJyuS|22|32-843-909-3457|6476.59|riously bold dolphins are slyly unusual asym +5939|Supplier#000005939|wV1CDeRBCxVkAaD9nigOmyV5V,|18|28-527-599-7883|5115.54|s maintain fluffily; regular asymptotes haggle r +5940|Supplier#000005940|h2 ,wLZdgkwlo7KSzQ7WxRr27gv7|21|31-327-330-2818|9813.29|alongside of the blithely thin foxes +5941|Supplier#000005941|gZ6ZVsh5pccB SexPBGz1cDvgaUENxPKtnYjW|17|27-339-679-1019|7075.10| deposits boost quickly pending acco +5942|Supplier#000005942|JYvMdHg512FjvcF |22|32-408-392-8295|-828.67|ackages boost slyly about the ironic packages. stealthily ironic requests alongside of the even theo +5943|Supplier#000005943|uXgygC5PfmPGESiO4IzkYj5i|7|17-119-172-1665|7500.44|uests alongside of the even accou +5944|Supplier#000005944|dwXpvLKEvm7hUbiZzTGMrhIiWwXSJIJeJALrNqf|1|11-301-158-8313|8947.06|usly silent, regular asymptotes. furiously silent ideas according to the carefully regular e +5945|Supplier#000005945|GgfC6umxoFXOiudVE|7|17-737-931-2581|1375.01|ithely special deposits use about the carefully regular deposits. blithely +5946|Supplier#000005946|l jKrqXCWzhmCRZI46nTLQX|7|17-363-366-2693|7788.05|uickly. furiously final frets grow ruthlessly. special, unusual instructio +5947|Supplier#000005947|LBwj sevn,um69MwZOycXrw,9MV6|13|23-173-428-8983|2882.61|the slyly special ideas. instructions about the blithely regular dependencies are across the blithe +5948|Supplier#000005948|,AVsj0Dbver|0|10-595-981-6728|1939.64|y unusual dolphins. regular packages around the ironic ideas wake furiously slyly regular +5949|Supplier#000005949|01sG2UA6athk798u6nwhBdwi9W5g9q,S,wONC6B|10|20-126-905-8626|4438.84|wake slyly blithely unusual packages; quickly regular foxes haggle beyond the carefully final c +5950|Supplier#000005950|0mWo1zpc8UfD8Lm|24|34-523-671-4859|-377.58|ers. furiously regular deposits nag into the slow sheaves. dependencies detect about th +5951|Supplier#000005951|vFLo9z,wj5W7eY7GwZ7uJRPL00A98mO9gbRD|4|14-698-491-1588|9877.79|xpress pearls haggle quickly. +5952|Supplier#000005952|VCieisiHXBo0ricf|8|18-631-188-1701|2145.18|counts. blithely regular theodolites cajole sly +5953|Supplier#000005953|195TTIj0ef3CNNBPdE8 ypE,ptuHgpUZIfKo8uP8|7|17-738-357-6501|4972.09| carefully unusual asymptotes promise packages. slyly ironic orbit +5954|Supplier#000005954|wOwcbbHNV2p,Glt8SijFnE9ZLcTbnhESeLvyFN|9|19-145-589-1532|7841.05|fully regular foxes nag carefully! even ideas cajole permanently special deposits. carefull +5955|Supplier#000005955|cOXdOORzpk|9|19-647-712-9789|2617.31|oost carefully slyly even dependencies. requests haggle carefully bold theodolites. pinto bea +5956|Supplier#000005956|iFVGvHqPU5lu8PiOXgtTky|21|31-490-709-5774|9026.09|even ideas cajole alongside +5957|Supplier#000005957|whhWEWU7QbD6TJGw5hfzcoTfZJOnuXP|12|22-251-381-2506|9590.47|y even packages boost alon +5958|Supplier#000005958|tsrssKVbGILzF6DWtfiJCcVrTg65g|4|14-893-866-4796|2143.64| pending, final deposits; busy, final asymptotes cajole carefully bl +5959|Supplier#000005959|wAhL8S1bqYV9j6 OxbgNTWn3SNsH2n|17|27-580-605-9781|1038.32|unusual theodolites cajole furio +5960|Supplier#000005960|Wenk3uGdxo,ZefVh3XCR8iYaNxmyzr3B3i|1|11-806-364-9633|2964.75|ly express dependencies boost slyly requests. +5961|Supplier#000005961|IWgRlEcGotZr3TViHAe|11|21-440-415-7794|7597.46|ronic asymptotes detect furiously. even packages nag blithely alongside of the slyly reg +5962|Supplier#000005962|KfUpiOuf8e|14|24-748-909-5232|602.95| somas. slyly bold pinto beans sleep slyly. regular, regular multipliers after the speci +5963|Supplier#000005963|twKG0NtM8vLJ0RThINUKmFcOMZ|1|11-896-628-1373|761.90|ronic pinto beans hang slyly enticing requests. +5964|Supplier#000005964|lo8SzMQEUybrAqiU4,p|24|34-686-209-9654|-774.55|s. pending accounts nod quickly. regular foxes +5965|Supplier#000005965|RXgGPHMpLdsK31ygxrY3AQR5U5RI3M6UhX6|9|19-146-379-9429|4116.24|structions are blithely. quickly even foxes past t +5966|Supplier#000005966|SuFErYNcjqywl5DncfjhV8OViSqBbfnYlmKS|8|18-561-292-3628|684.35|gouts. instructions boost +5967|Supplier#000005967|bu6rE6Eq0cHTpk1eFOpl9GidHUvZ|21|31-470-623-2505|4855.23|ial packages sleep blithely after the slyl +5968|Supplier#000005968|7vKwZe im8|15|25-589-188-8236|1835.25|leep. blithely express deposits thrash busily above the regular ideas. final pa +5969|Supplier#000005969|ANDENSOSmk,miq23Xfb5RWt6dvUcvt6Qa|19|29-520-692-3537|9937.84|efully express instructions. regular requests against the slyly fin +5970|Supplier#000005970|eMgvfIHEHqVgpV3MLy0s2aODiSNbioK|5|15-503-953-8421|-641.69|rding to the carefully special depths. furiously even re +5971|Supplier#000005971|9NcBom50Cnz1V63qhFKaBl,5FWYXWa|23|33-389-406-9230|4106.75|ages kindle carefully fluffily silent instr +5972|Supplier#000005972|IgZlzEujhv|1|11-832-540-3456|6156.02|o beans. regular notornis haggle furiously accounts. slyly ironic instructions integrate carefully +5973|Supplier#000005973|sp994hdQ2Z3 HypX0I B6|7|17-308-120-2714|7242.88|o beans. blithely unusual requests sleep blithely. fluffily final +5974|Supplier#000005974|REhR5jE,lLusQXvf54SwYySgsSSVFhu|3|13-472-800-1755|3086.60|theodolites. regular accounts use quickly alongside of the final, ironic co +5975|Supplier#000005975|S6mIiCTx82z7lV|7|17-992-579-4839|9492.79|arefully pending accounts. blithely regular excuses boost carefully carefully ironic p +5976|Supplier#000005976|GPvme3vYZQ,Px4E3quf Ug9VrprK|22|32-285-350-7599|-659.44|sts boost. dependencies are across the blithely final dinos. e +5977|Supplier#000005977|V8rEW7GYlWJM3S4IIO0nmLAclj6SetYsjCEYAlAg|24|34-138-830-6930|2669.78|osits unwind. furiously p +5978|Supplier#000005978|19sI2 ,TFEZpZsQQdo8c,LqfE|10|20-463-628-7425|44.09|ep even courts. quietly final +5979|Supplier#000005979|iIIPTg3efrljRNU5zdi|22|32-421-263-2851|2018.48|r requests cajole blithely even foxes. ruthlessly even accounts engage slyly fluffily special requ +5980|Supplier#000005980|TQnqbXl1GVlMAISOLXpAyFMz1k3k7quGLyRAX ty|19|29-584-733-8238|9556.01|nts boost carefully about the slyly even packages. regular packag +5981|Supplier#000005981|HSzmPWGPAgHMTJX7j00Y8dqe|2|12-984-985-6346|5929.95|fully regular packages boost carefully. slyly quick excuses doubt. +5982|Supplier#000005982| AL5c9zkqXQ5ma U hKoUwGLB|23|33-524-438-2827|2754.16|e slyly alongside of the bravely final platelets. even theodolites over the sly +5983|Supplier#000005983|Gh1XnJt4Oi|10|20-130-730-2808|-785.28| gifts. slyly even instruc +5984|Supplier#000005984|jopt 5IVOze2jYmlmFhW7rd8rD,8Y8tLV|13|23-189-979-2184|-230.47|al instructions. carefully ironic ideas are according to the unusual, expre +5985|Supplier#000005985|u7SE Tr79IsDD,PvVi|11|21-370-773-6403|7477.87|c packages. accounts grow slyly bold fo +5986|Supplier#000005986|3fTp1qsQZN9lR|13|23-138-282-3853|3525.78|ies. unusual, express deposits kindle quickly. pinto beans boost quickly. reg +5987|Supplier#000005987|TzZR8IiWBt tEspR9hcNACKHOk|1|11-688-883-4586|9821.11|kly ironic packages alongside of the packages are quickly bold deposits +5988|Supplier#000005988|IreOjLGbjMODbx |13|23-145-539-9356|6552.79|quests haggle furiously according to the furiously bold excuses. blithely ironic foxes sle +5989|Supplier#000005989|rjFY,5kgLpBu7c|3|13-151-500-7938|9697.97|cial requests. blithely ironic waters detect slyly. ruthless court +5990|Supplier#000005990|HuBcD,34d5MsLAEUphmVRTzPJn6yXHXqOuc lzhI|23|33-511-757-5149|4857.31|nic pinto beans. express theodolites thrash caref +5991|Supplier#000005991|jFAkBBG3JDqoH8nIXUZGn OzkVVfW6YsNh|13|23-147-208-2590|3739.94|quickly unusual theodolites. blithely regular excuses for the +5992|Supplier#000005992|fmaO4H3jynO4HVK6cvctYLtiVakw67npbgY|8|18-891-604-5329|1214.58| slyly final accounts boost after the f +5993|Supplier#000005993|gIEusATBqQyzh|2|12-377-997-4110|6445.08|beans. deposits wake blithely regular +5994|Supplier#000005994|d9VW,OP6ydzlhJS36Kao7HrlGTSrq99zHd|14|24-519-922-3883|3491.71|ns. fluffily bold packages wake furiously. carefully ironic +5995|Supplier#000005995|rXSzCk rTYRWsfJ|18|28-637-534-9421|9233.88|y ironic courts cajole regular, bold dolphins. blithely express pinto beans are +5996|Supplier#000005996|tfOVqjUrB7TTxCZ,OsPZZynV,C7wYT21HYu9I,Mp|14|24-604-411-5147|3814.64|t furiously across the carefully unusual foxes. regular, regular +5997|Supplier#000005997|x7icJA6OEr,j3GYkYNN|20|30-983-581-7749|2617.74|its boost slyly. pinto beans boost sometimes regular requests. final pl +5998|Supplier#000005998|qEAze72pOz8,QNcx|20|30-145-908-9029|5381.72|ts cajole. regular deposits haggle slyly final ideas +5999|Supplier#000005999|vy d AZoRC0QopXCb67K8ciDL3BG|0|10-264-460-2285|6512.20|ions are. bravely silent theodolites dazzle according to the blithely regular accou +6000|Supplier#000006000|iyk7ceSNxmt|21|31-542-384-1537|9561.30|gside of the regular, ruthless waters. blithe dependencies cajole blithely across the carefully pen +6001|Supplier#000006001|6S RG2sY99qt8Am ZugMI1,Rvcbl|22|32-686-776-1548|6823.45|ts sleep carefully. carefully even instru +6002|Supplier#000006002|M9DcHXRp4DbiCysFVhV5rP0SbvO4TrN7GQyEv7|4|14-701-105-2127|4785.60|p. carefully furious dependencies nag. +6003|Supplier#000006003|EH9wADcEiuenM0NR08zDwMidw,52Y2RyILEiA|23|33-416-807-5206|8607.69|ar, pending accounts. pending depende +6004|Supplier#000006004|pd34ETIJBB2pgR4Tyra44yv7fa|16|26-959-736-5551|6532.54|y final pinto beans wake +6005|Supplier#000006005|OUS5xoYCrdAZyqh|15|25-941-518-2602|-555.08|y unusual patterns sleep requests. thinly +6006|Supplier#000006006|tRq3yfXg0LlBUwlEYZQNHWFEZ3tw9Mn|19|29-164-563-4583|7631.77|r instructions would cajole. accounts according to the fluffily even instructions sleep slyly +6007|Supplier#000006007|BVT8yBgwqcl1 d49SmJ5X DQgw|9|19-428-244-8363|5169.18|ly across the slyly unusual requests. furiously even theodolites about the deposits +6008|Supplier#000006008|y0BjyPzky7zQ456w38UuA4eNG|19|29-164-591-2533|9495.38|against the regular deposits sleep fu +6009|Supplier#000006009|hlVgR6c 6jgsw9xaGnB7pr4jI oNKFmF|17|27-131-492-7118|9851.93|ly final requests. pending courts sleep blithely furiously regular depo +6010|Supplier#000006010|TYCRe0lBUFq4i0XF,29JF6oM9mgVL5X9o|19|29-735-426-5698|7446.75| even requests. carefully bold dolphins doubt quickly. even dep +6011|Supplier#000006011|kHzD2,R5QDSF|10|20-656-493-7027|8570.49|uses cajole quickly across the dolphins. blithely regular accounts +6012|Supplier#000006012|vef8Lme m3HOJ2rHo4iFhuJYJ7sbSjJn1t,CH|12|22-907-548-8362|8329.46|ts-- ruthless, regular ideas wake blithely ironic courts. packages cajole f +6013|Supplier#000006013|,,Jv41XwKEEUINXqSWOFoxiJH5Z|20|30-513-146-6860|5363.16|thely express requests. busy accounts ca +6014|Supplier#000006014|ul4swfQuRFt7G|16|26-231-513-3169|2575.80|nments are about the finally final epitaphs. fluffily ironic packages +6015|Supplier#000006015|gD r5B94LbZJMhWOezrUjy01NUeJOB4C7NoHSvHh|18|28-913-465-6684|4390.89|use carefully even accounts. qui +6016|Supplier#000006016|OmiSL2cwQ6YGQncYNAj8WZAFgz|6|16-757-121-2301|7669.38|ffily even pinto beans grow ruthlessly pac +6017|Supplier#000006017|IUor4VqlDPJqahcVWH6KpmA29cbnu5X0SnD6|15|25-309-274-1235|-763.43|e after the even instructions. carefully silent deposits sleep. +6018|Supplier#000006018|kZ1m1FQgw02zR|5|15-400-598-7522|3951.13|slyly silent, even accounts. deposits +6019|Supplier#000006019|tH9NeIeSfkGMPLh3MsOH9wh7Wu|22|32-583-180-7326|8690.26|unusual, special deposits eat quickly about the unusual accounts. fin +6020|Supplier#000006020|U9O4ceIDl0DMRPM0vvsKKj3gzGwNkcUeusC|10|20-817-604-6561|8330.85|ly? final packages are blithely ironic ideas. carefully special packages kindle blithely expre +6021|Supplier#000006021|qKVSqH8RyigVvRXjUUQbA|11|21-416-861-2634|6688.48|deas sublate around the packages. requests sleep after the ironic, final accounts. quickly +6022|Supplier#000006022|LUbOrHuCVuEIDlOI|18|28-483-926-6834|2354.23|phs. foxes sleep furiously with the accounts. regular dolphi +6023|Supplier#000006023|QzUNYNJKE1|13|23-467-863-9965|9468.49|efully slyly pending hockey players. instructions nag blithely sp +6024|Supplier#000006024|oRpraJFkxy ptVsFMwZW31ehtiLMSkcGVF3|18|28-307-686-2668|9048.70|unts are slyly regular dolphins. caref +6025|Supplier#000006025|XjLwtxC0K4ee5blGBn8h36GJHjm0Sk5,tXT|6|16-124-250-1297|4605.35|ual platelets. quickly ironic instructions sublate quickly bold deposits! pinto beans believ +6026|Supplier#000006026|GJnwCYGwSA FQDk,VV LtjlIKY|8|18-852-661-1445|4181.07|ests haggle pending theodolites. carefully final packages haggle quickly blit +6027|Supplier#000006027|uyjhxXMYx7Jf3CDL5i5CQ9xhs|20|30-990-840-4169|1159.25|ctions detect about the fluffily +6028|Supplier#000006028|jKQ8XFunoQPhOYh|12|22-254-897-8438|468.64|arefully ironic, bold fox +6029|Supplier#000006029|v2zM32stD8DvFa|1|11-488-306-3669|9410.91|lithely bold accounts nag enticingly. packages sleep furiously slyly final accoun +6030|Supplier#000006030|FZwls8Wy7mYrj netgHZS4E|1|11-166-588-6851|9261.33|ss, even requests sleep bl +6031|Supplier#000006031|pyc4JtJqTf|2|12-621-978-2294|3268.65| instead of the packages cajole final pinto beans. express, ironic packages wake carefully against t +6032|Supplier#000006032|gHchBFAQrJ kOlgborod 5AY9NZjma6b8pE|21|31-211-220-3819|152.62| cajole. slyly unusual pinto beans are idly. furiously final deposits wake after the un +6033|Supplier#000006033|ncdx8UR f8m09oNuQT x nu9uwEX5a|13|23-886-649-9731|5.61|tructions along the furiously express requests nag about the furiously even platelets. special, e +6034|Supplier#000006034|znWbAlMzjvrWGm9m,,|7|17-548-215-7100|4205.25|y fluffily regular instructions. foxes doubt slyly. carefully ironic requests na +6035|Supplier#000006035|bhR8kr2kzvAV|10|20-860-448-9043|596.72|fluffily unusual packages. slyly e +6036|Supplier#000006036|wYdBw5ugskHp9E8g0Sz|15|25-380-824-8807|5969.54|dolites. furiously expres +6037|Supplier#000006037|lb2Pbs9ECdqIEBApdgj91LF5v6jB|8|18-704-875-5171|8148.90|es. carefully final accounts nag slyly according +6038|Supplier#000006038|y958tTP60Qh4A|15|25-996-556-1165|6558.31|equests among the always regular forges wake +6039|Supplier#000006039|qKmsFDDVnXEl0CHV|1|11-694-980-5971|3486.79|ress requests. furiously unusual packages use quickly according to the final pinto beans. furious +6040|Supplier#000006040|y,nMWLIFjz6qhR1Rqnlji,8Uf|0|10-953-490-2366|3482.05|packages wake slyly regular deposits. blithely regular asymptotes +6041|Supplier#000006041|iv0m8vViqb0SVsBBH|7|17-203-709-1420|747.07| express packages across the furiously final in +6042|Supplier#000006042|CDRx,PoyijFwu4E 6hDmM,ezDtE8SXXPW,|21|31-432-590-1324|2945.31|ross the slyly pending dependencies. requests cajole slyly along the attainments. specia +6043|Supplier#000006043|KtgDpDfIWq22vBQB2cv|2|12-838-305-3827|-276.77|y regular packages. final dependencies sleep blithely according to the accounts. blithely exp +6044|Supplier#000006044|OngloI0oZQ hujfKtiO26 XQF|24|34-645-906-4179|-692.95|. blithely special epitaphs h +6045|Supplier#000006045|sUJ3CVPHg5y|11|21-957-372-3633|487.20|efully ironic asymptotes. deposit +6046|Supplier#000006046|tIu3fhRLQqQqSJsE8zrn|16|26-481-166-9872|2267.52|y express ideas. fluffily regular deposits along +6047|Supplier#000006047|kzwT4VvnHjttVeDCR9q b3zhgvoibLskr8 B|19|29-898-747-7501|4140.92|furiously around the furiously +6048|Supplier#000006048|OyHRSoJAGXxmO,qdDs6EEWZ7s|11|21-547-295-8299|8515.25|g the regular waters. carefully even r +6049|Supplier#000006049|9 ZuCU120T2cXkg7XS9GwC91J|1|11-991-560-6553|3846.26|must play furiously silent ideas. asymptotes use. quickly pen +6050|Supplier#000006050|WAbDi 8gXQkhTLxSp79Grcl|18|28-944-838-8126|6034.65|olites! bold requests believe slyly bold deposits. ironic ideas sleep slyly. unusual, even +6051|Supplier#000006051|d6Q ULNKOR65Rbfx6NBKbvaKMt ,FoHCVr2|5|15-412-970-8192|2320.70|ly ironic deposits wake above the blithely ironic dinos. quickly express pint +6052|Supplier#000006052|04g0w4n7VxqqRy eBd,W0rIbla5c5RLLtC|9|19-378-257-1098|6996.82|. final, express foxes ea +6053|Supplier#000006053|yT8f1YCOAJuw0o|2|12-374-539-8688|3529.31| beans wake fluffily express pinto beans. blithely express requests poac +6054|Supplier#000006054|J8eAsovizNKTFchDunimO4FjM|5|15-273-681-5978|3178.41|ests. quietly pending requests sleep slyly accounts. final requests try to s +6055|Supplier#000006055|j666uJusGi4A54fKl4lV1uC|0|10-911-168-8946|1521.73|furiously unusual theodolites wake. carefully bol +6056|Supplier#000006056|GoJkowKdxFsgPvTom6kGUarfGRg6DvwiL|9|19-167-423-3098|9719.33|ctions affix furiously furiously bold accounts. furiously +6057|Supplier#000006057|,KYlXBZ1bHc1qP4BUEsulAi,R1o6EBiS99|23|33-399-998-2497|834.70|ole fluffily express, unusual pinto beans. +6058|Supplier#000006058|t9uuW6i2,ZiwrLrk9S6bGRHWRC|22|32-860-405-2238|1782.87|et pinto beans haggle across the b +6059|Supplier#000006059|4m0cv8MwJ9yX2vlwI Z|3|13-203-160-5573|9583.71|e requests. regular accounts integrate slyly. quick accounts mold. ironic grouches detect +6060|Supplier#000006060|eEEAT295rCwR3zTVVq8,|4|14-337-767-7616|9469.45|al foxes haggle furiously according +6061|Supplier#000006061|xuzhMVdyWT|18|28-497-195-5649|3119.05|al asymptotes wake slyly unusual instructions. blithely f +6062|Supplier#000006062|x5lqzuyqnnDteV5UT1HY0X2 Qjbo,c|10|20-874-712-8366|4906.48|t the carefully express depths. fin +6063|Supplier#000006063|OQ8asgn nF5WiejUZkbfh3cOCJOHxO9o|4|14-571-321-2378|5169.84|c theodolites lose furiously according to +6064|Supplier#000006064|15WVNNJxLfwE2FgoMutg6PRW|1|11-581-536-8932|9535.48|o the furiously regular packages haggle blithe +6065|Supplier#000006065|UiI2Cy3W4Tu5sLk LuvXLRy6KihlGv|3|13-920-103-1401|1870.49|ully final deposits are. express dependenci +6066|Supplier#000006066|LZCyQOpfMn|23|33-968-534-4343|8226.79|ggle. slyly even ideas alongside of the thin, unusual packages +6067|Supplier#000006067|HCuEY0f8DnTNnaG5yJJLr|8|18-172-522-2460|6747.44|en requests. quickly even deposits haggle carefully; carefully eve +6068|Supplier#000006068|bToNEQv4zKzQ1NypZj00YjzYhrm94W2,sOYruF0|22|32-155-311-9768|3244.81|nal accounts. silent forges against +6069|Supplier#000006069|v8dIXRVHj36WcUF4RKLjoRq Bt X6B 68cFU|12|22-946-698-4046|6541.21|lites after the requests wake carefully regular instructions. furiously even d +6070|Supplier#000006070|TalC5m0pDrO6DZbngfmGmqe|3|13-850-134-1308|2075.72|efully express accounts x-ray slyly even +6071|Supplier#000006071|ojOm7rfzVPqx XWyF0rvU3wbJ|18|28-951-903-3383|3647.58|efully express pinto beans wake blithel +6072|Supplier#000006072|5p28ZdH7HLxFDniwyOSXNSB,yZK1U|10|20-814-761-9484|-120.95|y express deposits. quickly final deposits sleep fluffily. foxes haggle thinly +6073|Supplier#000006073|t rFESd1J5EGq1TDbV5X4F0NO5 za|15|25-589-567-8857|141.68|ic accounts sleep carefully above the blithely special deposits. ironic ideas are furiou +6074|Supplier#000006074|ISALQ7N4EGwy2CKTvMbQXkEOBU TE4HSr8ju|23|33-912-195-4358|3103.20|lent, express requests nag slyly according to the furiously regular requests +6075|Supplier#000006075|SaqEkAoon6kBs4BHleNpr7wjkxBUmdVTIL|24|34-974-382-9192|2487.78|s instructions. regular, final pinto beans haggle regul +6076|Supplier#000006076|EFMuwlPO4NXmL3dLVjgl WCuJH3|5|15-665-754-7994|-853.92| final forges? furiously even instructions wake slyly along the even, +6077|Supplier#000006077|RkI7p9xWfSyLWmT3x5CewQCpV1riHuivOp0|24|34-891-497-4142|1110.19| ironic packages doubt blithely around the blithely regular pinto beans. slyly express requ +6078|Supplier#000006078|g2U3DE6YRlyypDiEMUaKuzqa|10|20-162-511-5277|887.01|sleep fluffily around the +6079|Supplier#000006079|7Zz2wqr8P0V3hh|8|18-230-751-4134|922.53| deposits: slyly regular dependenci +6080|Supplier#000006080|aLvjFiysSAnKIB375JQGV7,nDTvP1Oxgu1zG|2|12-345-557-1296|3833.22|hely ironic deposits. special, regular theodolites along the furiously pending t +6081|Supplier#000006081|t9BY2txVHYFZ XtK11jfKU|5|15-109-845-1791|6890.07| dolphins. pending deposits use qu +6082|Supplier#000006082|ziRmQdcZa3QENhR364dVXMfJXRX2Nk|23|33-685-737-1893|2417.08|ose asymptotes boost carefully carefully final requests. furi +6083|Supplier#000006083|THtqkIcNIM8wApJuIk71hwJfFK|23|33-949-571-8118|9757.91|nding courts can use quickly ab +6084|Supplier#000006084|gv,vfg,ApqCQ37Sy2O0v3Gsr|3|13-672-183-7515|291.31|s. express accounts run furiousl +6085|Supplier#000006085|T3Ju68MUhIb2hpTO3f8OGG|19|29-528-113-9241|9915.38|iously bold sauternes. slyly regular asymptotes +6086|Supplier#000006086|p0WNQ0dxAfHnrN55EAMYvr6kEnWq t5|24|34-754-909-8718|6794.07|egular, unusual deposits. regul +6087|Supplier#000006087|BPDYkTeaG6er4cLOl5uMZysc|20|30-229-930-8440|5970.77|p. special, ironic instructions cajole furiously after the regular packages. fluffily +6088|Supplier#000006088|PSHhyqTMumGuiYHR3|2|12-367-961-8250|6707.20| alongside of the carefully final deposits lose along the regular accounts. blithely even packages +6089|Supplier#000006089|ckjiKJQneKDwr|17|27-787-635-6738|9020.20|al foxes. fluffily even deposits through the regularly bold +6090|Supplier#000006090|QOutX3sX5ybnB27RkdDWXCs3|6|16-152-633-9427|9547.13|packages boost. furiously express instructions wake final, bold deposits. blithely even instruct +6091|Supplier#000006091|A3oxMRbjPOWTaqGq|7|17-370-426-5206|-351.41|ly bold theodolites. unusual requests according +6092|Supplier#000006092|zu,8R7AJlO0PgWe144XHxWUehSOTexqMUDcbRpr|6|16-567-946-3590|9726.30|, regular packages. final deposits sleep fluffily regular deposits. +6093|Supplier#000006093|KJNUg1odUT2wtCS2s6PrH3D6fd|3|13-989-796-2627|7935.95|ts solve slyly slyly final requests. slyly regular accounts haggle regul +6094|Supplier#000006094|wq6EZr3njrSo0bUx50jU4cBWH14CEmNd|0|10-455-358-9844|2521.10|. slyly special accounts cajole. even, unus +6095|Supplier#000006095|dj8mhVnCtaxiQeC0yecDS|17|27-134-165-2408|9145.31|refully pending accounts. ironic, regular pi +6096|Supplier#000006096|hcaZGirqBgrKJAnz8BlQt9ehVWikq|18|28-201-798-4437|286.05|ideas about the furiously final forges +6097|Supplier#000006097|zNHypsRcwCwYDeUC|12|22-955-529-5998|9566.57|ld theodolites. ironic requests na +6098|Supplier#000006098|PMFvhiKYqjx4ab|17|27-525-181-6083|7398.07|een the ironic accounts. express theodolites boost slyly blithely final accounts. quick, bold theod +6099|Supplier#000006099|aZilwQKYDTVPoK|3|13-801-926-2087|4441.40|ly furiously even accounts. deposits about the quickly permanent +6100|Supplier#000006100|P5P1ly2xmfWauZx4b3AsuWa 5qb5|16|26-484-811-2316|9617.42|efully. bold packages according to the accounts cajole slyly carefully final realms. ironic +6101|Supplier#000006101|, qQhoVymyPSeFeagvCPLqb0J2brRLRnGp ,E,c|1|11-262-814-1479|4054.58|egular packages play furiously final deposits. even, bold ins +6102|Supplier#000006102|APhYFpQ5j2m|1|11-580-909-9223|9343.63|e ironically final platelets. blithely even sentiments nag carefully ironic theodolites. gifts sleep +6103|Supplier#000006103|l32l8iaPdbHgRXoq,kdjFAj3hZk2d|21|31-282-540-6922|464.68|grate bravely ideas. regular asympto +6104|Supplier#000006104|lMYW6r9xqIlGLRe,N0Ee|15|25-138-199-2897|8731.40|efully ironic ideas alongside of the carefully special +6105|Supplier#000006105|MK6GVEuCTdTRr7fy4 EJexX0odsVkWY|18|28-971-431-9387|6773.95|ng, pending packages boost after the furiously express packages. carefully pending theodolite +6106|Supplier#000006106|l5aC,I9W84DdKA394vhGFEQPX6eeQnYOjE|8|18-460-975-7871|8753.13|r ideas. blithely final pinto beans us +6107|Supplier#000006107|TLGzIjsKUQ EIpec7Xy2f|2|12-396-588-6095|8908.84|lyly pending instructions u +6108|Supplier#000006108|8tBydnTDwUqfBfFV4l3|6|16-974-998-8937|7992.40| ironic ideas? fluffily even instructions wake. blithel +6109|Supplier#000006109|rY5gbfh3dKHnylcQUTPGCwnbe|3|13-232-509-4334|5343.69|the slyly unusual platelets. always ironic accounts mold blithely. requests use slyly final +6110|Supplier#000006110|taGUPVJ5RlSFvrKyKKpts,C8k5T2Z j|12|22-365-553-8610|5429.17|ole after the bold accounts. slyly regular dependencies detect furiously across +6111|Supplier#000006111|SK7FPYUBox40Vg5Xx Kq97by25bXjsQ40vT|14|24-937-416-3863|1053.81|unusual, enticing ideas wake quickly blithely silent instructions. +6112|Supplier#000006112|UCt8ExSjrI1lnZu Ncwb|19|29-357-991-6874|-692.96|l pinto beans. ironic, regular accounts hinder slyly about the thinly even asy +6113|Supplier#000006113|fvnedZO9VSceJB8G|11|21-910-536-9332|419.44|ding orbits are carefully. carefully final theodolites boost furiously above the ca +6114|Supplier#000006114|6482 iaFjFugM g,kq3DhWWRn1r|8|18-416-950-9698|7544.88|nts play carefully. slyly regular packages w +6115|Supplier#000006115|kc7CCSyKhgkPCsz|4|14-778-204-8677|7865.65|iously regular packages cajol +6116|Supplier#000006116|IC,0fXQh3eNQC2,xrsH9p4VtRH|7|17-455-882-1752|8981.06|ven theodolites nag carefully quiet requests. platelets wake furiously +6117|Supplier#000006117|EtoCxmTc9IKNvlbjA0gVvh092, IBaRJ9bX|15|25-412-667-1819|680.29|ckages use fluffily. slyly even +6118|Supplier#000006118|GN3E6y4Jy8YaBLle,2Tcm9 oiQigXL7VYqv8Zz|2|12-949-371-2403|2504.97|heodolites. slyly even dependencies detect bravely according to th +6119|Supplier#000006119|ykqu5mM1iQrN4f|14|24-150-341-6915|2055.46|carefully. blithe, special courts nag slyly slyly regular escapades. slowly silent pac +6120|Supplier#000006120|byBcsYEkrMZLidZ7BUVkUv5GSntSLjZ WO57rAy|8|18-409-286-3099|4064.97|e slyly final forges wake toward the regular, pending requests. furiously pending foxes are care +6121|Supplier#000006121|S92ycWwEzYYw4GspCBJN1WMuHhoZ|3|13-279-149-8538|7860.33|le furiously across the regular packag +6122|Supplier#000006122|ZNTXj7qdCq n2NJD|10|20-246-708-8603|7542.01|s sleep along the carefully s +6123|Supplier#000006123|2bhyiEp bAZB841GyKL 1f|14|24-973-888-4782|3544.25|f the furiously regular deposits. thin pinto beans haggle slyly according to the +6124|Supplier#000006124|KsV6PYudv3JASdU85i|8|18-876-513-4459|1131.80|ar requests. even deposits wake pending instructions. fluffily unusual excuses was slyly ab +6125|Supplier#000006125|qss,ZNIC67LLACrk,xIU,|8|18-410-972-6157|1167.70|thely slyly even requests. platelets wake according to the enticing excuses. flu +6126|Supplier#000006126|YzSYcbXZ9bj0FKFGeM,8nd2py|19|29-917-807-3166|-507.89| ironic, bold foxes. furiously special platelets haggle furiously +6127|Supplier#000006127|nnt,r9fwTQK|1|11-630-893-3735|8766.33|ously slyly special hockey players. furiously regular hockey +6128|Supplier#000006128|aa9kaYp8MKT0nmfkeAQTwnh2,efLuIz,G|20|30-646-311-6358|-145.12|luffily regular accounts. carefully blithe requests use furiously brave accounts. carefully stealth +6129|Supplier#000006129|SeYweH 5LLy, 7 y0C|6|16-373-131-7534|978.84|ven somas. furiously final pinto beans breach according to +6130|Supplier#000006130|1yf,zRSF7QiHHbMAcw5SZN4a6PYwM,Wu68|11|21-563-250-9287|4263.28| are slyly. pending orbits haggle carefully. carefully special ideas doubt! final pack +6131|Supplier#000006131|00JMmsG0Cq|10|20-579-721-3330|7940.55|fully according to the carefully regular deposits +6132|Supplier#000006132|lOZaYI6nNeaGthbWyJ|1|11-732-590-2065|2882.23|eposits. slyly special depo +6133|Supplier#000006133|laVL18hHUi3|18|28-687-301-8431|6694.40|ly special theodolites. packages doubt quickly. even requests wake slyly after the regular, final pa +6134|Supplier#000006134|CJhdo9NAF6 w|4|14-365-815-2475|2835.43|ular dolphins are according to the blithely regular deposits. de +6135|Supplier#000006135|GEKtkBZ cfK5J2LClxlqd JA90vxCiBog|2|12-734-730-7167|976.65|al hockey players. blithely final frets run above the blithely even acco +6136|Supplier#000006136|H5tDfi,XJ8BuciyUcOao1WXbXOWIGBR|19|29-672-813-5545|456.08|al excuses. carefully permanent asymptotes haggle. +6137|Supplier#000006137|KZYZhX6y6KBg9Cs yzLG7K65sAdXG0FIfhjQktQ|24|34-285-113-7777|7274.43| accounts nag blithely slyly regular asymptotes. sly +6138|Supplier#000006138|y6un5si3ymHurd8n2mGZ0w3|11|21-795-823-6659|5453.29|ggle blithely. quickly ironic packages wak +6139|Supplier#000006139|jB7XkfKoKSdWr9YM9m4UEkul f|17|27-976-769-5223|-492.47| sleep about the fluffily even +6140|Supplier#000006140|93EycC3P9tiKu5XEkgb3duqG0|0|10-779-683-2433|3902.59|ent foxes. quietly unusual requests sleep carefully from the doggedly express instructions. bli +6141|Supplier#000006141|fAodvT2pNOMz,eRQ49lr5Kh2dcuu,gT08rdoas|13|23-595-486-7294|6835.24|furiously along the dogged instructions. qui +6142|Supplier#000006142|RMMW8ECkdaleNxLL9ecTfcyohxnUV1toa|5|15-359-277-9696|4896.22|heodolites. platelets boost across the quickly express f +6143|Supplier#000006143|WSDWjg9NFitDnmxvlYnUkNkH27x 66vZLvCDc|16|26-316-273-3393|8840.35|sits. blithely final deposits sleep across the furiously regula +6144|Supplier#000006144|gg3mh2VuLQeep4fRmxWa2iPPkENE|6|16-552-980-2565|797.65|r requests. regular requests wake quickly carefully final requests. a +6145|Supplier#000006145| YN8CD22,RU3,LytJj0JMsbXi7DGyU2U|24|34-851-907-4799|6054.62|orbits sublate furiously carefully special t +6146|Supplier#000006146|0cUtbW VknxJPyQn3x4cv|15|25-650-920-1970|7986.71|lithely final instructions a +6147|Supplier#000006147|ZBR7JPWLlgCKCA3QJ5DcVRaU|11|21-761-179-7642|8661.64|refully above the ironic, si +6148|Supplier#000006148|IHxniP,wiwlHinuOUQIRKEqB0AO3xD56rJ1,|7|17-170-603-1321|309.53|lar packages. silent platelets detect; ironic pac +6149|Supplier#000006149|ovulwnjZUCBbdoEtRsIj|19|29-682-230-5166|8137.99|eodolites boost furiously across the express deposits. blithely pending +6150|Supplier#000006150|hP46tthRqBUuuykAM0OEIV|16|26-320-415-9891|-828.50| silent packages boost carefully. final deposits boost slyly. instructions cajole furiously blithely +6151|Supplier#000006151|onrtIjRkYat7WIJeIx,tMMDHl5 z8YTjlO|24|34-641-785-4655|-652.73|tect. pending orbits sleep carefully regular instructions: unusual account +6152|Supplier#000006152|NiT9RiPbusNmbMVsm9t87xx2u|13|23-301-415-3899|5554.43|nal packages. quickly regular foxes across the quickly +6153|Supplier#000006153|x4 ZCQ4tNubRtMjG Ukz5qar5Tu4awO27vP|4|14-357-692-7678|5299.75|usual requests use slyly of the idly pending foxes. fl +6154|Supplier#000006154|vFpt0fIvZDCbtTOCihSJkpGtP|22|32-333-538-3997|3856.24|iously regular deposits nag furiously above the sometimes final accounts +6155|Supplier#000006155|HN5teSnZrU4|18|28-456-993-6763|8899.68| ironic packages cajole ironic asymptotes. pending accounts +6156|Supplier#000006156|3E BnFXjwiYoqmzq0eHiB|17|27-312-123-1541|7905.74|o beans sleep doggedly ironic packages. slyly unusual packages dazzle fu +6157|Supplier#000006157|bWwHUuQgVo698rHdr9S7tX2czhAepL3Lp4MU1m6W|17|27-613-662-3540|823.90|es. accounts along the slyly bold deposi +6158|Supplier#000006158| g3gnmaM7TX3h,,z1xp,O9rK5|4|14-853-863-2183|7305.44|uriously silent accounts. carefully special platelets use fluffily regular excuses. quickly ir +6159|Supplier#000006159|cAvSpNwr6Xbv|6|16-970-884-2915|9586.82|counts. furiously even pinto beans wake special escapades. blithely +6160|Supplier#000006160|kL4DGpFFTDPrxbUIBeU0YMjFPW|16|26-362-755-7430|6816.69| deposits. regular deposits believe carefully. reg +6161|Supplier#000006161| c94VMQ1bm31dgFeeda5P5fmWah,kcNG|14|24-735-112-2298|1085.77|ly pending deposits. regular packages wake blithely +6162|Supplier#000006162|fFvvXEZ9ONVAw1ol|9|19-378-457-4974|8507.56|final, ironic requests affix quickly. furiously bold ideas haggle blithely fluffil +6163|Supplier#000006163|2szRCosjkC0L,|12|22-978-145-4086|6600.02|otes wake slyly furiously silent theodolit +6164|Supplier#000006164|PwJeAuYCoZGaYAPw7W|23|33-725-717-4417|562.16|ic theodolites affix foxe +6165|Supplier#000006165|L7SKNMKhPRJmk|11|21-680-493-7835|8153.30|n requests-- deposits snooze slyly final ideas. carefully sly dependencies acco +6166|Supplier#000006166|aQWm0ff0V6IOsk|13|23-100-598-2246|3529.56|s asymptotes. fluffily express deposits +6167|Supplier#000006167|ka4jyY9IEX0Deh|3|13-431-411-9873|7284.47|ounts should have to haggle quickly! quickly regular foxes use sly +6168|Supplier#000006168|AZX0AUGidt4W0BdJHiXUCnti8u|17|27-773-867-9817|9164.59|s are carefully silent foxes! pending, final instructions integrate s +6169|Supplier#000006169|mpQWH0Rf3lBuFQ7Em4Bt6v|14|24-589-213-7426|5273.60|ress accounts nag furiously furiously bold packages. slyly even forges across the bold, blith +6170|Supplier#000006170|R0BHiZ7BNXVwPUa7|0|10-746-107-7971|4333.16|accounts. carefully ironic deposits use about the r +6171|Supplier#000006171|KVdQXR2UpWcBnjc|5|15-271-849-7468|5450.13|yly slyly regular deposits: carefully express platelets w +6172|Supplier#000006172|X6d5YALoQxG6XTaXlTKJbs8QvIatWUZ|16|26-142-635-6076|1426.03| unusual courts play slyly regular accounts. final depende +6173|Supplier#000006173|hBdratcVfL4LpWxsEpCRP g0AksN0CDhBZ|21|31-857-893-7518|7192.13|ily silent pinto beans haggle about the ironic ideas. blithely regular accounts besides the bold, +6174|Supplier#000006174|ZzeGcc24YGULnJL6BA|15|25-788-377-6501|8098.43|e blithely pending pinto beans. carefully unusual foxes boost +6175|Supplier#000006175|NVFNZHqL HZLKBDvRx9n6OEo7dQ 8rjte0E|21|31-368-793-6278|5369.32|blithely bold packages play carefully aft +6176|Supplier#000006176|F2bNnNdzMTFRSmsb3KAVk|19|29-726-581-2310|7124.23|ickly pending theodolites. furiously enticing tithes wake. quickly fina +6177|Supplier#000006177|QB4yo2kICrib HomjdgvWKflpDvLqqiP,PD8LSy|6|16-277-624-1626|-17.07|ependencies. blithely final requests haggle qui +6178|Supplier#000006178|HdoJKrXMoX3fNzftUbtZdDqvypMN9D1D5Pz|8|18-675-168-4377|8073.00| packages use quickly final deposits. +6179|Supplier#000006179|Z6,89fjYPmGO4eIHVsgbk9Ppd5TQcR|12|22-486-737-8624|7040.80|nal deposits are blithely inside the +6180|Supplier#000006180|xc1TzrPX1cyLBakOmh8I41S5|14|24-572-821-1898|5832.37|ent pinto beans. furiously regular ideas affix above the fluffily final platelets. blithely pending +6181|Supplier#000006181|,U2oNtO8z8CfXs3zeAgAuiewAWdj8MtyHJV|15|25-836-375-1011|1624.33|riously. slyly regular requests wake quickly. pending notornis nag slyly enticing +6182|Supplier#000006182|b j0rPTZhI,QRpdBbph1uCMGZiMMNpBv3HA|17|27-130-469-8203|7099.14|ual requests. accounts boost. express dependencies nod fluffily. +6183|Supplier#000006183|uKC69FZwJEVp|22|32-426-427-9149|7244.33| instructions haggle furiously. carefully special accounts haggle among the fl +6184|Supplier#000006184|mMV6154LO,nUXiKtxTQp6QlelBvB|15|25-958-711-1385|5462.48|efully quickly final pinto beans. furiously brave theodol +6185|Supplier#000006185|Wvmk27YkwJ5iUuGFMSkZS5f|16|26-463-874-7768|-16.86|express accounts about the ironic, ironic asymptotes wake regular pinto beans? slyly final idea +6186|Supplier#000006186|MmzhYZkIrc0Xrs0kuAHGfET 9vNinQ|6|16-167-323-3069|587.38|nst the even accounts grow slyly alongside of the foxes. unusual deposits impress ca +6187|Supplier#000006187|dcKVHu,FZPG81cR50cZdrVB16eePhglpZDRj|23|33-553-253-3113|3032.27|regular ideas. pending asymptotes cajole behind the blithely express packages. unusual packages nag +6188|Supplier#000006188|g801,ssP8wpTk4Hm|23|33-583-607-1633|9357.45|ously always regular packages. fluffily even accounts beneath the furiously final pack +6189|Supplier#000006189|LkbuzKeQK92BH3LxRROqP7A9DMS|10|20-643-623-1573|1796.87| have to are after the blithely r +6190|Supplier#000006190|H7y2WmBeemp2PrNqnvZAtSVuYxrTP2CU,edD|14|24-129-951-1922|2964.96|al deposits nag slyly final excuses. furiously even packages snooze slyly. quickly unusu +6191|Supplier#000006191|lYf3sL8KjpAyv,dFT|14|24-257-723-3364|115.28|oss the unusual, unusual requests. regular, ironic accounts alongside of the furiously special +6192|Supplier#000006192|hmk4yYcyWrb|4|14-141-759-1200|-528.96|ly regular dependencies. carefully silent requests nag quickly slyly silent requests. blit +6193|Supplier#000006193|J19B0Fhucw3DGyG ZuJwvsX8|1|11-208-207-5184|6753.33|unts-- blithely ironic excuses along the pending, bold packages nag carefully across the even re +6194|Supplier#000006194|G97kwdVWx1MGKWRZVbC3BLBG|14|24-111-670-9397|3204.05|y regular instructions-- ironic +6195|Supplier#000006195|C3C6Nhq VAzIFKS6JsU6xPaAoXMxX8Di,a|0|10-251-658-6550|3399.65|hang carefully. bold sauternes t +6196|Supplier#000006196|x9AuOrvAIzrMtgoWnsVb6scMwBG |6|16-720-697-2784|1897.34|nst the furiously ironic tithes. slyly pending deposits haggle blithely above th +6197|Supplier#000006197|ldcIIhw,wRVXM4i85LpAk1hxdD3JXqnMT|1|11-623-824-1214|8563.41|l asymptotes cajole slyly ironic accounts. quickly re +6198|Supplier#000006198|BKmrhC4GMB1F6|16|26-267-228-6726|-694.08|ounts haggle alongside of the express orbits. fu +6199|Supplier#000006199|U0o8Rw hqhDH4XaecvQA S|15|25-281-792-2764|3444.93| along the fluffily ironic dependencies. slyly even accounts cajole. unusual +6200|Supplier#000006200|6wsIBNjKnDHWL8No3aFr59WQr N9|0|10-536-289-9622|3990.56|osits according to the quickly final dinos h +6201|Supplier#000006201|VB8PhjCt0iWcSwSg5w0nnl,D4XdZfD|14|24-713-373-9372|387.55|r requests grow after the blithely bold excuses. carefully special dependencies affix carefully alo +6202|Supplier#000006202|Itv vUhL2DBuEkGs|13|23-344-205-4504|-534.22|s. slyly special foxes along the slyly final requests use furiously even de +6203|Supplier#000006203|CC34P,mkiltvswY O7qjt|15|25-507-654-6810|6208.58|s-- even packages integrate furiously after the slyly ironic foxes. quickl +6204|Supplier#000006204|w83 SdVHNrmuAK|2|12-995-621-1797|-372.01| special tithes haggle carefully after the regular ideas. idly pending packages use furious +6205|Supplier#000006205|LytAiNadMGGz7U6LT K1e2K,|0|10-874-309-9056|4535.43|beans engage slyly carefully i +6206|Supplier#000006206|UedK Py0XD5jONB2f|13|23-559-368-4510|6042.85| quickly bold theodolites above the silentl +6207|Supplier#000006207|lc7ok02O88prghZHZ8MA0v5IbJdeWHB0NhP7z|24|34-439-812-9056|8237.23|xes. final asymptotes among the s +6208|Supplier#000006208|KT cc4hGFVdzb8t5w6hf5|22|32-682-104-2483|6240.22|un. slyly final dugouts haggle against the quickly even asymptotes. quickly even dolp +6209|Supplier#000006209|j2Ztvy5aeSJhNVBig8EnTgD|6|16-812-490-7415|3452.01|e quickly express pinto beans. special ideas haggle. furiously +6210|Supplier#000006210|cxe apAEwK7z|20|30-153-120-7071|1938.62|s among the slyly even packages wake according to the silent packages. final platelets x +6211|Supplier#000006211|PZfXmaNP8DRGstAaTVnoOTKBMzWGERahyfY|8|18-507-756-6981|9360.72|after the furiously unusual packages haggle furiously over the quickly final instructi +6212|Supplier#000006212| FPwDtwiB3Lz66Xgam BiTqSi5vJOnj|22|32-668-988-5030|1726.46|ay quickly evenly bold accounts. express packages nod slyly; even packages was blithe +6213|Supplier#000006213|4gFjG,vvXST88|14|24-882-870-3330|7614.10|ymptotes haggle regular accounts. final theodolites haggle carefully unusual asym +6214|Supplier#000006214|BWNr,0S,rs3o3vXkMtKZCT0OyS|23|33-484-970-5494|5226.05|yly express accounts. blithely ironic excuses detect carefully outside the quickly special theodoli +6215|Supplier#000006215|j2iEbTsl,5PWdqWZ7k1yiISb7qtiiZljDIPEo|3|13-211-960-1484|8264.14| blithely final theodolites. pending, regular accounts are quic +6216|Supplier#000006216|xnmHrDgTArHIkpTWFCe2WEPjdZsGDgkU0GF9|10|20-826-360-5123|7526.59|pending accounts about the regular frays use blithely along the unusual deposits! regular deposits +6217|Supplier#000006217|RVN23SYT9jenUeaWGXUd|3|13-309-288-5638|8384.14|efully even ideas. express packages boost furiously. carefully fin +6218|Supplier#000006218|YuARFOUyI O4QDPuiX9e2ukSkr3nwb|16|26-563-526-1606|355.80|bold accounts. express, unusual accounts above the dependencies are flu +6219|Supplier#000006219|uXyi5RMOm9wDyS4cNSClomrFp,EfPkw3n|12|22-939-256-7229|4920.89|pitaphs wake alongside of the c +6220|Supplier#000006220|FeiFBC36pFsat5|24|34-392-299-5771|5408.08|dolites are along the ironic, express warthogs. spe +6221|Supplier#000006221|OH0ZgB NEcPdBvN,RPH7UBhIc|9|19-117-358-8828|9567.47|symptotes. fluffily bold dependencies poach! quickly special pac +6222|Supplier#000006222|CxUruV3DvLzkph1lKug,iNuT0vj sLHeR1u|4|14-435-963-9851|513.86|usly pending ideas are. carefully even requests sleep above the furiously special platelets. +6223|Supplier#000006223|h70ax62Oz9G2|7|17-469-708-1696|-658.42|ly ironic requests eat around the slyly ironic requests. furiously even theodolite +6224|Supplier#000006224|X1ifw7KS7dBGLYddu4WlChmjbnEBZ5GsOmWHm|7|17-983-498-7605|8936.26|rve among the asymptotes. asymptotes above the carefully b +6225|Supplier#000006225|SrmCApiIP0DxRwh1LRea0ASb3fBjLx6KsFk|18|28-970-112-8295|6791.52|eans. slyly ironic deposits according to the carefully silent instr +6226|Supplier#000006226|CKuDyeGAxPHeRHwC4a|21|31-699-107-7809|7191.06| cajole slyly across the quickly express foxes. s +6227|Supplier#000006227|xS8yhvoC6sYCbg|7|17-905-469-8595|1266.52|o the unusual deposits. quickly regular requests use carefully. quickly final +6228|Supplier#000006228|F2AkwjvpbXLJEil|11|21-114-548-5385|-720.23|y blithely bold deposits. +6229|Supplier#000006229|xBtJZwxIvrC|9|19-453-638-3838|3978.33|. furiously even instructions haggle. permanently special deposits are slyly. fin +6230|Supplier#000006230|RqvzNl,1g6lIrVruop2ev4AnnfiU7Ml,wkgGZiz|11|21-719-169-9893|3717.70|lar deposits boost slyly careful instruc +6231|Supplier#000006231|L7I4lcRcA6xjD|0|10-310-882-7464|1355.83|lithely ironic requests boost closely re +6232|Supplier#000006232|,yM729cm 7Jrc,pLW|4|14-248-518-3019|4453.35|oost slyly among the even accounts-- blithely sp +6233|Supplier#000006233|HwkulhDgsdEwXiTTrtgSdY2y3 xiRx|6|16-303-671-8881|1449.96|lithely around the express theodolites. fluffily final requests a +6234|Supplier#000006234|G3hp AoqeCrBENUrHBa 3FoSsLM8Hkqr,|7|17-535-635-2492|2419.96| integrate slyly with the furiously final packages. carefully silent deposits along th +6235|Supplier#000006235|cuLS2Vz5q0UhmF2BeL0,Qn6UuorqT |19|29-835-102-3960|3316.93|ously final accounts wake quickly among the bold requests. accounts along the +6236|Supplier#000006236|ceR7WkC7L9U5zvWwNvTp0|15|25-140-722-6454|8627.81|efully quickly final dependencies. special, brave excuses sleep. slowly express foxes use unu +6237|Supplier#000006237|ghp4,RZ08rhbRUlI352a|6|16-655-635-6500|3605.55|accounts are slyly unusual, regular deposits. even as +6238|Supplier#000006238|uJD8zM90,LL021MxuiS5RP9lV,|15|25-917-640-2121|8424.94|fluffily after the furious pinto beans. fluffily special hockey players int +6239|Supplier#000006239|Odl ASlp,CYETnUA4hvKGJ|23|33-972-802-7976|4073.22|iously regular dependencies are. quickly even foxe +6240|Supplier#000006240|d1Rz9X9NZzNJcAXa2fNc2CY9soMhiX1eYTz|1|11-242-258-8709|3518.19|bold packages sleep quickly. fluffily regular +6241|Supplier#000006241|gpFQ9E9bgG80ew bU,X8u3WJa0e|20|30-754-238-1118|3176.25|express packages. blithely permanent +6242|Supplier#000006242|JwTRkySmNoZIdjgWixSAm|15|25-497-887-5617|9158.66|ts. even packages sleep alongside of the carefully special ac +6243|Supplier#000006243|wU9Pt4UnjG,u9CHk7AGsKcyvTXSWdxrJtBi1|11|21-302-192-6652|4691.16|lly toward the finally final dependencies. blithely re +6244|Supplier#000006244|kdG6IroPXOK1UvDsIQTe0HGwLfkKLmfnCppY|7|17-355-549-8593|5252.19|nic requests boost. slyly express excuses affix furiously. enticingly +6245|Supplier#000006245|KkROogNYZnnNjjohxaMRr A1|14|24-185-415-5376|7532.31|d packages cajole slyly after the pending deposits. carefully permanent ideas are permanently e +6246|Supplier#000006246|bZfRctecsVimCe1Uc8, 1iJ2ZjYV7Yo,Pb|11|21-927-329-5043|5658.83|to beans haggle? slyly sly instructions according to the permanent, ironic ideas haggle abov +6247|Supplier#000006247|3JmrDbyjGCw1LmtsOo|24|34-266-484-9288|42.92|special deposits wake closely express theodolit +6248|Supplier#000006248|Ef5f,rmeFhpIwuC|9|19-968-511-5539|4294.13|equests cajole carefully unusual, regular instructions. unusual accounts haggle above the quickly un +6249|Supplier#000006249|r6YQbtMv06I,MnHSmdapyNFU7DbV|18|28-847-854-2171|9366.21|, regular excuses breach carefully fur +6250|Supplier#000006250|rCzFt1t4KiLkuV69kpMAoeQJ|9|19-311-882-7947|5905.93|ts maintain permanently against the ideas. furiously ironic deposits alongside of the pending foxe +6251|Supplier#000006251|INCJEsmMji2NS465MKkEyE0c7bGoZyS|5|15-470-707-9609|5969.73|he requests. carefully regular deposits nag above the +6252|Supplier#000006252|UwHj WvLeLCueyOUR|13|23-234-956-2802|314.00|ely slyly even foxes. furiously unusual packa +6253|Supplier#000006253|8,2GodYdtg5|20|30-997-262-3199|6412.90|g the slyly regular platelets. requests serve fluffily f +6254|Supplier#000006254|g7OY1vWNUb1vxIRgEl|21|31-267-164-9506|1613.17|efully silent instructions wake slyly carefully special pinto beans. quickly even pat +6255|Supplier#000006255|9dS8 2Vopjq4HrL|18|28-245-586-3399|6475.15|al packages sleep carefully unusual accounts-- slyly pending deposits are across the blithely ev +6256|Supplier#000006256|NStlIT1xnwfoNy,V6Tr8nSrGgDkzyZphYEzNZupv|13|23-516-827-1769|2809.39|ross the pinto beans. blithely express pa +6257|Supplier#000006257|NvNMxu5S3qtWWRG3gSOCdimN 6s5VxWIj|15|25-472-947-9532|3324.64|press requests. idly even foxes cajole alongside of the accounts. depos +6258|Supplier#000006258|qoPD9k OjZKX|6|16-323-688-3016|6271.69|final platelets toward the slyly unusual ideas integrate carefully above the slyly +6259|Supplier#000006259|cCDmxs1a3,8u,XlkIX,36 tve6ZKcDyAo9|20|30-100-280-3565|4381.52|ironic instructions. carefully express w +6260|Supplier#000006260|aZhPpMAorL|9|19-290-560-1548|4143.16| snooze requests. bold deposits believe after the special pinto beans. r +6261|Supplier#000006261|TPT oX927,LE,|13|23-575-322-3542|2095.03|nal deposits. dependencies cajole quickly after the platelets. quickly regular courts cajole blithe +6262|Supplier#000006262|T9usrQ2LNhwdQB|2|12-432-745-5354|9133.59|cuses. fluffily fluffy theodolites use furiously requests. even forges haggle furi +6263|Supplier#000006263| qXZqIkwG7AZoQMc6n|9|19-253-714-9239|1287.37|its engage carefully against the blithely bold excuses. quickly pendin +6264|Supplier#000006264|hsAqFN4vuG4UtNiHHN73WpOiOBSpUe8P|13|23-149-361-2166|8696.70|sy deposits. pending foxes doze. furiously b +6265|Supplier#000006265|V4cP0Jpqf9HemuJSwFvz3hqlgK5K |1|11-741-337-7382|6514.72|es nag slyly finally silent foxes. fluffily ironic dolphins lose carefully amo +6266|Supplier#000006266|XWBf5Jd2V5SOurbn11Tt1|13|23-363-445-7184|8007.83|as cajole carefully against the quickly special ac +6267|Supplier#000006267|UB0y3o3xdo02Xy55mK2qcIdv5ZJh88aE3EQRGHWL|6|16-460-193-2006|700.51| slyly blithely silent asymptotes. foxes nag carefully pending deposits. furiously e +6268|Supplier#000006268|0YmsNcKHfQgyrIIfNZpQ7D|2|12-940-640-2003|1407.48|es. enticingly unusual re +6269|Supplier#000006269|nbQEngNTbFInhIntk HMlJRQJCE,Q0UW3uzdWzr|7|17-710-792-5790|3440.39|ound the carefully final pack +6270|Supplier#000006270|KJeGERaiEn8osyqJc0abDnaVK4A,TiUws0Qc|8|18-622-127-8119|4562.13|ular packages along the ironic, pending accounts cajole slyly pending accounts. slyly unus +6271|Supplier#000006271|LgpupeuXWX|10|20-135-150-9014|3722.82| the ideas. regularly regular dolphins haggle carefully slyly express requests. ironic ideas sleep c +6272|Supplier#000006272|RqMqHKN,CFODzxIT,Gl7UmhlgEIe12QF|19|29-831-993-7297|-574.94|wake slyly at the furiously bold requests. ironic packages about the deposits sleep against the +6273|Supplier#000006273|SGMEJDhNRsinMcSCymSVfAq0uq|18|28-855-178-4941|3969.33|lithely regular requests around the quickly even foxes are fluffily even theodoli +6274|Supplier#000006274|S3yTZWqxTKUq g QQgcW9 AqhCkNZsW51hHuwU|3|13-956-812-3470|6572.73|nding foxes use slyly. care +6275|Supplier#000006275|b4jthW1EU 4NP5txI3D YX Bj6s1NSwn|6|16-939-109-4187|3944.47|pains. regular requests sleep blit +6276|Supplier#000006276|,dE1anEjKQGZfgquYfkx2fkGcXH|4|14-896-626-7847|6702.07|ze about the carefully regular pint +6277|Supplier#000006277|3dzdL9GPnRs|4|14-489-102-4639|-191.93| the regular deposits use among the packages. furiously final instructions nag quickly aga +6278|Supplier#000006278|FID,5jREsaq6JMFk|6|16-303-555-2725|6151.14|ronic accounts wake blithely ironic, regular deposits. fur +6279|Supplier#000006279|pN2YG7qZBGsVUe8F|23|33-526-284-6953|2172.62|final accounts. carefully even dolphins against t +6280|Supplier#000006280|DHbI2DZMqCY7PV|18|28-933-750-9364|4250.39|to the blithely pending instructions. quickly regular foxes are slyly after the slyly +6281|Supplier#000006281|MR ,g8qBCC QjngVrjqpcd4Bat|20|30-822-450-3636|221.05|ounts. final requests haggle a +6282|Supplier#000006282|hEcrcg2AlVQD|7|17-486-187-6942|6146.10|affix across the unusual fox +6283|Supplier#000006283|YnA,GQCNK6RpSmLxgxK4zgR6rm8LizebsthLB2|20|30-930-362-6160|5565.03|ickly unusual dependencies. furiously pending de +6284|Supplier#000006284|gK3spm0FpnPVXAiygSwHLDl|14|24-552-425-2693|7468.64|ly final pinto beans wake +6285|Supplier#000006285|jjR7 6B259HYtHaaYJLzSf|16|26-376-973-2470|2437.03| regular accounts affix. theodolites around the fluffily even requests integrate carefully +6286|Supplier#000006286| BqBfmh1kM,6Kp0C4fyByE2 |22|32-848-783-5771|4202.93|. final requests boost after the blithely express platelets. furiously regular instruction +6287|Supplier#000006287|ugzGtAHRSnWiMX|23|33-637-271-6072|5723.61|ld deposits sleep slyly regular packages. bravely pending asymptotes sleep slyly al +6288|Supplier#000006288|8lmR9vQ1yv,p1gx0IbxR 7m84P0oDGBFVUS g7ta|24|34-508-780-1690|-331.89|s above the ironic accounts boost quickl +6289|Supplier#000006289|W3Q 0yQkF790P5Xb|12|22-280-229-3372|7849.87| the carefully unusual deposits? furiously final requests wake. slyly special pinto beans cajole +6290|Supplier#000006290|UBy7c0lfhgNdggv,ZfiABr zr6a7Pp|21|31-212-467-6131|7872.05|sits. quickly unusual instructions according to the blit +6291|Supplier#000006291|IOfsCKTfWFqjW8IaF d3pGTlx|6|16-409-523-5659|6584.60|se, special deposits grow-- bold ideas haggle about the carefully quiet +6292|Supplier#000006292|4W5P7Twl,0zdDFTjjYA2jxi5Zm|23|33-887-621-6202|5341.10| regular foxes sleep carefully final deposits. +6293|Supplier#000006293|G8Cl7YvkoXL4jKmC4|18|28-569-527-9913|5961.00|the final deposits cajole along the carefu +6294|Supplier#000006294|CgjVPJnHnVcy65 K0URSEyIQKy|13|23-991-829-2676|7628.50|kly among the ironic deposits. courts boost car +6295|Supplier#000006295|,kIChX9uLiLe2,Rt9Iyzj cZJ|19|29-927-347-5184|5233.22|y regular requests sleep furiously regular foxes; carefully silent requests about the quickly fi +6296|Supplier#000006296|baByf,CQi60rl0YSmAUyQ9l10KR3q|23|33-577-460-4392|7774.48|ously after the theodolites. furiously +6297|Supplier#000006297|73VRDOO56GUCyvc40oYJ|3|13-829-103-1930|5678.43|s sleep about the slyly ironic in +6298|Supplier#000006298|3m3bGO,frzuKU59lWs|23|33-378-187-9064|-687.56|sly final theodolites. furiously bold requests about the slyly silent deposits use ab +6299|Supplier#000006299|ozE7ZNK8MaY2TKIKWeCS9vB2wDKT8bD1vIa32y|3|13-787-914-3226|3124.71|he furiously final Tiresias. furiously ironic deposits sleep furiously across the furio +6300|Supplier#000006300|uiFP3Q8mdV|0|10-134-364-3373|916.59|c theodolites haggle from the regular, express ideas +6301|Supplier#000006301|xBPspJhGUvc5Ujzak|9|19-366-114-6662|3112.69|y even foxes nag above the patterns. even, bold platelets haggle blithely. regu +6302|Supplier#000006302|siCR7A0n7no|12|22-954-431-6230|7956.36|ss the slyly bold theodolites are furiously regular, unu +6303|Supplier#000006303|9JV2LMsFaelu hiHWk|21|31-840-149-1133|4598.95|c, ironic requests nag carefully slyly ironic foxes. carefully +6304|Supplier#000006304|vd9L5pgN2B,vIrAUyE9gA48Yl,RBiCICLq7mB8m4|0|10-891-954-8640|6567.80|earls promise. carefully even deposits affix fluffily express instructions. unusual pinto beans a +6305|Supplier#000006305|SKb42tl3EoXxQF4zmvfdUlmNuBCd9T3sonW1|13|23-181-742-9276|8996.95| above the courts. theodolites alongsid +6306|Supplier#000006306|mSxhoCt9kzDJW00O3d5QHKvDEDqxD9RelvL|22|32-783-788-5511|3795.19|express foxes snooze carefully pendin +6307|Supplier#000006307|uf2wEQtEHrxWkYEUEo3|10|20-978-302-4194|5735.76|re carefully furiously final requests: theodolites could hav +6308|Supplier#000006308|7qJ58 M8unOheiGiVwzZI6CnwyiguoDdM|2|12-421-363-4262|5863.40| slyly pending deposits haggle furiously according to the regular pack +6309|Supplier#000006309|dwPuozxMI5LRuDS6BWZC5Ydk,|18|28-567-726-8258|6188.23|warthogs. blithely silent dolphins according to the slyly ironic dolphi +6310|Supplier#000006310|yqRw6vtIMslHmp2r0hmgpd2c7HG8oE|12|22-864-427-3604|9539.74|iously unusual accounts. slyly busy warhorses haggle after the furiously regul +6311|Supplier#000006311|PfMupGhJmv|22|32-567-413-2348|7408.57|e regular packages breach slyly furiously final accounts. carefully regular reque +6312|Supplier#000006312|Ouokin5ZDB0CL7hydeJc|2|12-484-391-3878|4846.91|refully unusual accounts wake carefully regular notornis. slyly even accounts ac +6313|Supplier#000006313|KSl8lI,tC19,,|12|22-278-780-1705|5024.55|ic ideas wake carefully according to the silent dependencies. express, ironic instructions +6314|Supplier#000006314|JX6JZ7eMyZ,9JGsEOXA|0|10-783-712-7597|9655.87|urious pinto beans are slyly final deposit +6315|Supplier#000006315|6,60ngEvFt1v6|22|32-573-372-1627|1324.33|ages. blithely even dependencies are slyly final p +6316|Supplier#000006316|VCmgK0e9AK1YTn3fltIRN,LMrapXUEMzMyK7S|13|23-383-196-3658|2924.31|s poach furiously theodolites. regular accou +6317|Supplier#000006317|zmswEUfyQ6a9SP|7|17-770-212-1020|9924.69|yly silent instructions boost blithely +6318|Supplier#000006318|wfTdhyXHHjyB8f0VCXr6gVhdlZCmqiYEmG6A|19|29-345-112-7505|9295.56|p according to the carefully pending requests. instru +6319|Supplier#000006319|pUwA09iYxW1I ir4Dy24dt8jpUfIhhQB|9|19-691-845-4827|6475.09|h fluffily furiously pending instructions. +6320|Supplier#000006320| YRjdYgv6q8NpX|22|32-211-274-3030|1853.10|uickly ironic packages. quickly expre +6321|Supplier#000006321|qgEJ2RqfkCZ ftPOVNAGeYIuiR6 R0nd,3|0|10-301-973-8567|1985.39|nusual ideas. blithely careful dependencies across +6322|Supplier#000006322|3hJBOqhZ,YsWaVFOaZbi6U3O5gv43 q|5|15-978-516-4873|3368.59|old accounts. requests need to cajole fur +6323|Supplier#000006323|wYTCmD9tv53H6w7CglX,F|8|18-518-921-4884|7839.19|ckages. furiously final ideas haggle final, special accounts. furiously regular theodolites cajole +6324|Supplier#000006324|oTr6OS8K3vXvZ|16|26-396-854-9732|4379.96|e slowly. blithely ruthless depo +6325|Supplier#000006325|VEAbBo9vdI5nmJLJ26NrsLLg003XOMBkQtZJ0Mtc|11|21-665-388-7326|-90.03|egular accounts are furiously final, bold packages-- silently regular foxes sleep after +6326|Supplier#000006326|LHy0dm5io93xQP7BCS|20|30-187-796-8720|2846.03| ideas. regular, final requests cajole careful +6327|Supplier#000006327|YDxXt1y6B1iMa|9|19-314-243-5572|3165.34|yly unusual pinto beans. fluffily final foxes after the furiously +6328|Supplier#000006328|10nuH u4FoWGNY|19|29-734-302-5442|1827.47|luffily even instructions. special instructions pl +6329|Supplier#000006329|pS,19ulEFGhc2dc6q|12|22-544-832-8536|259.16|s. slyly idle ideas use quickly excuses. unusual requests around the slyly regular acco +6330|Supplier#000006330|UozlaENr0ytKe2w6CeIEWFWn iO3S8Rae7Ou|22|32-561-198-3705|8655.99|symptotes use about the express dolphins. requests use after the express platelets. final, ex +6331|Supplier#000006331|P0Va2RoE8ZN5UHgsHr4YLz|11|21-857-910-9555|4252.84|y regular accounts. furiously final packages poach slyly. blithely final reque +6332|Supplier#000006332|Y45cOBhl9EZvXMhZOI6jo,|19|29-540-415-9434|8413.76|ts maintain. bold, express deposits wake carefully. regular theodolites doze until the blithely r +6333|Supplier#000006333|FSfiGO8u55egm|10|20-326-566-4368|3143.35|thely express multipliers. fluffily bold accounts cajole carefully furi +6334|Supplier#000006334|y8DiyuolE9NZaJ5C1E5w0 yoEjapsGqpujIwkxj|8|18-842-574-1012|6191.87|le slyly according to the quickly express deposits. carefully final deposits are car +6335|Supplier#000006335|JcA0q6RIDepXmvBBha2QGWhoaUC|23|33-160-248-9175|4715.60|ounts. blithely express accou +6336|Supplier#000006336|gA9KavvjYz49J9T3BbP0VoLg7EKoj1n|1|11-518-414-3019|9024.11|orges wake slyly along the regular, even accounts? slyly ironic theodolites affix slyly +6337|Supplier#000006337|ilQgNOdCAysclt3SgODb6LeJ7d4RzYD|10|20-950-345-8173|9343.35|quickly ironic packages affix aft +6338|Supplier#000006338|bqDN3c3KhAcaGqkVWZWDK9zcw6hez,sVDSVf|5|15-557-297-9599|6781.20|le carefully requests. fi +6339|Supplier#000006339|7DRKGrpsNzi4l,joQjQrsC G|22|32-998-703-5244|9625.15|eposits are blithely among the slyly bold deposits. ideas +6340|Supplier#000006340|p9pZspXKY4PcoUOSic2x0FGF65IuJNxtd1|6|16-656-270-4190|1925.83|even realms according to the carefully reg +6341|Supplier#000006341|XtXSvFQoDs2wMiN7hi5wv080ce|18|28-497-381-1214|3423.89|. carefully special pinto beans by the pending pinto beans sle +6342|Supplier#000006342|PE 4qA5tQBD0xEvI2DrfNjsLvlIdqFdRS5alxUcc|20|30-847-753-3542|2273.10|usly special instructions. pending deposits affix slyly: quickly bold de +6343|Supplier#000006343|UIRkduD9Iv2sxX9ly6MPt|4|14-617-999-2300|9998.20|odolites. unusual packages was carefully reg +6344|Supplier#000006344|CnoxEmht6YuyQLFq5HWZFH,QHjyCFnQo|7|17-494-458-5494|1810.37|final pinto beans can cajole sl +6345|Supplier#000006345|VSt3rzk3qG698u6ld8HhOByvrTcSTSvQlDQDag|6|16-886-766-7945|9847.57|ges. slyly regular requests are. ruthless, express excuses cajole blithely across the unu +6346|Supplier#000006346|1UMASn Ogy1ngmGXp9yx4tU|3|13-287-735-3672|140.04|nic, final requests about the slyly express accounts sleep slyly after the ironic, fina +6347|Supplier#000006347|wwR5j4kdIAwLe33uBwo|13|23-340-942-3641|8514.86|uests breach blithely ironic deposi +6348|Supplier#000006348|f2KDn2rLnadX8I DZR|21|31-428-570-8960|8339.22|ccounts haggle furiously bravely regular instructions! even excuses +6349|Supplier#000006349|pzREPFv5Qqm2WQFILxyRJyVfARzLmR8B|17|27-232-385-8198|4360.68|ckly ironic requests x-ray. ironic deposits believe carefully across the requests. expr +6350|Supplier#000006350|XL9SXQigASK7Zpo7Ot8cS8n93GBlrf 2t|22|32-648-317-9783|5921.30| blithely unusual instructions haggle slyly after t +6351|Supplier#000006351|Oa9MQNZKuXgQfaKdq|7|17-884-320-1089|269.54|o beans. furiously unusual requests detect among the final, bold foxe +6352|Supplier#000006352|KLkSonR8IDWuTJ|23|33-131-648-6999|7061.71|s about the ideas. theodolites about the furiously unusual requests hang express packages. pinto +6353|Supplier#000006353|DoJWyLP3iNP9fF jWmMfEnH|23|33-671-650-5004|5255.87|eposits cajole after the slyly bold gifts. final deposit +6354|Supplier#000006354|ESd0lukbVG4EGNVMeHyjm7|2|12-140-122-6149|3498.74|de of the regular requests. slyly final instructions among the express, +6355|Supplier#000006355|Sw2t9Cr1,DzKTgZNce|19|29-571-768-8366|6300.97|bove the ironic, unusual pinto beans. slyly ironic foxes haggle? blithely even instruction +6356|Supplier#000006356|bz0XdIOgP9YWQEWh7jjEwfm0|1|11-670-108-2673|1499.78| requests cajole fluffily ironic accounts! slyly silent theodolites print. quickly express dep +6357|Supplier#000006357|seiEPN3dM5I4osXQVTQ,GLuRwkHZ730Tyw jctB|18|28-876-575-4174|3676.55|en packages. even accounts are. final deposits detect. car +6358|Supplier#000006358|isQ76W0izOrHKfR9fTQw 4f3|15|25-211-503-6000|2752.44|er the requests boost carefully slyly ironic foxes. ironic, r +6359|Supplier#000006359|QyUuVHYBp8sTd7Y9WveNfsz|21|31-165-643-7917|3231.73|dolphins. even packages sleep alongside of the quickly bold foxes. final requests sleep car +6360|Supplier#000006360|NC2ZhKppyWcUNX5e76pwHHgkuJh6IqeSjbU|15|25-368-442-6089|6587.12|y. fluffily regular instructions are car +6361|Supplier#000006361|4 xClcX1vZVHyR67ZXitMln9skIjgvIotCO1|6|16-998-315-2550|3824.06|mptotes. blithely express dependencies about the blithely pending asymptotes cajol +6362|Supplier#000006362|Re6wYN1 4Tdxf|14|24-863-219-1934|6883.38|l instructions. theodolites after the blithely even platelets wake according to the ironic ideas. +6363|Supplier#000006363|Np0ZNsWKrUjAv7BbDT|0|10-481-194-3873|81.94|packages solve slyly across the quick packages; express, bold sent +6364|Supplier#000006364|myJZlVAUJbPOdm9 Hv4EVNJvrafSUEu|14|24-839-889-8482|164.98|nag ironically express pinto beans. fur +6365|Supplier#000006365|S2UJO3tr6vmWCjhkGdqTkZ|1|11-662-645-2588|9770.02|e blithely ruthless accounts. carefully special ideas boost blithely regula +6366|Supplier#000006366|cwEztmA0DXvIeMJz4ae2B4JRjjUPX6f|9|19-192-292-6849|5773.43| ironic requests. quickly final instructions haggle furiou +6367|Supplier#000006367|DmcL2rx5XYWM9eJvokF6h,CejdFjCAO5K|2|12-114-763-7346|1004.33|d the final, silent deposits nag carefully +6368|Supplier#000006368|Ydi8miTcDpYHce|14|24-735-697-9466|3682.38|ard the carefully special theodolites. furiously even ideas sleep against the slyly regula +6369|Supplier#000006369|3gvpMAyHtsmmErVHsD7N273fN|17|27-517-872-5692|1760.12|uffily silent foxes cajole. slyly e +6370|Supplier#000006370|3hUWgZ6DWw34Yxa92LFqBUZGg695je|3|13-884-872-4984|1331.27|ular ideas. even, even requests ru +6371|Supplier#000006371|S,L4tTXgvegIusPLEV3zAYBvpzLOLWEJhDeIG|23|33-702-175-1305|9646.85|ct furiously according to the bold, regular accounts. even deposits nag +6372|Supplier#000006372|FFKEAz5F QEhkDI0Yx RnPngOABPu|2|12-175-537-4891|3680.90|ld accounts. final pinto beans cajole. furiously pending +6373|Supplier#000006373|J8fcXWsTqM|7|17-813-485-8637|9871.22|etect blithely bold asymptotes. fluffily ironic platelets wake furiously; blit +6374|Supplier#000006374|zaZS HQ2BBpZtOgqRE4|21|31-531-178-1807|1670.45|to the carefully express pinto beans. ironic accounts wake quickly after the regular, ironic requ +6375|Supplier#000006375|O32whHBMXNKe 6Rm4|2|12-288-665-3977|1226.77|ously ironic packages. furiously bold accounts will are. doggedly even asymptotes sleep slyl +6376|Supplier#000006376|gG,EzjUOgEXYaHYyfdP|23|33-858-633-3470|2260.92|ing theodolites. slyly express accounts a +6377|Supplier#000006377|1,3JwsvoowdMB7zoe0myv0Xk|15|25-680-413-6096|3198.21|yly special theodolites. quickly silent excuses above the b +6378|Supplier#000006378|lJ7TDntmPExlSQNPJLmvJmkniqH|20|30-223-339-7680|4336.35|ndencies are blithely pending theodolites. foxes nag quickly slyl +6379|Supplier#000006379|DuEaRd6EiB|20|30-343-403-9779|4090.27|st furiously among the express, regular pinto beans. regu +6380|Supplier#000006380|owSMs4OjCGDQ50wrm2YSY|16|26-575-664-4836|4172.27|unusual packages. slyly final deposits for the packages haggle slyly around the slyly regul +6381|Supplier#000006381|3 ADNrPsw0Uo6L9sMLtRxs|20|30-174-486-7624|7281.23|cuses according to the special du +6382|Supplier#000006382|tvoB5S0FEykK6oEuWKhUwQE1 PEthJ1Kc2HF6e|15|25-862-837-6488|9613.10|ic pinto beans wake carefully slyly slow ideas. slyly even realms sleep slyl +6383|Supplier#000006383|7kQeP8yOiz,C5rv e6y2Rhoh8i6NZbYNbmkk|8|18-161-652-2581|8478.07|sual requests detect above the permanent +6384|Supplier#000006384|1IwNn8fcFhwrkOupH0VRA69|20|30-562-636-1925|6026.71|alongside of the fluffily regular instructions. final, +6385|Supplier#000006385|bI3lW8qb0nMmTOKFXxkxnSXumzwF1q8R52G 06dN|17|27-266-389-2532|962.20|jole carefully despite the blithely express asymptotes. furiously special pinto beans a +6386|Supplier#000006386|C6LXPOhlXY wNSJwC2MX VgBNtD2 uNfaKaUy|5|15-588-214-7300|-296.46|ss excuses haggle. stealthy packages are quickly about +6387|Supplier#000006387|LpcsMpJYXymD|15|25-511-449-3281|9368.23|egular requests. slyly special requests sleep furiously carefully bold dep +6388|Supplier#000006388|sIEnnJ0qa4ZMJP8Bv2EfEdOdyUc24mpO3|8|18-850-554-4696|6961.78| according to the regular theodolites. quickly final courts serve +6389|Supplier#000006389|hLJ01QW0RBHCuKNDT7iiiKz|0|10-535-625-9644|8657.96|ts. furiously express accounts after the ironic dependenci +6390|Supplier#000006390|SFEZ0fgPL4kmfFKnUE Y32mR QBLjWo|13|23-748-886-1807|4447.53|ts do cajole ruthlessly final packages: furiously regular tithes according to the stealthily re +6391|Supplier#000006391|lIWTJp,U7E|12|22-173-912-2217|2604.56|onic ideas. regular accounts nag quietly. slyly +6392|Supplier#000006392|,E8AY8n74zxV8|7|17-524-150-7447|3756.46|tions cajole furiously against the slyly final depos +6393|Supplier#000006393|3O2Kmb4F7s6rHSuH0ZNON9BVdJ2h|1|11-712-520-5154|3733.56|wake carefully across the permanently even foxes. pending deposits use furiously according to the +6394|Supplier#000006394|tL8yyWfEENhLhfmHFo,5PhLB8KUhgUPngd|20|30-494-694-6272|3881.45|f the braids. bold sheaves cajole carefully final requests. quickly final accou +6395|Supplier#000006395|NANvpbrsuGCWyV7q|12|22-780-233-1619|3546.38|hang on the final accounts. depos +6396|Supplier#000006396|Zw7uydYx7jR4eBupXo|11|21-165-843-8357|3955.17|y regular deposits: special de +6397|Supplier#000006397|h3rp6cKPeZpMhWBwAZSoXyBkk|21|31-941-870-7526|-518.45|bold instructions hang blithely across the carefully brave de +6398|Supplier#000006398|8xolYqRypVnqYeaJvjxu3xG6XyfMdZ4|7|17-203-383-4815|7287.20|r the final accounts. boldly regular theodolites waCustomer regular, slowRecommendses s +6399|Supplier#000006399|eC4XbrvjvcN1v|11|21-476-663-5034|-37.56|after the carefully even accounts. foxes above the packages wake furi +6400|Supplier#000006400|ifM8MbuxVQ16VGr7B0ACb6n,C9Zi|10|20-360-877-2726|4683.83| unusual requests nag slyly about the express dependencies. quickly bold theodoli +6401|Supplier#000006401|0GnAOyOsGqapvp82v53uhYklXzLyN4zVR5QYB|1|11-506-872-5124|3198.84|he furiously pending deposits haggle against t +6402|Supplier#000006402|GvjuIUGvhMyCZnE0wXXnRdRO5yDG|21|31-149-118-4122|3938.52|olites wake boldly instructions? qu +6403|Supplier#000006403|VVR,QQ0CKk 59JyAMgk3LG71ZU2L8oFM3|11|21-609-223-2042|5648.81|according to the furiously regular courts. expres +6404|Supplier#000006404|eUbXmjStnnebzzjnJlFseeZJN1w5mdHZoZyhp|10|20-354-610-6355|7545.29|s-- quickly ironic ideas alongside of the pending t +6405|Supplier#000006405|CTY5LQAqZJou3U8 ,r,ZT4hg5CllPNrmrfdl |1|11-943-664-7522|3772.74|ding to the furiously pending deposits. quickly pending packages +6406|Supplier#000006406|6poMVDDzXYbs1NEQraofgkesi,k OM|6|16-865-709-3599|2472.15|lyly ironic packages among the final accounts haggle blithely pending platelets. carefully en +6407|Supplier#000006407|RS2MJ ymEhzUau2Oka01RrpnhXur|11|21-945-999-4100|356.94|onic packages sleep quickly ironic +6408|Supplier#000006408|m9kAV0yvdtlzmj2exl7DyN5 u8VnHncTvV5awq95|9|19-723-538-9273|2051.05| furiously ironic deposits doze slyly after the ir +6409|Supplier#000006409|eO8JDNM19HrlQMR|11|21-627-356-3992|6985.93|sts. slyly final deposits around the regular accounts are along the furiously final pac +6410|Supplier#000006410|817Tla2QV1|22|32-242-382-9137|7524.22|s nag fluffily. quickly regular accounts cajol +6411|Supplier#000006411|JsXBFa8pcpcCE|5|15-967-750-3353|1662.97|gular requests detect whithout the carefully ironic requests. blithely re +6412|Supplier#000006412|Oy9uRyGgsCmWmwJeCS OL|0|10-984-296-5608|7488.10|ld packages-- packages along the ironic foxes print furiously slyly ironic p +6413|Supplier#000006413|v,b2O08ar3|8|18-218-737-9405|-526.45|l have to engage regular, even deposits. final pinto beans ca +6414|Supplier#000006414|TYcVckkbz8su1MdIwsV4dge|19|29-997-905-6029|4934.88|excuses use carefully careful depos +6415|Supplier#000006415|EWaeWc2Po68CD8u6tSwIKHT2i|5|15-212-904-2698|9474.66| across the regular theodolites. regula +6416|Supplier#000006416|oYDSMEcdAap|9|19-578-959-2228|2736.53|he regular accounts. carefully regular orbits sleep. pending excuses sleep carefully b +6417|Supplier#000006417| JQv0FIkmE82xpAkcPMfYM91zJesK3qmruv83|0|10-592-410-7303|-246.13|ely fluffy packages nag across the even +6418|Supplier#000006418|kfgHHiyyspAilMlCo|11|21-113-879-1794|8159.87|st the express platelets. regular, regular theodolites boost special, i +6419|Supplier#000006419|ph9GjjxPfC|14|24-363-440-5054|4063.69|arefully express excuses. deposits about the stealthily idle +6420|Supplier#000006420|WsuMIhxVOrPENcumh1aMY9MjZIJHhUEtsUmn|24|34-210-737-7162|8401.60|ly express deposits. accounts haggle +6421|Supplier#000006421|AwhpC5y15ua6plU8rtMwx|4|14-514-734-7640|7789.12|ial dependencies; regular pinto beans use after the furiously bold somas. idly dogged packages ha +6422|Supplier#000006422|ZB4jC7Mkv12AF69jXunQu8FZEPdjF5Incs|23|33-182-240-4174|9847.19|ecial packages. quickly silent accounts about the fluffily unusual theodoli +6423|Supplier#000006423|W7T4dvJBJWQ2SWbv|11|21-126-547-3000|4726.24|aggle slyly according to the special requests. furiously even foxes after the +6424|Supplier#000006424|1lnDhvqe6,KYaOW2Up4WPJdG1xmXyW,FX|6|16-215-126-7999|850.57|. orbits boost ironic foxes. quickly unusual accounts affix across the special, even instruc +6425|Supplier#000006425|M40,4vzvHXPnti8pBmvCYXVC|24|34-173-377-3598|4466.03|to beans boost slyly regular deposits. ruthless foxes eat slyly unusual sentiments. carefully ir +6426|Supplier#000006426|uyJ7mouse7RTDny8LBNIesAu4rBjVWMLcMcMFoec|8|18-465-419-6121|5532.41|st the regular, ironic ide +6427|Supplier#000006427|rrMkXW7o0O0U5,CsVTzEKtSRfAWtvhQe5Iu|19|29-908-367-5652|9309.80|packages. final pinto beans cajole. carefully ironic ideas doze. bold accounts cajole along the +6428|Supplier#000006428|fwLkcQltdvKack735j778yg|10|20-845-151-5520|3807.79|ages integrate bold pinto beans. carefully re +6429|Supplier#000006429|25JSp02FAStUL02dHWDmMMhjnoCyx7toe9U|18|28-981-964-2827|8481.23|. permanently unusual packages use fluffily. accounts affix quickly alongside of the +6430|Supplier#000006430|F2RrkeaNcs6po8x2PyYvcPa1rtKd,fT2AMxP|21|31-286-383-9133|4152.89|haggle furiously after the fluffily final theodolites. ironic pinto beans engage slyly reg +6431|Supplier#000006431|cbUFbYgCcICskrbCmxmOxZQgvY8Oe0m|24|34-882-129-8462|621.90| regular, bold ideas cajo +6432|Supplier#000006432|HEGX9DKcGuIEGvyUGj0JQ|11|21-743-129-5051|191.13|riously. quickly ironic deposits unwind accounts. tithes aga +6433|Supplier#000006433|XFR4njr2MWVhePaXAXhoNqb,gJsyf,cK|2|12-971-857-5905|9328.74|equests boost slyly past the foxes. regular foxes are +6434|Supplier#000006434|9sNRbxkVSwYyuP|16|26-265-927-1012|3246.06|kages. bold ideas among the carefully final foxes poach fluffily ironic packages. quickly even id +6435|Supplier#000006435|xIgE69XszYbnO4Eon7cHHO8y|3|13-233-835-5544|5510.68|nal pinto beans sleep blithely against the even requests! carefully +6436|Supplier#000006436|dR1Wq 0Zx3AJ1zd3fIqE|9|19-802-581-4826|7006.85|ffily even requests. quickly unusual deposits breach +6437|Supplier#000006437|LBzpb2ohl uXX5eB0D7YYMMrtY,MIEtZgAQks0|23|33-970-347-1656|7189.13|cuses sleep fluffily among the sly +6438|Supplier#000006438|m3UcjwFrquBPPmK15v9j|18|28-477-827-9412|1688.92|nic accounts. furiously bold ideas nag idly. quickly dog +6439|Supplier#000006439|HFrNZl6C,ISfJPrjF5tNO7BUP187|7|17-474-346-5159|3580.49|fully ironic escapades affix carefully above the ironic courts. furiously unusua +6440|Supplier#000006440|RGFwLQetEihzvW4T8yqzcH5WcenjjMp|8|18-277-803-8011|5333.66|ular pinto beans impress regular ideas. final pinto beans about the f +6441|Supplier#000006441|RSa6AJeehSQXTKbOjFkd7eONHTDYDgFAQiQ|14|24-266-114-4009|1821.61|, final deposits. ironic pinto beans sleep above the even, final req +6442|Supplier#000006442|K iDKLdFAYkO0MHruw|2|12-642-123-4079|821.12| across the blithely bold e +6443|Supplier#000006443|0UvLTRsK68fRYmylwHb|19|29-761-310-8290|4425.01|are? blithely final asymptotes nag carefully furiously idle deposits. final requests boost during th +6444|Supplier#000006444|eu Ou8CTJTZmFkwON|17|27-721-454-1887|9212.04|refully ironic accounts cajole blithely after the even, final packages. furiously final +6445|Supplier#000006445|RKWTc3AfiWaI9UOh4MzAPikF|8|18-661-694-3094|-155.18|special requests. slyly eve +6446|Supplier#000006446|zc UqeKpg3W8Y|4|14-901-464-2207|7817.89|ts use fluffily. furiously regular foxes around the +6447|Supplier#000006447|Qu3COUmk3l|24|34-164-930-5009|9256.66|usly even foxes should are furiou +6448|Supplier#000006448|cGjwmT3KeZwzyDfi5navjO|16|26-320-296-3817|1717.41|old, express dolphins sleep slyly against the final pinto b +6449|Supplier#000006449|7Q3BIdoISAl8JI85e9IusWcgx,R|5|15-943-315-1987|9441.30|ngside of the even, bold instructions. silently even request +6450|Supplier#000006450|oMHwJtJnddbm hNg3Tvz,wqT7Y98WKdhIqg08Y|20|30-689-104-5182|3088.51|y quickly special packages. furiously bold deposits are blithely +6451|Supplier#000006451|cabW,blwKcgfkjfzb|12|22-178-750-1370|8621.51|he quickly silent requests. blithely even deposits across the unusua +6452|Supplier#000006452|J7l NvAtBy|4|14-248-932-7645|7821.52|integrate carefully even packages-- ironic packages about th +6453|Supplier#000006453|25kSp,1PUc|20|30-282-264-2979|4326.80|ress, ironic accounts alongside of the furious +6454|Supplier#000006454|komDCUR7dlXquaEJj|15|25-948-574-9647|8520.51|. furiously ironic foxes haggle quickly. furiously unusual re +6455|Supplier#000006455|oZZWxYiImfoIahkVvgjDK7|4|14-336-801-9381|4221.06|blithely ironic instructions above the q +6456|Supplier#000006456|FgbEnwK,bZvQgg0|22|32-701-665-5100|2399.18|leep carefully about the blithely pending instructions +6457|Supplier#000006457|L8ka6BZ3JZfCDmNRPk3mSV,h3|18|28-597-963-3831|5970.95|nic ideas are against the slyly regu +6458|Supplier#000006458|1wfeX ru80V7mKM6OQq8EVVA0|0|10-314-874-9392|9722.33|arefully along the bravely special pinto beans. quickly regular packages ca +6459|Supplier#000006459|4 pm9lBbjp62WaXPIyko,tED|23|33-618-537-8210|7347.70|pendencies are along the i +6460|Supplier#000006460|6rNdnHIAPAP4Zrl4pua lL6OV9gYN2DFlOjqgEZe|15|25-632-577-5023|4764.32|s wake carefully even orbits. regular accounts haggle carefully. ironic, reg +6461|Supplier#000006461|yCnkZZac6nTbjBq6|0|10-338-418-5503|2987.86|ges. busy, regular ideas cajole quickly. regular, regular attainments use +6462|Supplier#000006462|SxQBQUpcOEd|23|33-810-638-2473|-97.18|ly at the blithely regular requests. slowly regular asymptotes sleep quickly slyly special realm +6463|Supplier#000006463|7 wkdj2EO49iotley2kmIM ADpLSszGV3RNWj|3|13-140-645-3319|3619.36|iously unusual requests solve slyly against the packages. final deposits sleep close dependenc +6464|Supplier#000006464|IG7mlCryWzcA,eDg|14|24-356-621-2381|9600.63|oxes integrate about the pending, final instructions. blithely bold deposits among the re +6465|Supplier#000006465|R0ofppl4Gkm,b,U5uCA0YL9wm3el luro0T|23|33-155-333-2168|7814.08|ious dependencies. slyly regular depths doubt. regular foxes above the final +6466|Supplier#000006466|Tq2 yKGuSA2eJq|14|24-889-470-6280|264.29|alms wake quickly final ideas. requests unwind furiously above the carefully regular id +6467|Supplier#000006467|SrIv,c4Ikw2Cz6tlOrM Ek1XMR|22|32-518-126-6211|6808.81|furiously. final, final instructions sleep slyly regular, ironic in +6468|Supplier#000006468|eskj2YxXsvqnSrFjYwDWD,0CcYS2RpByLH,gA|22|32-785-783-3793|723.94|ly express packages. pending requests may wake Tiresias. carefully ironic accounts are carefull +6469|Supplier#000006469|lUtE6Q0REOZ4jCMioY27QBK4qcnVMjBQtl|23|33-616-515-6899|-843.01|ounts haggle toward the ironic, even deposits. regula +6470|Supplier#000006470|QVFuSQyZonHQDgwpMCkM8ZqouVxKivg3L,NN0z|7|17-615-255-5151|8509.91|lets. bold instructions sleep above the slyly final accounts-- pinto beans sleep quic +6471|Supplier#000006471|363dE4fDCx7AaS|9|19-226-259-4310|2063.88|accounts on the carefully pending accounts sleep slyly +6472|Supplier#000006472|UgcCfa2 5Y8a|22|32-960-199-3921|2598.92|ithely unusual instructions use slyly quickly iron +6473|Supplier#000006473|u,9PUBiVGKHnDoP,b|22|32-889-606-4292|8053.25|packages. furiously regular asymptotes engage furi +6474|Supplier#000006474|ahwuZZlUGX,5TvGmUF30WUPQ|4|14-201-182-3108|5045.07|ide of the furiously final theodolites. even hoc +6475|Supplier#000006475|kIsWI8VZKmY|8|18-450-232-8764|2523.28|ove the silent, silent deposits haggle carefully special, final asymptotes. furiously bold dolphi +6476|Supplier#000006476|t9x1qudRC4|5|15-381-545-9309|1782.84| dependencies mold slyly final packages. pinto beans boost carefully according +6477|Supplier#000006477|duOYjBd,nbU67p|2|12-120-666-4532|9081.37|slyly even deposits. slyly unusual theodolites are blithely. furiou +6478|Supplier#000006478|bQYPnj9lpmW3U|3|13-688-485-6277|3716.52|s according to the bold theodolites haggle carefully +6479|Supplier#000006479|MmGZ6AAzGQ5,ue1hU4G2zxoiZxCfBdIz|12|22-234-775-8368|3591.45|y slyly quiet deposits. quickly ironic accounts use slyly among the courts. unusual f +6480|Supplier#000006480|G6EMQfMmx,Slpyg|8|18-876-601-6046|2447.76|as. carefully unusual requests cajole. quickly even foxes above the slyly pending accounts serve +6481|Supplier#000006481|i7o5TKSv,bYHzagKxJA4M3rtfZhRHSMvDyy8Y|15|25-746-155-1214|8504.71| deposits impress. furiously bold accounts again +6482|Supplier#000006482|o4,qxBTiDQAF9SD5GbbGl3tB5,7jnQArlvUU0|12|22-579-880-1544|2640.11|osits. deposits nod. furiously reg +6483|Supplier#000006483|4t53VXQgd0pp02|12|22-822-534-6790|7311.79|y. carefully final asymptotes integrate a +6484|Supplier#000006484|7sWhOboigE329aPf6R9j9MttZ|2|12-185-533-2806|6294.94|g to the blithely silent de +6485|Supplier#000006485|1h3ozUyD11zAw|10|20-350-853-9284|-892.36|s sleep regular packages. regular requests haggle blithely. furiously +6486|Supplier#000006486|MPI18JUSDV|12|22-165-725-4850|3981.29|ven sheaves haggle above the ironic packages. carefully express gifts beside the fluffi +6487|Supplier#000006487|G0UqR 8vMfI,YBcHlwydjJtqwvW|12|22-230-835-2325|9460.36|ng the regular, furious asymptotes-- blithely final Tiresias above the special courts h +6488|Supplier#000006488|97rW1R9HcsaS9v8dMMmIGBKCT5nk6dk|5|15-767-873-3625|4335.88|posits sleep blithely slyly express packages. special, quiet instructions serve. dependencies c +6489|Supplier#000006489|1O90ndPLcE|21|31-210-687-7613|7136.84|fily bold accounts. blithely bold packages boost about the blithely final requests. quickly speci +6490|Supplier#000006490|Uc6PWa4z8J9RlL630WT0ni7e JpzrRqGR|7|17-618-365-9174|2868.13|ts wake across the blithely express packa +6491|Supplier#000006491|BR71X0IhQaDFXWHZ3rTN LHa,fX4LikF xfK|15|25-682-162-8442|706.63|us accounts haggle slyly slyly final dependencies. carefully final deposits sleep slyly c +6492|Supplier#000006492|H,X6eulSW3LVy0uxdM|3|13-423-274-6179|5296.29|packages against the carefully bold accounts wake fluffily thinly unusu +6493|Supplier#000006493|ojV f,sNaB6Hm7r,fknDVTL63raJgAjZK|3|13-420-373-9605|764.57| the blithely unusual deposits wake furi +6494|Supplier#000006494|hwm GtBNhnQNc,02WQ|7|17-470-322-7330|38.00|ress excuses are along the even attainments. even instructions wake carefully among the +6495|Supplier#000006495|vKOq9W5U9vh2,KuaVFibFDa2kqjUR4dkj2H|24|34-929-655-1316|9576.64|nstructions cajole blithely above the fluff +6496|Supplier#000006496|4o,E0cY32jMVQ|1|11-704-240-1504|618.00|to the even dolphins. entic +6497|Supplier#000006497|KwLq9nzwUhse jlv3ae506lN8KTo1poG|20|30-489-991-7455|585.26|lly even instructions affix furiously. blithely final packages at the regular depe +6498|Supplier#000006498|QgR rA3XAEiTcZEv|6|16-802-671-1709|326.68|sly quiet asymptotes boost always. furiously express +6499|Supplier#000006499|gysyqVaAObZmfBmz|18|28-237-157-1486|-575.75|packages sleep special theodolites +6500|Supplier#000006500|KgQ7AdHVOGdtGUeds0|23|33-273-488-3077|1350.66|ggle blithely fluffily express packages. final accounts sleep before the carefully special p +6501|Supplier#000006501|8SgyHYIWUpJstMX7FuHw3k3|24|34-947-812-7804|5536.05|rding to the regular packages try to play along t +6502|Supplier#000006502|rRm19GWGn3x|17|27-336-540-8414|4367.71|ependencies use fluffily regular packages. foxes boost carefully blithely final packages. bli +6503|Supplier#000006503|SoQkixNfgtyInyljxN0Q3J5jyBlX237M8O|2|12-211-133-9136|3148.87|xpress excuses cajole among the furiously careful deposits. quickly pending foxes wake sl +6504|Supplier#000006504|9 4mu9KG8RYcBl|9|19-674-607-2603|5.48|ructions nag quickly agai +6505|Supplier#000006505|UO8JYMQLbvfgP6aA h44B|20|30-863-795-9838|2052.05|ncies run slyly carefully reg +6506|Supplier#000006506|5WVwiK0Kh34VCgBEW,vnBQ|24|34-864-857-7626|1788.90| boost above the blithely silent pearls. furiously regular pearls +6507|Supplier#000006507|JsvEUIzH1,8UhsmD|22|32-367-593-1161|1077.70|cajole carefully furiously even requests. even, even in +6508|Supplier#000006508|RLfHOFrzvDo2OhR|2|12-316-460-3467|4865.95|ts wake blithely; always final sheaves affix accounts-- regular, final deposits cajole car +6509|Supplier#000006509|acnx8i SgMshmL4xmWDPPslVCZ|9|19-863-711-6784|6103.18|r the busy realms sleep carefully furiously final d +6510|Supplier#000006510|5fzRzkLbva7zT73538jbDBsB|4|14-903-716-7501|-783.51|slyly. express deposits nag quickly bold ideas. pending d +6511|Supplier#000006511|MwdvL34nlkXi rQUPDoqhkm9XaMx3WF|8|18-355-691-7466|7520.19| the blithely final requests. packages su +6512|Supplier#000006512|sMKlZGHvP0ib8Lpwl9OIb|11|21-404-615-4633|-192.70|s haggle. quickly pending requests haggle quickly pendin +6513|Supplier#000006513|DD3jWKmxv9PwnLPDwEJ92UPmUBJHrp2JZZTQ0|22|32-150-700-8635|4238.83|elets are packages. ironic pinto beans sleep slyly final i +6514|Supplier#000006514|5yIp smf4s36|12|22-113-782-5493|4079.01| regular, even packages wake furiously according to the bold, even instruction +6515|Supplier#000006515|n1HaaAgnJXyq0uOJUPZc4OtR6dFab998dsj5ojW|19|29-985-947-8800|1831.99|ts. slyly final requests cajole slyly express, special the +6516|Supplier#000006516|89XwFOC,hLRxGq5rL0txv0EM9F|21|31-220-151-6412|-864.50|ependencies wake blithely fluffily regular instr +6517|Supplier#000006517|lmHUsvpoIND0cyGpuS,uyOc1mB|6|16-409-145-2586|-749.47|ckly final asymptotes. furiousl +6518|Supplier#000006518|3ttN5q7Qk6y6F88PklDp|2|12-789-951-8212|7627.30| ironic packages. carefully final pinto beans haggle slyly. pending, re +6519|Supplier#000006519|rvFp1JCtAVXlE54m1XGBv44vs|19|29-607-892-5392|7096.75|ace of the quickly regular ideas haggle carefully regular +6520|Supplier#000006520|Xo9 7N0NtA2kFTKbT|21|31-548-301-3239|199.37|phins. packages sleep requests. slyly express ideas r +6521|Supplier#000006521|b9 2zjHzxR|3|13-246-267-1193|4744.95|s kindle. quickly regular ideas maintain furiously final theodolite +6522|Supplier#000006522|PS,nptEo6IBkGNCwlHeIHs,sIM06qoENMwHI1C|19|29-787-279-7974|7388.01|packages. carefully final deposits among the foxes try to cajole slyly ironic pack +6523|Supplier#000006523|QYA7LJ8f3qcqUW70f8x2 7nU9Xf1BRh20iV|11|21-675-692-5699|299.57|osits through the dependencies haggle carefully after +6524|Supplier#000006524|eaGKaiQ6KIdx|19|29-380-982-1928|5826.08|thely blithely express deposits. +6525|Supplier#000006525|,Yrjv1AKlEQSu1drjji0shdb6spsaEV|23|33-603-123-5888|6019.63|ggle carefully. blithely even foxes haggle blithely. idl +6526|Supplier#000006526|cnKnAomKmY|21|31-394-134-1288|2926.33|ut the furiously blithe ideas. bold, bold packages are. finally even theodolites +6527|Supplier#000006527|ppB3i,lkoK0Jm6r0O6TxV3T0lSo62ph0CrR|16|26-352-294-4384|-123.62|osits. blithely final pinto beans could detect furiously after the slyly express orbits. ironic pack +6528|Supplier#000006528|Kd8Qbk3b0IWegm8vNV0cXXre5|19|29-191-918-8100|-784.16| packages. platelets across the blithely busy pinto beans sleep according to +6529|Supplier#000006529|DCIcqQG,v02EcdiZFCN7uc3g|3|13-747-854-6478|7501.78|usly careful foxes will wake bold i +6530|Supplier#000006530|QyKRu05w,UoOZ0jm7 HBdG101ouPSI98iQt6Fk|24|34-362-259-1264|9411.49|d packages. slyly regular deposits wake furiously above the grouches. r +6531|Supplier#000006531|e5uNBPGGUf12YZPJVpd0|14|24-258-734-1073|7996.74|c requests thrash blithely carefully bold packages. carefully final id +6532|Supplier#000006532|Yqg7SYRlEiQHeGB3rrxy3MoD|12|22-561-546-8625|1280.23|ely final requests. bravely unusual requests impress along the pend +6533|Supplier#000006533|hjMcu1gUBGNcs4OE0u9jZNTCraIf|22|32-905-947-7274|9747.42|y express foxes. packages use ironically even reque +6534|Supplier#000006534|JLuCE7ib,ZdF0H0KR6xA1ScfpNk0VB0zgGaj j6B|9|19-740-622-6170|-87.69|closely. sly deposits cajole above the carefully regular ideas. +6535|Supplier#000006535|4pI9S,jeKmLLwnbZtusbZoe5za3X5Meluse Bixn|8|18-222-598-5337|9158.27|hely pending ideas nag furiously special, ironic instructions? care +6536|Supplier#000006536|n24 pSj4wdWDUbw|22|32-639-527-9813|9183.60|ts haggle carefully alongsid +6537|Supplier#000006537|NSZmLpbFozfkiHwCGXYd8Dua7ioULg|20|30-482-351-1243|5896.72|accounts use stealthily among the furiously regular p +6538|Supplier#000006538|P3cQB4pXMpKnmFaV6KoSH|5|15-992-941-8372|5262.96|express theodolites cajole furiously according to the final, ironic packa +6539|Supplier#000006539|sLBC76iRNMe57dw0rDZiE09EYR65YZMc88qbafxc|18|28-156-673-1119|1660.64|e slyly special requests. unusual requests d +6540|Supplier#000006540|5Ei79tKvLpWmzv|6|16-484-323-6983|1412.68|the carefully special foxes; furiously final accounts boost. instructio +6541|Supplier#000006541|gNLBAjeplBWFTiX|15|25-145-275-5921|2920.29|ual deposits. slyly silent ideas sublate. regular packag +6542|Supplier#000006542|q NausyQAzNHAeAz7ieqQFVSsfX3gw16|9|19-652-306-9099|8238.27|y busily express ideas. regular foxes cajole slowly. foxes before the furiously qu +6543|Supplier#000006543|u3 yVNH8Fa2hBi,My2|8|18-570-252-4956|1682.50|riously. carefully ironic accounts haggle furiously. blithely final deposits about +6544|Supplier#000006544|1yFDGy78U3qRK2fq1S|6|16-777-983-3958|5801.24|sly regular requests about the pending instructions haggle around the slyly pending reques +6545|Supplier#000006545|vBsEn74Yl5Xh0n87Jy|10|20-375-299-7293|6517.95|cajole special platelets. c +6546|Supplier#000006546|GWVVgkQ8axWn|4|14-917-296-1241|7270.49|usly alongside of the quickly regular requests. furiously +6547|Supplier#000006547|MM3qyaZ0mEk20pkJTtoM3qpggUC5|20|30-768-643-9197|5093.95|lly quiet accounts. pending, regular realms wake carefully special d +6548|Supplier#000006548|A0myAYVJ1uvzV8pSeU8PcQHIb|21|31-196-299-1842|-900.49|ainst the slyly pending pinto +6549|Supplier#000006549|IaBh,XI4ZAbNV8ge1a5mfF2a|16|26-859-202-3582|-554.79|r the pending accounts. blithely even instructions nag carefully. pending, +6550|Supplier#000006550|K,Cg3KfeS2VZ1ZOGoJVBtdeG7P3GwCtq|16|26-527-787-6375|1509.72|heodolites haggle carefully along the blithely iron +6551|Supplier#000006551|MNg1Qmb3xuq|0|10-251-464-8925|5857.94| regular foxes across the special, regular excuse +6552|Supplier#000006552|zA5d8 k3R WZ9yJ8kFGbzl9QRe4SBZoNN6fl|8|18-628-594-1404|2380.15| to the even instructions. fluffily +6553|Supplier#000006553|ukiIbIzUAxKXbJCRD|12|22-286-234-3766|5178.31|ickly final deposits are quickly careful packages. furiously +6554|Supplier#000006554|ZN,HiSqwAB5F|22|32-694-886-4964|8815.58|uctions. carefully ironic packages believe f +6555|Supplier#000006555|2qJ5eQsPt10yXTYGE4wQG5rAULeHH|4|14-611-111-9250|2481.52|ter the express packages. dolphins acr +6556|Supplier#000006556|M7HOWhcG73u7wvkJV5Gs YYuTeBKxcXXrqMmfty|17|27-491-862-7834|3874.46| blithely ironic sauternes wake carefully. blithely ironic ideas are. regular, permanent dependen +6557|Supplier#000006557|phvDCgafHUxva1goImFx5TyMUisPY1IWHmhK6M|12|22-238-932-9426|7054.10|se slyly pending accounts. regular requests are blithely furious +6558|Supplier#000006558|JWcbxwFjM4HXUy|17|27-921-648-8569|9325.95|al instructions. quickly ironic accounts haggle even, silent theodolites-- +6559|Supplier#000006559|wQXd8ML1z7KsEz,KUTpFmU|0|10-175-586-1002|366.84|ess instructions unwind fluffily at the blithely bold ideas! express, final hoc +6560|Supplier#000006560|iKk468EaypFxNorvhzXa3S2j|17|27-494-461-3214|3392.72| deposits among the ironic multipliers are slyly among the f +6561|Supplier#000006561|phvrwuWeMivBx7,6l19|20|30-884-693-4702|6398.20|ronic pinto beans. carefully bold excuses could have t +6562|Supplier#000006562|dZ2GI2vqp3B67V|19|29-109-614-5695|7553.37|its past the final deposits shall are blith +6563|Supplier#000006563|mIpIpSthVxvlQ0emprRXEmqkUirKkvXg N,QT|16|26-698-773-1700|7484.17|counts across the decoys cajole +6564|Supplier#000006564|2qWFeJqImGoT0Mo9mjuKiLoFZW0IaIlv|15|25-686-765-7730|-582.40|kages detect slyly. blithely final asymptotes haggle furious +6565|Supplier#000006565|93bSUVzAR7StP1gOL9UdiVGNrH2|24|34-626-753-9376|-542.31| even accounts nag quickly ironic accounts: ironic platelets cajole carefully dogged de +6566|Supplier#000006566|m HQUuYUa4zmWVbcjqTv8|8|18-726-992-8016|2301.99|ly final theodolites. requests use. s +6567|Supplier#000006567|SRFmBCIEgbhI4V1Zb9FUNUAiFD9BvT,nErMd0ne|6|16-355-676-2706|524.72|ly alongside of the quickly regular foxes. ironic ac +6568|Supplier#000006568|DJVrhRFDrVv,alKI3|17|27-226-930-9685|5904.77|ies. final, pending theodolit +6569|Supplier#000006569|sk5AhyyRnxAczOZ76XvjGU uDEnyHQSVIQ2|20|30-561-849-3346|8433.57|os according to the permanent packages snooze bravely fu +6570|Supplier#000006570|nE41Xzb7GYhcCF4|15|25-300-157-2922|6081.85|s. regular requests maintain furiously. slyly special gifts against the packages ar +6571|Supplier#000006571|3r2brZx9EKf,R5vlQkGWDhZGJ|18|28-791-798-9886|3674.45|e blithely slyly dogged deposits. regular, final theodolites +6572|Supplier#000006572|GNLh0uS8 T2ouk|9|19-330-728-5559|529.81|ithely. slyly special attainments sleep blithely acco +6573|Supplier#000006573|v6RXFq8uzgx2LYir8|21|31-341-125-4738|3495.42|hely pending accounts haggle carefully carefully ev +6574|Supplier#000006574|oSqM6ea26sTo8xPh,UxDepk3wHBCw|4|14-213-311-6861|5545.35|theodolites. blithely regular pinto beans might haggle carefully. fluffily ironic deposit +6575|Supplier#000006575|XUOOYMOM3MeFgdnzGWhrIAAf2racobwc3yY|7|17-398-776-1487|6694.78|excuses impress ironic Tiresias. furiously unusua +6576|Supplier#000006576|LIUs4NqpT1|6|16-159-763-6098|9202.04|lar foxes should unwind ca +6577|Supplier#000006577|q2zbWFHaulmNjbAoT8UXh|24|34-426-828-5678|7731.15| even pains haggle quickly blithely pending deposits. unusual instructions haggle blithely +6578|Supplier#000006578|YsksCvxXZYMfY7t12P|24|34-177-278-8859|3540.39| the ironic pinto beans! quietly even excuses at the evenly ironic req +6579|Supplier#000006579|Jgbu,YujvdzYzQnXvRRNjCVxy8N CTs0AH|21|31-565-158-9205|2359.71|ss the express instructions. slyly quiet deposits haggle carefully alongside of the sile +6580|Supplier#000006580|e8rZ0Xf769DxBp tj5k|10|20-197-678-2305|2543.89| silently blithely ironic pinto beans. slyly unusual accounts according to the final, express p +6581|Supplier#000006581|HEE2VBl28eR7OtwV|15|25-692-584-8380|7083.29|he bold deposits wake after the regular, ruthle +6582|Supplier#000006582|OK0BWKI5eWR7IZLOj |12|22-269-943-1304|4898.12| hockey players against the slyly sly accounts wake quietly slyly re +6583|Supplier#000006583|5XubeRuMEEM5o1pBOetTpIcr4J2wB2MXA|8|18-162-379-5398|7741.71|nusual, final ideas. accounts sleep about the blithely ironic accounts. si +6584|Supplier#000006584|QVFOiAnYb,UOVwtHOdPKk3ZlUfDlqTiDRAcdy8|10|20-772-795-4434|4974.33|totes eat regularly boldly regular dolphins. furiously pending deposit +6585|Supplier#000006585|MlKhzETUB7,v5GUr|15|25-903-191-8689|1621.42|ependencies. carefully even deposi +6586|Supplier#000006586|PW7VOUWGU RVU0Et89FHHc84|22|32-959-302-3652|2218.15|ic platelets. slyly special pinto beans haggle bl +6587|Supplier#000006587|i8kzRBraWO5XhLQ2LSRpZeAFo6GXnm|6|16-661-419-4572|-556.78| pearls. bold, express escapades among the asymptotes haggle aga +6588|Supplier#000006588|9S8MN3Z5e6,bA1vL9Icoj7oALFG|1|11-239-687-8913|1169.37|. blithely ironic ideas engage regular ideas. regul +6589|Supplier#000006589|,plO63VoVildNy|7|17-116-386-6680|5485.83|. never final packages alongside of the regular, final accounts haggle furious +6590|Supplier#000006590|eXjko8KE0ZwmkgtWytg|1|11-817-638-9323|7440.78|de of the fluffily final grouches. carefully r +6591|Supplier#000006591|ui4bWpVdjn|0|10-969-505-4306|7240.36|ess foxes cajole. even pinto beans wake fluffily. regular tit +6592|Supplier#000006592|SBoTfwf16wLnaLcPhvOWNyXqeMi|2|12-173-853-7482|4785.90| slyly final ideas. quickly final pinto beans +6593|Supplier#000006593|M NRR7IkrJWxfq9Fph aZRBlWeaIfCLKlsdyZ|1|11-676-929-1226|2690.88|ironic requests must have to sleep carefully against the bold accounts +6594|Supplier#000006594|Lsaex723qeymtiq|1|11-794-494-3265|2668.25|furiously special gifts. unusual, special requests haggle furiously u +6595|Supplier#000006595|lwibmHh DJcmwev5BxgqDtlBE82keiFxRJH5Q|7|17-920-614-7661|7323.65|quickly. final, final excuses sleep fluffily above the slyly fi +6596|Supplier#000006596|k8qeFxfXKIGYdQ82RXAfCwddSrc|20|30-804-947-3851|7599.20|ously unusual deposits boost carefully after the enticing +6597|Supplier#000006597|1672 fSWkBvfAP38n1wAx,u5XNCSd4Q0J9m9|0|10-264-545-2117|7826.05|requests sleep carefully along the quickly bold asymptotes. packages haggle bl +6598|Supplier#000006598|VRZvWPH2PbDw4Mfr5in|20|30-129-445-3409|3392.11|ithely permanent accounts; ironic +6599|Supplier#000006599|VOy94g6EkB|19|29-221-934-7969|7094.94|urts? ironic, special theodol +6600|Supplier#000006600| h0rM2C3AFiiZmEe3LZ|1|11-834-174-2914|2686.04|the final asymptotes. blithely ironic courts boost across the ironic request +6601|Supplier#000006601|eb66Yi0F,bZ8HKUY4CLqN|16|26-449-507-9529|9189.60|to beans shall haggle blithely regular foxes. even dependencies among the sp +6602|Supplier#000006602| ffC7h3Zjhi28cjKA|1|11-262-302-9138|7691.91|arefully regular packages are. furiously ironic instructions cajole ironic, ironic fox +6603|Supplier#000006603|KegZ16m9o3,15ZUJS9rt7a6Y5TU4s|9|19-996-749-9430|4044.98|ithely against the blithely final accounts. always ironic accounts cajole carefully regular, +6604|Supplier#000006604|WpvmKwKdhWwK|18|28-378-810-6675|1912.37|into beans along the furiously even foxes are special, ironic packages. unusual, even plat +6605|Supplier#000006605|e,4MVOFQkJQiUpyisXkZfP cJD|0|10-276-155-4210|8228.36|ests wake slyly unusual foxes: carefully ironic dependencies affix +6606|Supplier#000006606|aAV0t1qLnruMZDW3MJX6vie|5|15-893-401-2489|82.31|nts unwind slyly for the slyl +6607|Supplier#000006607|3F 2e2gqD5u5B|3|13-743-119-2349|6301.22|ayers wake among the unusual +6608|Supplier#000006608|ejKj,80wFoTwYU0Hg29QfUELnB,O9F2|1|11-892-425-6686|5578.20| slyly brave asymptotes. +6609|Supplier#000006609|80qGzNVlUr6jXFKaGU0RCVO|24|34-595-991-6078|8859.48|r the slyly ironic instructions believe stealthily packages. de +6610|Supplier#000006610|6bjbWMa6okoLOW36LwkfOjP |24|34-399-762-9864|3264.31|ctions. pending, regular accounts thra +6611|Supplier#000006611|RvtPW2ijJd5 y9VLdJjvTMpdPXg|20|30-365-325-1791|4236.57|t. fluffily bold accounts are carefully doggedly bold pinto beans. regular, +6612|Supplier#000006612|KeNLynpTmU6m9PCKc|24|34-468-673-9731|-946.71| requests affix carefully. quickly unusua +6613|Supplier#000006613|f,BpmOBBdvPnllIpNvvY6|14|24-288-900-1987|3521.48|xpress requests wake slyly about the ironic accounts. busily final asymptote +6614|Supplier#000006614|DUg VfHci9UBN2a3k1TyNwPw|15|25-248-487-9345|3491.52|es integrate furiously in place of the excu +6615|Supplier#000006615|61LDepEIpKUZdy9HSV5UFQQB1iAp4,4odqsrUDJ|9|19-817-277-5263|5990.00|deposits across the close, even warhorses nag furiously blithely final deposi +6616|Supplier#000006616|VuVCB,p7KWmzR1i68|23|33-315-825-7545|32.21|ructions; furiously special dinos haggle furiously. accounts doze furiously qui +6617|Supplier#000006617|S9TqwK9 gNaRQ5s8Kt,1Gu0q1vF|20|30-151-102-4779|323.80| against the regular dependencies. furiously unusual ideas boos +6618|Supplier#000006618|ZD3BP5nU5RKdrq6dTKw5DycYqoFnDXJYTtdnjm|1|11-187-565-8631|6075.04|ffix against the even packages. carefully final dolp +6619|Supplier#000006619|3iqawedRWdxI5ovcNN3Kh7nd|9|19-138-384-2442|-852.36|lar somas. ironic requests wake furiou +6620|Supplier#000006620|X5gmUW17s2FJ7VzZtH3P,DK|4|14-875-753-9706|5357.75|s. fluffily express theodolites cajole slyly-- care +6621|Supplier#000006621|T35LAZIU0Q|8|18-391-305-1312|-229.07| bold accounts cajole across the blithely final dependencies. ironic dolp +6622|Supplier#000006622|m 7juZ4mDJReLY tf95lkIQYUPi|10|20-248-274-7898|8937.75|pecial deposits was furiously furiously even deposits. blithely silent +6623|Supplier#000006623|TPCL9zGFF,mLfJt6rvGrRR09VBf0|10|20-315-283-3886|9833.40|ely ironic instructions sleep among the carefully speci +6624|Supplier#000006624|rlxJuUtYOaDRNijnzzDH,1ogemxKmJVLqpm0Vv3|20|30-623-320-7853|4021.03|y even asymptotes. ironic, pending pinto beans about the carefully silent ideas affi +6625|Supplier#000006625|4e8W9wdzG6VoH|24|34-711-680-8478|6503.83|round the quickly even accounts. regular pinto beans affix blith +6626|Supplier#000006626|ObYBeLozdoDjS7f791dqrXWBgfQR7xwOigm ,eVg|16|26-897-536-6470|933.88| carefully regular foxes boost slyly regular packages-- bli +6627|Supplier#000006627|FSFCGNcYFZLrCI2xsBepAe9i9,yfQW|23|33-321-376-1961|7652.53|c, final packages haggle. carefully final packages +6628|Supplier#000006628|fWQJKMZljR|13|23-733-619-5750|2047.01|gside of the carefully ironic requests! slyly ironic asymptotes haggle furiously arou +6629|Supplier#000006629|lNsTxICxfK3T06mS5Yf5,QbPw|20|30-675-996-4698|5073.44|ccounts. blithely final excuses haggle. furiously bold deposits haggle +6630|Supplier#000006630|tIX9iDX2BNAdXfAnQe|12|22-120-242-1555|6310.62|arefully above the even ideas. +6631|Supplier#000006631|oZJTRXYYSH2Ih|14|24-622-570-7871|9573.78| fluffily blithely regular accounts. furiousl +6632|Supplier#000006632|TkcuZHSWRFtos 0fylpyqk|19|29-554-139-5114|7585.24|s across the furiously sil +6633|Supplier#000006633|ZHottW682gzdgvuAtkqxOwy|9|19-977-101-9990|4244.23|ross the regular asymptotes. slyly special foxes cajole slyly ironic accou +6634|Supplier#000006634|IPNINbVIYnAc6zgUzdLLuJbU4JbkR4H|18|28-581-692-3079|6805.19|about the carefully thin theodolites. furiously bold pi +6635|Supplier#000006635|VGeWt93EdIE6QmpIExjVLseFt|17|27-437-500-2708|5804.19|ronic pinto beans. quickly regular packages among the carefully final platelets cajole b +6636|Supplier#000006636|uW3QoDKNqmMoQ|8|18-662-756-3294|1115.56|ckages haggle carefully abo +6637|Supplier#000006637|JPXJMMZSBwuQ2S0skz9SVNbqIdS fSZ |23|33-711-641-8750|3197.64|ges was? carefully even requests run blithely along the bold, +6638|Supplier#000006638|DwONGWZWPFxc8PO7|0|10-918-771-1784|5632.80|, pending packages wake slyly. special packages haggle blithely bold depths: theodoli +6639|Supplier#000006639|a2jKQLSoKnn8lHtl63x rXMPyjn|17|27-511-417-8478|2437.49|al accounts sleep ironically furiously final accounts. quickly even ac +6640|Supplier#000006640|V1hZ8aN7Fn2a1yRwXJtvtMejUDAqcEBXdqWsi|17|27-556-561-1969|891.12|sly above the even, ironic accounts. dogged, pe +6641|Supplier#000006641|yCeaywkJ4HDR8wcKo,PbkLPM8BvSkk q8UOO|9|19-605-797-7454|4080.56|ss warhorses integrate bo +6642|Supplier#000006642|N,CUclSqRLJcS8zQ|3|13-302-100-2480|6022.73|lyly. fluffily careful requests haggle. +6643|Supplier#000006643| IMlyO8MM4icUuEup5Y|13|23-752-508-7799|55.07|ions. furiously regular deposits will haggle against +6644|Supplier#000006644|sM18j0ks9xI|13|23-263-254-2472|-428.87|ounts about the furiously special foxe +6645|Supplier#000006645|66atrH65661mtBBwH,zYw2,G|3|13-972-520-8948|5827.53|ully. blithely ironic plat +6646|Supplier#000006646|5VczmIu iRxCVE6airwEYhXs2|19|29-200-662-3772|8337.57|across the furious requests. sl +6647|Supplier#000006647|MGroKuG,dU52|24|34-231-410-7689|9642.43|nt pinto beans. final, unusual depths haggle after the pen +6648|Supplier#000006648|NIqU9bODm5plkP|6|16-543-459-9176|6377.00|uriously special foxes nod carefully +6649|Supplier#000006649|JEr VUzDpQ0TDFuk07kfkwxWEC|5|15-750-526-1453|3191.39|gular foxes. slyly unusua +6650|Supplier#000006650|OiU5XM3Q8L1lI|22|32-192-761-9702|-260.71|usly ironic theodolites. furiously final requ +6651|Supplier#000006651|OI9gD4TmCEeMpD|0|10-531-794-1582|5235.03|xpress dependencies among the +6652|Supplier#000006652|bW70qLS2xm99YT,hqG06pKX7sE40wG|4|14-710-152-7796|5134.40|to the quickly regular acc +6653|Supplier#000006653|WwpiochhF7rKPsIqQguH|12|22-436-477-8684|7830.05|s. even, regular decoys are furiously. regular, +6654|Supplier#000006654| gMLElLOn6hxkom|15|25-961-756-2910|4633.01|ependencies. blithely regular requests haggle quickly slyly expr +6655|Supplier#000006655|ncrPA0O7hFroxmRn3io|11|21-353-986-2625|6919.41|regular instructions. dolphins wake after the idly ironic notornis. even, ironic theodoli +6656|Supplier#000006656|Bsqlxg0akZ0z5fvveCUNJgaHSHbFVIrCwr7|18|28-867-807-7373|526.92| affix. furiously final d +6657|Supplier#000006657|9a0mbaiz3,SVNBYkUX5nZrG|8|18-420-339-5474|9865.98|ideas cajole across the slyly unusual packages. pending +6658|Supplier#000006658|h25j,DUjfGv9,N1iHhAP|23|33-754-729-1888|1631.83|ronic packages. bravely silent account +6659|Supplier#000006659|iTLsnvD8D2GzWNUv kRInwRjk5rDeEmfup1|3|13-746-779-9180|1067.93|ges boost slyly furiously sly accounts. silent requests are carefully evenly regular +6660|Supplier#000006660|XvFLyA9DDpR4F7XEQrVZMvz zf,CAk|4|14-453-628-1249|5637.28|t the slyly ironic platelets. theodolites above the slyly pending depo +6661|Supplier#000006661|hq78u t,hA8yGLnlgdTLuoTwTXuxR|12|22-191-699-8789|789.11|even frays. fluffily even accounts haggle furiously. pending, even accounts cajole +6662|Supplier#000006662|qgDqnB8G1iyhqBQpyMYU 3XrdVS7TzmzCrC1PM|11|21-797-491-9676|5534.65|s across the careful pinto beans are boldly across the regular +6663|Supplier#000006663|aWsfP,BNP5KKnmFfOSQowkHg|4|14-735-113-3127|2896.21|use fluffily among the carefully unusual requests. ru +6664|Supplier#000006664|djmUye,bf2QR4ly8onJcH|13|23-298-402-1869|-492.37|yly final escapades. slyly silent d +6665|Supplier#000006665|v dnTfUCHnaXzw7dN8ZSawQPuKqce54|3|13-219-224-5237|5731.66|al instructions. blithely +6666|Supplier#000006666|MdBp9zbA2xxhTMHo6YLut4|22|32-542-658-4737|3005.56|nt requests according to the blith +6667|Supplier#000006667|UO3LbziJTUvl64vlXNrIq5Zuu6FX1nkRew0DTkr|20|30-286-112-4408|393.66|y carefully ironic ideas. carefully pending foxes hinde +6668|Supplier#000006668|c3En1x00tCL|14|24-895-273-7175|3128.08|ironic packages hinder furiously ironic accounts. pack +6669|Supplier#000006669|NQ4Yryj624p7K53|3|13-828-116-9742|2697.64|es. slyly bold requests boost. even deposits cajole fluffily. fluffily ironic depe +6670|Supplier#000006670|rItTxnc02OrhI0jSERLu2z|7|17-741-390-4794|6130.22|nts nag platelets. theodolites boost fluffily b +6671|Supplier#000006671|TP0Qu5qXGOvVx4p2j4BPxnvNTlcobjz,9Z|1|11-876-975-7891|2595.79|lly regular pinto beans sleep carefully quickly regular dolphins. ironic instruction +6672|Supplier#000006672|s33z5RNSRRSH6R3OwFpQmraGBaSf,gm,puh|6|16-269-662-8552|-580.83|press slyly regular ideas? furiously bold foxes after the final pinto beans affix after th +6673|Supplier#000006673|mAGjBpXoqVNYqcDlxjYlzWjhmJe|13|23-836-803-7300|3955.35|accounts cajole quickly stealthy packages. ruthle +6674|Supplier#000006674|uxsle6 lQxuyRMYrHjEJS2RNxgT|7|17-468-954-1940|2475.93|quickly express pinto beans haggle. quickly ironic ideas boost blithely aroun +6675|Supplier#000006675|r86gT qdVa1SToXdwSWGMcEHR|11|21-643-398-5866|8908.01|ccounts after the carefully special asymptotes +6676|Supplier#000006676|pIyeSgJYXbGYTPR5h|6|16-637-528-1730|5069.92|onic asymptotes after the fluffily regular dolphins mold along the accounts. quickly sly accou +6677|Supplier#000006677|sB1ICehk,w|13|23-675-601-6718|2141.61|arefully. bold, thin instructions cajole furiously final, unusual instructions. slyly express foxes +6678|Supplier#000006678|U,iqrG67OmH|5|15-685-267-5244|9.41|arefully final gifts boost carefully. slyly careful asymptotes wake regular requests. i +6679|Supplier#000006679|lRBQi9D3IWGayjQhcw|0|10-778-528-5351|2161.13|fully regular dolphins. unusual, regular +6680|Supplier#000006680|6P ,,FjHsgKlqSuGA0BEXdy XfYSl|15|25-937-193-2337|9013.06| among the regular dependencies cajole carefully +6681|Supplier#000006681|oHMogYLVxq0jaf1f 1p|11|21-117-184-2175|-456.30|nts. furiously final requests believe-- bold asymptotes will have to integ +6682|Supplier#000006682|d0qmzarF6hAWYJH|20|30-940-391-6322|6936.21|n requests. regular, ironic requests wake carefully: furiously regular packages are blithely +6683|Supplier#000006683|2Z0JGkiv01Y00oCFwUGfviIbhzCdy|6|16-464-517-8943|7843.52| express, final pinto beans x-ray slyly asymptotes. unusual, unusual +6684|Supplier#000006684|0vCaC46Y6l9E,ll,ybX2QdcX96C,Gt|20|30-566-475-2493|18.79|ages at the furiously special pinto beans sleep carefully silent, silent packa +6685|Supplier#000006685|E4KuBERFrh9BKiU|0|10-131-953-5444|5793.83|y even pinto beans detect evenly. brave a +6686|Supplier#000006686|Yac23OocaPvt1W|16|26-698-751-9073|2989.28|al platelets according to the quiet dolphins detect pending ideas. carefully pending accounts s +6687|Supplier#000006687|b7NzEGGTXOcdi9V0W2csyOijvEypK|12|22-770-229-5068|1882.86|al Tiresias boost slyly about the slyly close pearls. ironically ironic +6688|Supplier#000006688|7la7usoFEFhQRTyE7t71,u,KcY5u8fCJ|11|21-329-778-1273|7191.46|ly above the furiously bold instructions. regular, regular foxes among the furiously quic +6689|Supplier#000006689|u,oKtIl0c8DB,H8DtPNvk|14|24-684-648-7852|7841.47|refully ironic asymptotes. carefully ironic packages are fluffily ironic th +6690|Supplier#000006690|qth0Sp41EDkhCnGTLN9698YMQyu G|23|33-403-164-6874|9570.19| alongside of the ironic accounts cajole sly +6691|Supplier#000006691|,r4UTxGeXio72O6ZRqL3contwCR|24|34-456-721-4261|2276.32|efully quickly silent platelets: regular, fluffy pinto beans dazzle after +6692|Supplier#000006692|eNFxFbbCFKdk|14|24-300-371-1687|9069.47|lent excuses. fluffily silent packages nag carefully according to the final platelets. sly +6693|Supplier#000006693|RcQEvgOXhyvJOQndULyO4DWl|20|30-374-257-1774|1710.29|the regular, even pinto beans. patterns are thinly according to t +6694|Supplier#000006694|v4hTv0CKaMhfDsX5N0b2d0tVpO7vm5tBP6|6|16-122-738-1212|-277.33|nstructions cajole quickly ironic deposits. even ideas boost furiously a +6695|Supplier#000006695|8PQEZ8ubOtqYg9G4cI4e18URQwBAj|7|17-237-232-1067|773.18|dle deposits. packages according to the bold, even gifts nag above the f +6696|Supplier#000006696|ljWztaEfF576Y|10|20-998-715-9446|1705.20|kages boost ironic courts. re +6697|Supplier#000006697|oUoBWC43kunlHNyObqoJEmd4c9WLeAhZSh|15|25-520-716-9359|2265.36| requests after the quickly +6698|Supplier#000006698|gyjSPCY4GC nc x|14|24-695-425-3045|933.77|ounts. final, ironic ideas cajole fluffily quickly regular +6699|Supplier#000006699|H,xWggrqs1r6DvcvKFr5KYpQ W9yQ4iDns6Blf14|23|33-214-777-8854|5851.35|es will integrate carefully epitaphs. quickly even accou +6700|Supplier#000006700|BWjerJH5kbEPu 8h9|21|31-280-796-8202|1677.37|oss the final requests wake carefully carefully regu +6701|Supplier#000006701|eu S14KUaD i|14|24-639-417-9664|7553.38|lithely silent foxes. fluffily final accounts doubt furiously. theodolites sleep carefully above th +6702|Supplier#000006702|9AhU1MrHSCnKJQio l5sEd0zYF1|14|24-413-421-7609|7651.96| express deposits. furiously regular packages about the regular packages sleep flu +6703|Supplier#000006703|RMQe,2nNcihLXXr7OzRybP,NAE|23|33-185-555-3648|3456.23|. final deposits use furiously carefully even instructions. slyly +6704|Supplier#000006704|97rxJlAImbO1 sUlChUWoOJ0ZzvQ2NI3KI6VDOwk|10|20-588-916-1286|6880.18|old accounts wake quickly. ca +6705|Supplier#000006705|8nS49d5oBs0dWYx6Vf6dWhtWzozc|20|30-660-386-1612|1302.17|s haggle carefully ironic pinto beans. unusual pinto beans at the unusual, bold asymptot +6706|Supplier#000006706|Ak4ga,ePu1QZ6C3qkrqjosaX0gxvqS9vkbe|3|13-412-723-3327|4521.52|ests against the furiously regular packages are across +6707|Supplier#000006707|INAEWdWz3CtsdfoO|13|23-723-452-3152|3170.76|rding to the blithely regular packages wake furiously final asymptotes. carefully regular ins +6708|Supplier#000006708|zXJlfd qrkfQltGI8c|16|26-462-518-7306|5023.10|uriously ironic somas. special, regular ideas +6709|Supplier#000006709|55mmIOYe07Rz2qebkhmo405t0Mkfck,|5|15-704-822-8514|7262.51|endencies. ideas are slyly. slyly ironic accounts use carefully alo +6710|Supplier#000006710|QTW4Ykw8brmlqB8tfndv62o|11|21-708-909-2822|3110.10| boost quickly. even requests are quickly blithely +6711|Supplier#000006711|i jqPUnq8DM8S5UHIuiNHxK,iv|12|22-657-264-2458|8410.80|bold theodolites. unusual dependencies sleep blithely even theodolites. bold acco +6712|Supplier#000006712|DflLd0CRqf2aTVkRVgMMrqai0sysHxBAkV|15|25-775-848-9286|8396.46| furiously even attainments. fluffily sile +6713|Supplier#000006713|JjmCvBijjmCzRJBXy0Up5EjZ9eM6o9bw|7|17-366-234-7745|9312.95|ic theodolites are along the packages. blithely iro +6714|Supplier#000006714|OjQDnZvu7 ApsqC7h0ohCAbJRKh1ApQZmN8Jr|1|11-859-742-1776|1760.84|es boost furiously. slyly pending forges +6715|Supplier#000006715|SX47Fkv7ADR9IhG6mMDRaGuRcSyjMRi3A3ex5,|19|29-811-331-7891|2393.52|nic packages. blithely regular deposits according to the slyl +6716|Supplier#000006716|BJaPDjafIrQQlD4cP|13|23-153-259-3388|9895.14|ng requests against the slyly specia +6717|Supplier#000006717|VmfuQL38joPwfE6|10|20-573-215-2745|7782.83| haggle furiously across the slyly ironic requests. quickly regular deposits sl +6718|Supplier#000006718|7FbbD6wsyg,yHF5JmlofiB,f,s,t612hXI9z|2|12-845-848-1171|3131.35| carefully careful instructions nag furiously across the carefully ironic ideas. +6719|Supplier#000006719| LXrBw9PLQdqDeB|9|19-288-721-4630|2585.30|ounts. blithely regular pinto beans sleep carefully above the regular, regu +6720|Supplier#000006720|ylUyy,j2,U4UdToTQ08j4E2ltkiH|18|28-960-228-5409|-868.57|re blithely about the blithely unusual platel +6721|Supplier#000006721|06QvzeHSKW|0|10-758-994-2567|7928.85|carefully even instructio +6722|Supplier#000006722|f3FX7aAQ6afOHw93QZ|8|18-454-448-1437|4223.52|ic packages among the enticingly unusual asymptotes haggle slyly furiously speci +6723|Supplier#000006723|zPUrD,ylISpYlQy6MEpB9FqCqOkK9AI|11|21-568-966-5500|-305.48|he even requests are even deposits. sometimes even deposits wake. ironic, final f +6724|Supplier#000006724|JnHhIl76r17oFyNFVwBHO3|7|17-720-850-6599|691.67|y. carefully silent accounts cajole blithely express packages. +6725|Supplier#000006725|m8eAP,MLxqxalhoCeWcK0ZRlGD7j|15|25-381-336-1918|6846.16|ainst the quickly bold gifts. bold instructions across th +6726|Supplier#000006726|nLoBfJ42KEnu 9ZWyu937SyPI,EQJ1mswcORSLrp|23|33-802-359-4814|1415.19|aggle close requests. evenly express requests haggle carefully. furiously regula +6727|Supplier#000006727|6z1So4EzSpV7MLcfChPq 4j|7|17-590-653-8773|995.43|nusual deposits alongside o +6728|Supplier#000006728|U8c2Hu4L1OG7ARgvc0eI4S|3|13-143-487-9465|8121.13|er the ironic, special deposits ca +6729|Supplier#000006729|9wa1GsLYHZq6WdiWSdr6RLmolbqM1K|15|25-497-757-9563|1809.42| the furiously silent deposits. ideas wake about the permanen +6730|Supplier#000006730|plZmlM0yb4Q5OgL11,obH1ycelwAWXx c|21|31-526-874-2001|4850.57|ts along the ironic instructions sleep quickly along the depend +6731|Supplier#000006731|qw1GNdIyo0qJa2IX6w|11|21-226-983-4516|611.06|sly regular warthogs after the quiet, regular requests wake slyly inside the +6732|Supplier#000006732|DHsC3w09xHfDv4O|22|32-396-161-1647|5059.17|osits poach fluffily after the even packages. even, re +6733|Supplier#000006733|WXXy8cTwPiDV1BsLl9173TFWYjkyT0N4|4|14-526-623-3438|8432.02|ng to the dolphins. carefully pending deposits cajole furiously even requests. p +6734|Supplier#000006734|vmaZ8HNigX9UX|14|24-959-545-1083|2053.38|es besides the carefully even +6735|Supplier#000006735|Wjnzex5KZDAFe,NS|8|18-210-756-1470|2190.92|ages wake slyly across the carefully express deposits. pending requests detec +6736|Supplier#000006736|kJsa2l74Y5J u3C QUh21,sGG|19|29-504-137-3715|3685.57|sits. carefully regular packages alongside of the frets +6737|Supplier#000006737|7PLN81E2Ma6ltHkP1daCKbm|20|30-554-918-4617|405.68|he blithely even theodolites: slyly special frays against the quickly final excuses shall +6738|Supplier#000006738|JJjGB5ZgaSW0uJDVi1c3B0aD9k8|2|12-634-194-1116|7233.20|sts. furiously regular packages play carefully about t +6739|Supplier#000006739|EwtgRIn Qs,u9ySJtADD6Gu6V |22|32-859-389-6252|5370.44|ke fluffily unusual, final instructions. accounts sleep carefully +6740|Supplier#000006740|YNzKcJ4mxt02JGvq8c5im2LPglWONcsrtyO|15|25-554-623-3540|4040.11|the bravely final requests. quickly regular sentiments are fluffily. regular pinto beans run excuses +6741|Supplier#000006741|v,63qKk6xB4bH3Za0b5|16|26-128-837-5709|8187.68| boost fluffily after the slyly final requests. slyly special deposits boost furiously across t +6742|Supplier#000006742|Z5JdKmVw3X|11|21-591-253-3872|9311.64|slyly pending accounts could haggle against the regular pinto beans. express +6743|Supplier#000006743|U5yY1aUP36fVbSmsxBoRmHmH|4|14-915-345-8665|1807.96|ly alongside of the slyly even requests. blithely ironic packages d +6744|Supplier#000006744|ybeO9yuPgFQ2W4CSQw|0|10-746-930-8387|2344.70|posits. regular, regular f +6745|Supplier#000006745|aVTZhpRHiqAeX, SrlxjZE34nw1,rFZ|12|22-609-129-9551|3278.05|ely at the deposits: even, even courts haggle blithely sentiments. furiously bold ex +6746|Supplier#000006746|kbVg2yNJtBmanF9D pDjnEvj|4|14-798-330-5013|4207.94|e bold excuses nag after the silent ideas. pending theodolites haggle slyly against the silent foxes +6747|Supplier#000006747|Z1bUvWZL,uXLooW6jrf1|7|17-544-782-4892|1890.85|nal theodolites. special deposits haggle sl +6748|Supplier#000006748|rC,2rEn8gKDIS5Q0dJEoiF|3|13-738-629-6562|9476.20|al theodolites are across the slyly final dependencies. slyly express instructions engage after the +6749|Supplier#000006749|817NF3IQgdL6771WyUqN6o1ZQ5|17|27-553-500-9046|6500.44|ecial accounts: daringly special requests cajole carefully. +6750|Supplier#000006750|nYQx9ZK0xmoFMbSf|2|12-128-392-9206|9865.54|ans. even forges cajole carefully ironic dep +6751|Supplier#000006751|kLs225lhdhLLN1P|18|28-878-158-9686|5277.41|ickly thin accounts are across the fluffily regular requests. car +6752|Supplier#000006752|i8fWpSw62jh|18|28-968-869-9241|2403.02|carefully ironic deposits. quickly bold packa +6753|Supplier#000006753|vkoG26rE,dMq6n5 1tmQOW2wrU,xyXEahegZN,|6|16-508-617-7667|4582.83|unusual instructions according to the blithely pending excuses are about the accounts. express, +6754|Supplier#000006754|kMUCGph4PWmLLdBxHBZCypMzEoZ 1MVMSkl|14|24-579-493-7622|-363.91|en asymptotes. carefully regular platelets along the slyly even requests are furious +6755|Supplier#000006755|2Ntn6GMUdXtbz,B4|15|25-475-448-2362|8759.95|l, final asymptotes sleep closely quiet packages. accounts alongside of the blithely slow a +6756|Supplier#000006756|QQDPe79chxWaEbERpLDL6hg,uDS|20|30-811-893-7042|2605.69|ickly alongside of the even, final Tiresias. unusual +6757|Supplier#000006757|o6fggGBhInzj1FNFIMXTQuazwNK lN9|20|30-672-585-8438|822.91|etly even requests. carefully ironic +6758|Supplier#000006758|4WLcLJOz3DV3|24|34-988-450-5226|1593.82|ly final realms alongside of the regular accounts cajole carefully slyly regula +6759|Supplier#000006759|KFoVyGtWziHB|6|16-843-345-1679|210.89|kly unusual requests are carefully slyly regular requests. packages among th +6760|Supplier#000006760|UGCtNGBPw0Fwj|18|28-180-247-9445|283.42|ntegrate. furiously regular deposits cajole accounts. regular dependencies haggle blith +6761|Supplier#000006761|n4jhxGMqB5prD1HhpLvwrWStOLlla|3|13-806-833-5081|9376.37| the deposits boost at the furiously final deposits. +6762|Supplier#000006762|xlEBW64hMxE59TgkMBq7|13|23-233-827-5081|691.05|riously sometimes final platelets-- even courts nag across the quickly iron +6763|Supplier#000006763|pB6ZoSr8Sss0b8iFns2wJh5D|13|23-794-361-5382|7487.37|s wake. fluffily ironic packages serve carefully ab +6764|Supplier#000006764|caeE02gh59 c7Q6m9LPAJV f0W,OCVWjn8I|3|13-458-611-1601|9287.24|nstructions. regular, regular frets do +6765|Supplier#000006765|q3Y7IQR6N3tp Br04nwI4Rq|2|12-265-854-7709|9681.10|as haggle furiously pending, bold packages. blithely final ideas s +6766|Supplier#000006766|IHpjESJE1BfTfRWlvR3z85qq8R|12|22-775-674-8795|7272.77|ions cajole. final requests +6767|Supplier#000006767|Yje6EU5noxDU8o4LUg9vRDHQUiLpt4x9im5N|20|30-554-349-2194|-586.84|t slyly regular pinto beans. regular packages nag. regular foxes are carefully. +6768|Supplier#000006768|Y0gGNRSEVWfV5aDW4YlNX5nqz|0|10-386-623-7588|1945.53| ironic accounts was stealthy tithes? blithely s +6769|Supplier#000006769|zV2bQ8CsFCELz,WMmV7LjYKIIMv7d9tcAGjtQAP|18|28-306-438-3579|4533.15|hely. blithely bold packages near the bold idea +6770|Supplier#000006770|wWzv6yqskaiASMZunvWpcUaFl|17|27-839-943-6295|691.54|equests after the carefully final requ +6771|Supplier#000006771|eli9DBu1IuS88v1ZvailRSd3lOI|7|17-321-864-8504|4691.67|ructions thrash furiously fur +6772|Supplier#000006772|MwWpwKvmrUayCLYwGVtFlPPsm xoNGb|7|17-248-919-6693|2903.64|onically final packages integrate. slyly express foxes above the specia +6773|Supplier#000006773|LkkyhDBh7bTAPvzFq5HUmWo2|24|34-196-486-9760|3428.70|haggle furiously among the quic +6774|Supplier#000006774|svP9cI7FSB 3a|18|28-425-377-5849|6160.91| furiously thin ideas nag slyly furiously special courts. q +6775|Supplier#000006775|aph5vhayqx|24|34-175-566-2401|1447.70|foxes. slyly regular asymptotes may boost boldly slyly slow theodolites. sl +6776|Supplier#000006776|ni714Zq w0eayXDfODJLP8BsCltjz6EXR|7|17-220-322-3622|8452.00|s packages wake furiously. requests wake slyly according to the special, ironic escapades. qui +6777|Supplier#000006777|Fk4ar1mbQLcZ,QscUyNDIenEMGFjjAsbFhM 4vrk|17|27-303-982-8941|7886.84|ending accounts. regular deposits sleep furiously regul +6778|Supplier#000006778|fYY4Fb1rFZFEetR11tNQGoQxk|11|21-819-720-3101|2106.72| the carefully pending ideas wake about the express packages. carefully regular pinto be +6779|Supplier#000006779|opViso sAFIJjMfb5aPtA5j3Hlbr9|0|10-723-759-3094|7641.65|cial deposits; blithely ir +6780|Supplier#000006780|VXqCDoOKm8uMqzdOlObgHutnby5yZYhsoELTkBh3|2|12-332-117-3090|6184.83|platelets. foxes haggle carefully pending pinto beans. patterns haggle furiously furiously +6781|Supplier#000006781|yq0CkqVOtfZqGGto9RnU2LncAfB5Mj6fTP,0I1sn|6|16-273-828-1660|7438.01|are blithely silent instructions. carefully even instructions +6782|Supplier#000006782|Dt4TTM ,rc5mekbZe63Hzf2qx M,2XQsLUREfyyU|24|34-713-153-5267|4630.17|above the blithely enticing accounts engage bl +6783|Supplier#000006783|pYZG29KCyH XbMS41dTlJCeyqbkvIiOSDV|1|11-879-159-6019|3279.41|ress packages. blithely bold platelets haggle about the carefully final accounts +6784|Supplier#000006784|mK1m3ZYQydhYyaUmI6y0dFGfXWLliVX6whBvZV6|13|23-122-413-9616|5015.18|p final deposits. never fina +6785|Supplier#000006785|lyo6PpwulTeN9ZfIkvWag5NucL,XMC 89Kn7U|21|31-644-455-4622|6130.78|. final, regular foxes haggle carefully according to the final, bold +6786|Supplier#000006786|onV6Ztoe99QOOvl7cJukHA|21|31-930-330-2985|992.69|xpress packages are furiously blithely unusual courts. deposits sleep against +6787|Supplier#000006787|uc7YzttoH5LKqbaQSKsX|7|17-353-912-8165|6158.33|gle among the furiously final +6788|Supplier#000006788|8104AC BwYFTnMX,mvS9289cbToqLx5sUxLk|1|11-878-632-8562|2108.76|ily final excuses according to the blithely ironic requests sleep furiously regular deposits. accou +6789|Supplier#000006789|SQdi546ULN,A x5rbMgOl5JyTdWR|2|12-894-357-7064|8847.12|according to the quickly bold requests. even foxes cajole. ruthless, special deposits cajole final +6790|Supplier#000006790|oqjOciDKJPSSVtvu6wfFF32Sco|13|23-442-147-4424|4612.50| to the slyly express acco +6791|Supplier#000006791|3mzLq8uKh4nce9fv4cJZZyGzjCHF0,sLBc8AmTlB|7|17-778-162-6650|3234.09|le carefully carefully unu +6792|Supplier#000006792|d6zeMkuKyz|22|32-251-872-1479|2295.05|lly final accounts snooze: regular deposits wake thinly slyly final packages. furiously sile +6793|Supplier#000006793|mJ4F2 l ZR8boTDtN|18|28-489-867-4830|4866.74|sts play among the requests: slyly pending theodolites nag according to the asymptotes: sly +6794|Supplier#000006794| CIDhQ,JzqcCTauC|7|17-945-768-1700|8335.31|ly special Tiresias should nag regular, regular requests. requests are carefully ironic accounts +6795|Supplier#000006795|tsjzf,sh701ueTQS3Hv0XqAG1q,0LsyX9k|20|30-695-128-5947|4601.82|boost carefully about the blithely regular accounts. sometimes regular packages +6796|Supplier#000006796|LP,d1tDOO29q2XNpjssPfKAcpfUjaUP0|9|19-568-927-8554|-249.91|he even gifts. quickly regular packages at the carefully unusual requests wa +6797|Supplier#000006797|9Gm3zO9oS5PTNChz0 E1R8H0KZu67CLD|13|23-421-897-4578|-369.50|its haggle boldly according to the blithely bold packages. fluffily ironic pinto bea +6798|Supplier#000006798|1qkM6d7VA1UBS,9Yj4 zyVgDkLtUMpymLEZZS|16|26-322-315-1118|2667.57|tes sleep above the slyly ironic accounts. even, brave foxes wake slyly. express asympt +6799|Supplier#000006799|4ev9pKH Aa4Yt8h|18|28-676-634-9579|6783.88|e carefully bold packages are daringly above the slyly blithe depths. special packages wake alon +6800|Supplier#000006800|trVyLGUXtwe7a porF|20|30-822-828-9691|-317.89|s. furiously final attainments sleep alongside o +6801|Supplier#000006801|XVh,3tCaWuxEWw31t|15|25-698-334-6000|1179.82|as boost carefully final packages; carefully +6802|Supplier#000006802|VLU KFIpY3rQyGMN T8|2|12-505-683-5692|5156.14|ts wake furiously. theodolites after the express accounts dazzle fluffily slyly pending +6803|Supplier#000006803|g9Dd93lhEp9TH5AicacXS,s8tHU2Sm7VSqh|17|27-115-320-6495|-811.90|uests boost carefully special pinto beans! slyly even deposits detect slyly. pinto beans against +6804|Supplier#000006804|tLWrHf JErAkH3GiidHmewVkOyyyf|19|29-180-526-2807|3277.69|ts maintain carefully. special accounts cajole alongside of the +6805|Supplier#000006805|,TIyURnd1chWJuhWdcQtS|12|22-163-139-8839|9173.20|efully escapades: blithely even dependencies integrate slyly quickly bold requests. ironic +6806|Supplier#000006806|IMy5ZwsFKxspxOGlrs7CLPbV|16|26-274-468-5614|6996.24| final theodolites nag carefully. instructions around the slyly pending attainments b +6807|Supplier#000006807|4rZ7qKEOp 2YjbQJZY IXTug7a|16|26-955-524-6808|4835.57| requests; quickly regular packages are slyly. instructions among the slyly even sauternes wake +6808|Supplier#000006808|HGd2Xo 9nEcHJhZvXjXxWKIpApT|3|13-436-397-1525|1474.02|p furiously ironic courts. special deposits engage fluffily above the furiously iro +6809|Supplier#000006809|iaSmZjRFKAJ1ot99CEg1M|22|32-332-666-8608|-230.34|press theodolites nag carefully among the fluffily unusual pinto beans. close, special pinto bean +6810|Supplier#000006810|fiDYW lauQ38y|14|24-126-432-6057|9665.93|cuses cajole quickly after the furiously even foxes. carefully unusual accounts are pendi +6811|Supplier#000006811|u15ozhK1f8FzvhykF9RvW 3z52aj0UT4KB|2|12-395-942-6314|5334.44|ns. slyly silent requests wake. regular +6812|Supplier#000006812|,sghf7YCSXRNwtOcE7fb4yfwKBBU|6|16-737-258-6577|3934.68|ct blithely alongside of the quickly busy requests. deposits boost furiously. enticin +6813|Supplier#000006813|7GFCEnduSKDJ3u|5|15-270-267-2414|7122.07|ven packages use blithely regular, final accounts. ironic, ironic packages since the regular a +6814|Supplier#000006814|83A GGXkYiww58|4|14-312-658-3824|7631.14|zzle among the blithely sly packages. pending excuses pri +6815|Supplier#000006815|ScsCDzfPudH8tEn28jy7phwc02RNJjqH9F|19|29-174-191-9073|5590.78|fily regular epitaphs wake. quickly silent dol +6816|Supplier#000006816|I9XsfxxkTTxiIleWJ|0|10-659-842-8621|285.57|ess packages nag about the +6817|Supplier#000006817|txd0Gjo9v4vG9of9RTlfFhg3YzBcCx,N|24|34-717-369-5926|1496.67|nal excuses boost. slyly even dependencies use quickly stealthil +6818|Supplier#000006818|JzXudpt6MUt3gndOcUTkl9cQD|4|14-438-215-6560|6417.04|ithely final packages? accounts are furiously about the theodolites. fluffily final packages +6819|Supplier#000006819|zBCTca5TCD9lhT,XF4fiGkB|12|22-734-954-3358|5192.79|arefully foxes. furiously pending foxes cajole carefully across t +6820|Supplier#000006820|kvj8tFQ2RKa4mzpWbqSgYeIY6a0LMRcKTP2xSGve|12|22-171-850-6598|-789.30|press requests according to the platelets integrate slyly quickly +6821|Supplier#000006821|KDpvnfTk5SzlUInH78Hl1tp8S7sgkZPJ|14|24-408-920-1588|9166.27| regular requests are around +6822|Supplier#000006822|13dOiArfC4Jnh5t6BDkj0mFLf2SQ,OC|12|22-554-730-3187|585.35|long the foxes haggle whithout the final, bold instructions. furiously special courts haggle furious +6823|Supplier#000006823|7pOXTwG4sC7JVgYFFyHAE5K3kyPJ2xed|13|23-123-271-8103|1914.09|ole quickly alongside of the foxes; slyly unusual instructions sleep +6824|Supplier#000006824|aBVKYb4T4N98fvuKJxYsMLR |20|30-997-948-6329|4102.28|es use carefully. fluffily enticing requests sleep carefully quickly ironic platelets. +6825|Supplier#000006825|HMEhRT5SMldr VoKyyO|20|30-339-441-4176|8022.55| the ironic, unusual water +6826|Supplier#000006826|jKW6CeLZMdd2Am0mxznE,E8MDrH6o,cAW5jTivxV|5|15-839-357-6200|6083.65|the carefully pending theodolites. b +6827|Supplier#000006827|,kaE2xJan7WCW|14|24-464-693-6384|1013.71|al, unusual depths use after the furiously unusual foxes-- furiously regular platel +6828|Supplier#000006828|JQdz6T9i7ahvrLZPBK2w8wTzLd|19|29-609-373-3716|7421.67|egular instructions. blithely special depos +6829|Supplier#000006829| PLr pxVMR1mk,IGWwSe7MiEk19r,Cn|19|29-340-513-9131|5017.94|. silent packages are fluffily +6830|Supplier#000006830|BC4WFCYRUZyaIgchU 4S|22|32-147-878-5069|8503.70|pades cajole. furious packages among the carefully express excuses boost furiously across th +6831|Supplier#000006831|30WWc5G73cb0Zj77hr,ckpBh kq0V9ebX1wE4z|15|25-284-452-1411|7799.60|regular patterns haggle against the slyly even instructions. +6832|Supplier#000006832|UgadmHfm2qXtLvuyK2MABgd7NGe0G2p0b9G|24|34-323-820-1126|9903.08| the blithely pending deposits: blithely ironic instructions +6833|Supplier#000006833|x4IaVRBV1hzeI5eXy5DXmLn83y mmbI|18|28-641-301-2578|9208.83|sts use carefully. slyly pending requests wake after the +6834|Supplier#000006834|AaevZtSW0U3ftE rxnX2UT55WmOsd|8|18-639-665-8911|806.52|its. pending theodolites are. final instructions sleep above t +6835|Supplier#000006835|ZjVTncEepyAhxkfHCdJlNUIVq6FtHM1Imwj23|23|33-653-506-8452|1086.68|d excuses wake quickly about the slyly re +6836|Supplier#000006836|fs,fRRfdtxXGT1Yujqd|24|34-471-400-9992|1746.80|wake ironic pinto beans. f +6837|Supplier#000006837|vokfcIaDMaf2hG|4|14-530-767-7768|8377.27|unts along the slyly regular instructions detect slyly unusual deposits. +6838|Supplier#000006838|MYdba9tj08kL5Q7FteV6b8|1|11-703-493-8571|5799.35|ly pending instructions. final packages are furio +6839|Supplier#000006839|ylo, 1pWf0xfyQmGenO4t1apcOX1|6|16-622-617-7760|7637.42|y along the stealthily regular accounts. dependencies wake after the furiously pending i +6840|Supplier#000006840|QfA24,Dr32kUAfj4kFDzgHU,a1Mh7jyxta|4|14-369-156-6102|1247.40|nts cajole blithely. pending theodolites bo +6841|Supplier#000006841|u96uvgvtworeR|15|25-228-261-9772|7948.52|ents. regular grouches are slyly across the fu +6842|Supplier#000006842|oR7e8UykKPlckXRecMqppxZG|2|12-399-336-9424|4794.72|ily. carefully special deposits sleep +6843|Supplier#000006843|JDyLWm,Nn8,Pkfx,9zB,iWgq 4|11|21-443-521-5425|7890.01|al, regular pinto beans sleep furiously above the ironic, even pinto b +6844|Supplier#000006844|I5Zl4KJ7 A7,ZELv Hrxca|7|17-123-209-1299|519.24|quests boost. blithely regular ideas haggle carefully accounts. express, idle accounts are. caref +6845|Supplier#000006845|FXxR9d7XcfP26Ze71bAk|16|26-655-526-4256|1742.86|after the blithely pending asymptotes. final, special packages instead of the ironic, +6846|Supplier#000006846|BdKq0qLm I2m1kByN,HjzwGzdzsHRkxfU|9|19-150-912-4818|8248.74|carefully unusual foxes use blithely above the sl +6847|Supplier#000006847|EkeQQQ2c4H60TR,oCwHz|13|23-662-633-2010|1572.25|regular, final packages snooze furiously. slyly unusual packages hinder blithely. even packa +6848|Supplier#000006848|W5Vk6EiCzsZ0|18|28-216-244-6996|736.60|re quick platelets. enticing foxe +6849|Supplier#000006849|Eo77Qmtl vTUb,ZLr9f3|0|10-944-712-6413|2972.91|excuses sleep fluffily. slyly un +6850|Supplier#000006850|0J4csyTaGBM|15|25-786-893-6950|1748.84|al packages wake according to the carefully special tithes. +6851|Supplier#000006851|MJaGguxWeT3VfMompj8IIUztvoi3,bCJaT|8|18-367-247-2589|9659.88|ts integrate carefully pending ideas. regular deposits doubt slyly blithely ironic requests. blit +6852|Supplier#000006852|x1v6PqA4Yi8KSZzq S|11|21-525-349-1994|2663.46|uriously even packages haggle slyly regular, regular deposits. final request +6853|Supplier#000006853|sc,rPmQqLxnBImN26PwFR7pgfCenQpbiVBcZpRj|13|23-336-326-2531|3679.56|lyly regular deposits sleep quickly i +6854|Supplier#000006854|Jf0FFX9tBXB sNmCycIpd|5|15-606-967-7758|9367.97|yly silent instructions ar +6855|Supplier#000006855|ytCDS VWibP|22|32-285-557-7800|2489.75|ly regular deposits boost. slyly bold theodolites wake blithely. furiously even pinto beans thras +6856|Supplier#000006856|P0TMrpdsH6oCXz0ba43wadaU4Q33CZtgyk|18|28-105-165-7642|-823.40|avely bold requests sleep among the blithely special excuses. accounts boost thinly +6857|Supplier#000006857|tclwRUXRt g K9thunFEfhaVP6wMul8I|17|27-737-864-6129|8032.79|es! pinto beans cajole quickly furiously reg +6858|Supplier#000006858|fnlINT885vBBhsWwTGiZ0o22thwGY16h GHJj21|3|13-654-336-4681|2167.28|ages. quick, final foxes hang quickly regular sheave +6859|Supplier#000006859|IH0G4Izp4WkPJ|5|15-395-879-1706|2360.40| sleep at the even, unusual accounts. unusual accounts use carefully after the final requests. q +6860|Supplier#000006860|,,wvwtQ,iPnSGUDE9mJ45MUVBjx4oRyI|0|10-646-424-2985|3140.73|oldly even packages. pinto beans nag quietly final ideas. fluffily ironic platel +6861|Supplier#000006861|bZz6ZxOpgInQ DJ6|3|13-188-614-8063|6305.48|e quickly final accounts. blithely dogged pinto beans are pen +6862|Supplier#000006862|iqEIDdYOlGfOa,H9|5|15-569-302-5521|173.97|thely according to the final, pending packages. platelets whithout +6863|Supplier#000006863|ElFcmfHn2n|3|13-952-862-9857|6913.09|ic frays. slyly pending pinto beans are furiously grouches. p +6864|Supplier#000006864|lQiU7pCJ0na2Eh|9|19-770-606-1338|4352.64|gside of the slyly bold ideas. carefully pending packages thrash quickly blithely express theodoli +6865|Supplier#000006865|Kd pDf1iQh8DZe,08pwRy 3BP12HO|15|25-864-452-7671|2899.11| ironic packages. furiously special dinos along the quickly even ideas cajole furiously +6866|Supplier#000006866|9nuulgAdSQFPJfRiF3htJljoFo|22|32-455-654-5044|4330.94|onic excuses. express deposits boost slyly. bold instructions are car +6867|Supplier#000006867|QqN,6nvxhphZJM84MvBXuadwxJdSwgC9N3BzG|18|28-106-503-2513|1099.28|ly. even deposits are fluffily among the carefully final accounts. slyly ironic +6868|Supplier#000006868|TR7ZhIdq1hL01b9u9DRGFTbEmt58L|4|14-265-241-8293|4487.45|ithely even instructions. +6869|Supplier#000006869|w1jwpeDGzLSkemFO75cNt20l2Rb|12|22-997-249-8024|2639.48|ole furiously carefully ironic +6870|Supplier#000006870|aHlcaI7YjJ9mMnVE6GvTjBqL5NcWoIjToy|5|15-736-432-7997|7780.13|boost slyly unusual, unusual dependencies. instructions sleep slyly. care +6871|Supplier#000006871|uyE sjDnosxcmA3SJHC4IO3klzjWO60n8rJYsfLm|8|18-669-429-8562|1760.34|otes nag. quickly even packages sleep slyly deposits. furiously final idea +6872|Supplier#000006872|XIDPiA7PLXCWK6SeEcld|3|13-858-328-4826|4982.95|s. regular foxes cajole carefully! bold, pending reque +6873|Supplier#000006873|gV4gphLrUceksCZpy,XCVxvh2I7|17|27-480-892-7114|3468.81|fluffily express foxes. brave accounts cajole fluffily permanently r +6874|Supplier#000006874|0pEC2MdwBUKMZlJr9R|7|17-789-226-5880|9301.36|luffily express packages across the regular deposits affix around the quickl +6875|Supplier#000006875|UERFmEvKRfeAsNL2tKKfWhIm2|6|16-439-339-6116|5060.33|ronic packages. even accounts integrate blithely abo +6876|Supplier#000006876|XaNImTYkih,p10 YQdM1wBhFznzkPN,h|15|25-784-248-4463|5937.45|refully regular pinto beans sublate for the regular packages. even ideas believe quickly furiously e +6877|Supplier#000006877|TrOIloTruKK4MGx9hf HDFtAwDFxhen8,vs|6|16-178-416-6628|7736.63|. deposits was carefully. permanently +6878|Supplier#000006878|cTqJN5XokXwY8xe,Tc|7|17-550-239-2865|7667.26|integrate slyly permanent requests. furiously regular excuses snooze regularly. +6879|Supplier#000006879|JB4WhwriRKofOxppl7nWJe27OBSBqXwhflKk8X|2|12-181-983-3781|7649.86|ve the unusual instructions! regular, ev +6880|Supplier#000006880|s6Sen8cTC,P4Rk1PKA49|9|19-777-889-1296|1208.47|ve the accounts. furiously unusual instructions haggle quickly dep +6881|Supplier#000006881|jhPkRK0FFkLtAZdl|8|18-929-390-6095|2644.51|y silent notornis. blithely bold pinto beans wake carefully. careful dependencies are. final reque +6882|Supplier#000006882|yXazSpOtfQSgwrLAWFQ6iX|22|32-628-473-1005|4256.10| carefully against the caref +6883|Supplier#000006883|7Jow14LNWvnADcfBjCQZuq8AMIuu4e,kf|10|20-745-172-4921|8208.02| according to the fluffily pending packages cajole slyly carefully pending accounts. quickly +6884|Supplier#000006884|LKEdePXRdzmYopcfKhEvM9aMgZ19xtKRwaFR35|24|34-671-991-3914|-758.94|ly final deposits. fluffily bold ideas above the always even asymptotes print carefully carefu +6885|Supplier#000006885|Z86smeUpgHKvZ,7tUWPM|5|15-625-719-7144|2375.41|ly regular packages. ironic platelets are theodolites. carefully regular dol +6886|Supplier#000006886|sN0BeIMWowAvTUOshFyAvm6bMb5TLphY|11|21-395-309-3477|2530.63|sits. blithely pending packages against the even attainments haggle +6887|Supplier#000006887|SJ3DVcv4HILypegf9|18|28-121-512-1798|4186.31| carefully since the ruthless, ironic packages. even +6888|Supplier#000006888|oiMuX7iexPrhOI0k4wOWBncFhNp4mvvmdusIq841|21|31-895-252-3066|1281.21|inal packages. ironic platelets nag fluffily. excuses are regular instruc +6889|Supplier#000006889|ppbqrbESBScOqu0,hRfPivHSINaa4Ya7|2|12-931-753-4177|4241.85|ly bold instructions-- furiously unusual r +6890|Supplier#000006890|nouw9WcbaK7PG3p57ZBmQp|14|24-479-791-5419|-279.98|foxes promise ironically depo +6891|Supplier#000006891|Sx9AC8YnEcejMh549HEYlMY6fc|6|16-622-852-5856|377.05|courts. special requests can integrate. slyly final deposits +6892|Supplier#000006892|9DEMqP3EF CkC4VC1lNhY9oKSfxtLV6s0|20|30-564-583-9703|9975.89| packages sublate furiously. slyly final asymptotes doze ironic packages. regular, special +6893|Supplier#000006893|1qx2LTrRiB6nflwp RPEsLbBRNQspABzH Z|4|14-513-183-2336|824.10|egular, even foxes. special +6894|Supplier#000006894| sgoktoW6JG,f|6|16-943-753-5572|-398.68|uickly regular ideas cajo +6895|Supplier#000006895|hIHfCilwAb|18|28-398-229-8466|7306.32|. ironic pinto beans among the slyly final packages promise carefully e +6896|Supplier#000006896|ZoVm4r7M,s|10|20-249-701-1620|2352.39|ly regular foxes. requests nag blithely bold accounts. finally silent deposits sleep slyly. +6897|Supplier#000006897| v2RKw2H8orc1A|17|27-396-482-2298|663.93|s. boldly final pinto beans sleep. slyly regular +6898|Supplier#000006898|XTzSOHWXM3ZwqTN r5WXE49f0T tn8HT|9|19-386-937-9547|2577.65| across the ironic packages. sp +6899|Supplier#000006899|Pbpjj0EGV9LcoqhoJT8BhDnDComelq3DiGWF|14|24-479-609-1113|2568.41|lyly pending deposits. requests wake. final pinto beans wake across the even instruct +6900|Supplier#000006900|slSxyCKtaGJXZ89vzUam5ckg8H2JA|15|25-812-757-1953|1179.12|ideas during the deposits wake after the ironic, even deposits. blithely reg +6901|Supplier#000006901|Q0FjypxZYnPmKIkAlcKS|0|10-948-853-2796|4378.33|nic ideas nag fluffily above +6902|Supplier#000006902|gbh8Rw2xmQFbnZGxpQU|17|27-906-885-9468|5549.17|ly express waters. unusual deposits cajole slyly ironic requests. careful packages cajole qu +6903|Supplier#000006903|UgUQMp80h0vqpULCwS0MJ,hMqFXNH3UYkSDZg6|24|34-349-298-9497|4895.58|quests. permanently ironic +6904|Supplier#000006904|EKpE28VGC6ShEWvrYdlDVcX4sm6utl9 ibJDf|17|27-999-609-8430|9093.47|ly express instructions wake special platelets. fluffily regular accounts cajole according to +6905|Supplier#000006905|VAktOD8JapqInpe5xjxDUSic2DnfLFOt4kyTWEw|12|22-648-880-9962|8973.86|ounts cajole among the final deposits. slyly +6906|Supplier#000006906|kE9uNsC,Rb6,bXmrH |1|11-460-499-9621|783.58| express deposits wake quickly around the furiously ironic packages. fluffily unusual pl +6907|Supplier#000006907|zeis6gjDRIeBkhAIK,SCDt55y0RAtAg05YY A|0|10-623-933-5398|595.61|ongside of the ironic foxes. final accounts eat. regular platelets sleep ironic packages. u +6908|Supplier#000006908| DJMzK8rdo1thhIU7oZ|22|32-720-524-3933|9403.62| even, final accounts cajole carefully: final foxe +6909|Supplier#000006909|f9A W4Urm31d RTFKjt1,V3|20|30-189-225-4097|7656.34|ffily after the thinly regular instructions. quickly final deposits against th +6910|Supplier#000006910|w4KossB5Mz99LQK|19|29-354-704-6035|211.66|ions. quickly silent requests cajole. busy theodolites cajole across the regular, final accounts. bl +6911|Supplier#000006911|hLcmxv1mKc|0|10-884-395-8359|2260.29| bold deposits. quickly regular pa +6912|Supplier#000006912|hff1tjJJ8LKrYA4Mp|24|34-793-902-9284|3656.74|boost blithely around the quickly bold hocke +6913|Supplier#000006913|ulndzeZ5Ga8xkIV2M6CjSANx26dLHSEhZB96O6H|0|10-692-656-9129|-146.63|ly final accounts sleep about the bravely bold theodolites. furi +6914|Supplier#000006914|h5jdLFf9ePs6JtJNWpQZ55|18|28-466-133-5235|8436.51| the dolphins haggle fluffily beyond the final deposits. theodolites wake blithely. furiously fina +6915|Supplier#000006915|rKeFMKV459Zrqek,sA0lU|4|14-224-353-9827|512.53|cuses haggle fluffily. special, express deposits detec +6916|Supplier#000006916|bu9b0Mng KRve7qcgbLvSU4r EuO 2ZyMafCz|24|34-710-909-1278|-967.88| pending, pending asymptotes nag bravely slyly unusual sentiments. fluffily regul +6917|Supplier#000006917|95Z4rKNRaRv05WsD1,vu|13|23-920-479-9564|-237.26|instructions. carefully regular accounts boost enticingly +6918|Supplier#000006918|kaEM1yahLjYbzGD2dlA4lefzCmSTNmAqaOdg06PR|17|27-656-821-3120|3284.74|gle carefully above the carefully +6919|Supplier#000006919|THb6susPFrL,uN9|6|16-231-445-9839|6598.86|final, unusual dependencies sleep along the even theodolites. pa +6920|Supplier#000006920| XaCixMyGHvET|23|33-882-957-4261|3085.14|uffily final foxes kindle furiously deposits. deposits cajole slyly even, final pack +6921|Supplier#000006921|EkJ1LWRGGkNzw|23|33-101-159-7400|6952.21|ilently final requests sleep closely ruthlessly ironic platelets. deposits sleep since the slyly +6922|Supplier#000006922|293uG72TawGSJ|0|10-773-914-3045|1900.53|y final asymptotes. quickly final theodolites integrate carefully af +6923|Supplier#000006923|BmLo4 nU11tFm0|11|21-561-469-4307|7220.16|carefully furiously pending foxes; ruthless requests cajole slyly after +6924|Supplier#000006924|maqzwBjfGNx1w0UMNIpN|23|33-294-250-9214|6306.88|quests about the even, reg +6925|Supplier#000006925|zDJtY9Vp1asvKVRsvDd|20|30-984-364-2432|1684.14|uests wake slyly regular pinto beans. ironic, brave deposits about the blithely final packages caj +6926|Supplier#000006926|axXBFVp upk,dFFqMH|7|17-781-309-7819|219.47|es use carefully regular packages. regular +6927|Supplier#000006927|C0SM0MXX AnJR1b3F4akAsVcxoUA86CjVnVWn|2|12-250-790-6744|1561.15|gular requests lose slyly among the pending, express accounts. pa +6928|Supplier#000006928|QTHoa0QWOsPaq a5L|13|23-928-964-1675|1218.61|. regular, final packages wake thinly final dependencies. final accounts a +6929|Supplier#000006929|h6q8ma1wYUJsfz5PTVIE,Ms3qf|23|33-318-546-3182|8063.57| furiously regular requests haggle furiously bold pinto beans. quickly silent requests haggle a +6930|Supplier#000006930|2QQb2qxS O4mzLmysOhDLL6tri2GqT7Nxr|13|23-696-752-1816|2488.31|rmanent, final requests haggle quickly final packages. slyly unusual requests after the furio +6931|Supplier#000006931|7webcEwedomtstU0Ktsz8Gu4|13|23-332-379-4657|8009.13|, ironic theodolites. qui +6932|Supplier#000006932|R4ikTRIOmmuFaC,wGmx6iCQa|0|10-158-496-9309|525.29|elets. carefully silent platelets haggle slyly. pending accounts wake bravely. special requ +6933|Supplier#000006933|0PeUrDuRwi4Yy2Mt|0|10-144-918-5458|4445.79|bold ideas. dependencies ma +6934|Supplier#000006934|wnar78tt,IXV|11|21-760-189-8193|6309.57|the carefully special orbits sleep blithely quick multipliers. furiously clos +6935|Supplier#000006935|gjSB5Ymu4x|10|20-529-389-8747|2951.67|ic packages according to the fina +6936|Supplier#000006936|XhPUrxphgHjY09AvhwqyUzchuy|8|18-366-578-7585|7629.18|s wake slyly. regular packages can sleep blithely. blithely unusual a +6937|Supplier#000006937|LT7ikhD4lzfaCNN3eL99k5SlwX2hPc M|16|26-683-895-2989|5610.20|ccording to the fluffily pending package +6938|Supplier#000006938|NkZxX3c1RYqz2NPR|5|15-776-230-3663|7574.04|riously about the carefully even packages. pending, bold deposit +6939|Supplier#000006939|C hSTG9rCq|20|30-535-353-4920|2978.35|along the slyly regular accounts. requests along the regular, +6940|Supplier#000006940|ieKVy0qc7Knj,aaBeOQb|24|34-654-743-2155|5167.01|y final deposits are quickly unusual packages. carefully final +6941|Supplier#000006941|829 63db0Yo5dZTejZtn5GAj0oGhaoUpkDN|18|28-113-649-4031|5940.25| of the slyly regular deposits. furiously regular deposits haggle-- depos +6942|Supplier#000006942|MSHsXiqBblQhXjp|2|12-666-478-1768|-245.10|nto beans. carefully even instru +6943|Supplier#000006943|O1KJE67Z,KykRf8mV72VTnDG35PhSR0S0CJlYFi7|16|26-566-449-6283|3554.03|uriously express asymptotes wake closely among the furi +6944|Supplier#000006944|LRbDy0Wl1Ct|7|17-550-483-9225|587.07|r packages. furiously regular packages boost carefully beneath +6945|Supplier#000006945|iri0I0IUBBVBnWMvHEPtuStMIU|15|25-239-550-2712|8763.81|carefully. ironic deposits nag after the idly bold accou +6946|Supplier#000006946|To6Slo0GJTqcIvD|3|13-685-283-4956|1459.23|ithely regular accounts. carefully final packages a +6947|Supplier#000006947|fqnwZNiDzAWepFK|16|26-526-555-2820|4228.94|iously close theodolites. slyly even theod +6948|Supplier#000006948|JmtU7EjQb2|9|19-166-824-6916|2831.61|round the furiously final instructions haggle pending excuses: blithely +6949|Supplier#000006949|mLxYUJhsGcLtKe ,GFirNu183AvT|3|13-776-634-2532|2792.37|boldly bold gifts are deposits. even accou +6950|Supplier#000006950|ZSVSSPBY5Z|13|23-569-624-6273|3372.46|lithely even pinto beans. slyly pending t +6951|Supplier#000006951|zMRAWjldjixfgiN,6qCnfPsUI1FoZ9uVYXts|21|31-367-166-1777|6870.16|e after the slyly ironic foxes. quickly even accounts sleep. blithely final dependencies against the +6952|Supplier#000006952|juuw6Dp yVcc55cwQ,la8QRJmVKQ0Gj|17|27-896-763-5133|8879.18|ithely. regular pinto beans sleep. even +6953|Supplier#000006953|OlF,o4P08RTJAdN|11|21-541-464-3162|4476.02|cing ideas. fluffily ironic platelets wake blith +6954|Supplier#000006954|apRGut95pdzOMmQcjlML5lx|23|33-140-200-5961|8484.65|out the blithely express foxes. carefully silent packages wake quickly slyly unusual dependenci +6955|Supplier#000006955|A4APIhGkOaWcFVGtI wU4aq8ppd|12|22-255-353-1311|6620.41|of the foxes use slyly blithely silent platelets. slyly bold instru +6956|Supplier#000006956|4OHI2IxjfK3eSgseE6lg|7|17-576-857-4485|2344.98|ic pinto beans. express, regular theodolites boost. quick, bold requests about the carefull +6957|Supplier#000006957|8lvRhU5xtXv|10|20-312-173-2216|7995.78|ly ironic accounts. stealthily regular foxes about the blithely ironic requests play blithely abo +6958|Supplier#000006958|0n9BD,gRzUc3B,PsFcxDBGp4BFf4P|10|20-185-413-5590|7186.63|against the instructions. requests are. speci +6959|Supplier#000006959|AWSMTKyZmMC6j290WFBVXMGh1U250bImRH1C32|9|19-428-904-1948|3779.07|ly final theodolites sleep. slyly unusual deposits are carefully regular +6960|Supplier#000006960|29pZoSofXDVw6IzjAF9pTgCd|14|24-894-676-4904|9775.72|s. pending, fluffy pinto beans sleep carefully bold accounts-- slyly ironi +6961|Supplier#000006961|ctlQlwwwlE15RbfayrBKteB|3|13-850-330-1316|2591.15|quests sleep furiously bold, final pinto beans? blithely ironic deposits +6962|Supplier#000006962|rYBTZbWb,3Bauk6x7HYEVYDc7Plnq8FovC3qZ,yS|2|12-147-994-9913|9463.06|lphins. carefully unusual ideas detect dogg +6963|Supplier#000006963|pTYkcW,v6m09aY4TA,Mon1LNLQvPwvmhfVzNJi0|6|16-122-543-3501|5772.06|arefully express packages. regular +6964|Supplier#000006964|6QkT8h6,A,yp6|1|11-529-924-4414|54.35| accounts. instructions engage slyly against the even fox +6965|Supplier#000006965|VbyGXfyyucsCa,COTUXXQiN5uKmIq1ggGXc1M|18|28-297-592-5284|9456.26|slyly silent requests! furiously regular packag +6966|Supplier#000006966|Eqy rgV0ao1J,P|4|14-601-835-4463|1513.77|furiously pending accounts. permanent deposits kindle sometimes +6967|Supplier#000006967|S4i1HfrSM4m3di3R9Cxxp59M1|20|30-193-457-6365|7812.27|ely. dependencies cajole quickly. final warhorses across the furiously ironic foxes integr +6968|Supplier#000006968|MYqBZH8gOrzgoiL6SwwFA|5|15-584-254-1458|2794.77|ges doubt slyly quickly express epitaphs. carefully regular platelets hag +6969|Supplier#000006969|gHahGeOEeEQN837a4oV85IE5qi|18|28-514-538-7460|-266.45|pecial packages wake slyly carefully i +6970|Supplier#000006970|kjSuwyDKc9j8WLM5lYSBdtCaMcvz,|1|11-276-801-2193|7662.04|le requests nag. ironic, unusual deposits cajole carefully. final, ironic deposi +6971|Supplier#000006971|S2pAN0uYjU,0fODwI8HBiSxzglO|12|22-523-651-4190|7884.48|yly express hockey players. accounts after the special, final deposits integrate above +6972|Supplier#000006972|t Cj14WjX65vSj8OC0g|18|28-271-809-1877|6559.96|es about the sometimes pending ideas sleep above the bl +6973|Supplier#000006973|bbFI923ouQl|19|29-495-862-4590|9225.09|oggedly silent asymptotes among t +6974|Supplier#000006974|dFI1WJMAYZV zbw4VKGVyATMRPL73B9KwGYKv|2|12-709-586-8799|-842.30|ets. furiously bold deposits sleep sly deposits. final foxes above the regular pa +6975|Supplier#000006975|GSjo3R0,9AyFO2xe7AdDsU7d6cJTkwm|6|16-926-547-9831|9441.64|posits. final instructions cajole daringly +6976|Supplier#000006976|ld8Ibdnw6TJ6I|23|33-705-178-5183|8788.01|accounts after the slyly bold accounts integrat +6977|Supplier#000006977|Y0A2ta8q6gykgS2,GWuXcBrcvIi6z|19|29-554-293-8901|1226.51|ss dugouts boost carefully slyly regular +6978|Supplier#000006978|FS17zlWYWaXqA3aUj3oOR3sdMhTN3|2|12-218-384-6106|8863.78| the even asymptotes cajole quickly abou +6979|Supplier#000006979|eHXl70pXPk8O5vrZJOG1XrMunsmu8iZPk|5|15-793-442-6510|9533.47|oss the blithely even packages breach after the furiously regular requests. furiousl +6980|Supplier#000006980|RsnAFGUkMVbYNCK7|9|19-410-393-1033|1064.56|eas should have to sleep. frets c +6981|Supplier#000006981|M5kfDPiO dfJfqWKrRmEUxFLK8rCLkSQ01Guj|0|10-697-192-8784|3934.75|r, final ideas boost quickly regular theodolites. quickly express theodolite +6982|Supplier#000006982|JIkHvt5KOLdwZhyYTUP|19|29-551-357-9007|8358.66|are final, even packages. regular foxe +6983|Supplier#000006983|Zlxf ygdYCpvK V7jsO|14|24-726-665-9001|1127.82|sts are against the furiously even foxes. fluffily bold dependen +6984|Supplier#000006984|BhNtundEYvjh|24|34-886-920-4044|254.11| of the slyly ironic deposits wake carefully slyly final excuses. car +6985|Supplier#000006985|PrUUiboQpy,OtgJ01Z4BxJQUyrw9c3I|3|13-598-379-2237|841.01|s are carefully. quickly pending p +6986|Supplier#000006986|aEjjqMOrb4QaOOmOfv2YIgG|13|23-248-840-7380|8851.24|s detect foxes. carefully f +6987|Supplier#000006987|UH1zBxTNjTminnmHRe h YUT1eR|11|21-963-444-7397|7844.31|nag quickly carefully regular requests. ironic theo +6988|Supplier#000006988|bSQuWswGiw1oiv9NZuiSjCR5nFq892Hce1|4|14-474-999-8694|2430.33| ideas. unusual requests affix blithely bold +6989|Supplier#000006989|KVh8JH kp01pF|16|26-936-359-8088|1551.42|r hockey players across the slyly express ideas integr +6990|Supplier#000006990|gTdOppzI1p,B|15|25-101-737-4124|2715.58|carefully even foxes. final, bold t +6991|Supplier#000006991|R02DHSUDD4KwLdc6EwZLR77JKdh|13|23-388-666-9112|-198.32|ke regular deposits. final theodolites above +6992|Supplier#000006992|Wrosic29lr6isSctUf OShRaPgFbPcLB|18|28-147-515-6998|8776.34|sual foxes according to the furiously final braids kindle quickly carefully +6993|Supplier#000006993|PETkj3WFEhsJP6m8Ad0qD6dJIA94O0UhI6y7|0|10-643-746-9191|2966.73|he carefully ironic packages sleep express, pending dependencies. furiously ironic reque +6994|Supplier#000006994|XAX2GA4eRiWOiEs8,Yawi|19|29-224-770-4470|7871.10|s. furiously unusual requests +6995|Supplier#000006995|TIUy84oVy5e7|23|33-202-318-4971|1615.41|special, final deposits are furi +6996|Supplier#000006996|a7UQKYM5EMrE|20|30-313-379-9662|165.90|packages. blithely regular requests +6997|Supplier#000006997|q4RXcPrWqZaJ2JhRxzciXdb|16|26-273-500-5154|3171.02|ironic packages sleep. courts haggle slowly. carefully busy pinto beans accord +6998|Supplier#000006998|r2i3HfkSQh9dvho, NpoabdMsPBG|21|31-950-540-3460|9305.22|bove the fluffily ironic deposits. regular +6999|Supplier#000006999|B5AxXKOtimMyUhbI5nu9 Xo14Um6Z|20|30-784-764-5379|3847.92|the regular, even instructions. unusual dependenc +7000|Supplier#000007000|Shh5qKPCXtvp6ZqDcb2F1Htax3TeV2|19|29-885-256-9772|1006.09|lly furiously ruthless accounts. sauternes snooze. furiously final tithes cajole ca +7001|Supplier#000007001|Fflu,eq0tREWymWt2sKtp7|17|27-489-290-3215|243.99|s are depths. multipliers wake about the carefully r +7002|Supplier#000007002|x tsVJawp8Z3texmcVMq3pZDD|20|30-191-749-1323|6662.74| beans according to the final, ironic deposits wake slyly around the fluffily even requests. +7003|Supplier#000007003|ratdvUMoA6ERhe4neyN15S|9|19-273-946-6063|9642.15|ly special grouches about the slyly +7004|Supplier#000007004|2Bt86wwHjo5y9dsk2h211hP7VgeuJqi,J2yx|16|26-178-705-1024|1752.16|ole after the slyly regular packages. ironic deposits use slyly fluffily bold grouches. fu +7005|Supplier#000007005|fwpTcZmsMJ9dheVlCf|9|19-728-987-1492|-565.29| sleep quickly foxes. fluffily unusual sauternes +7006|Supplier#000007006|szR,rd4jftCF|22|32-587-422-7789|1130.64|are blithely final ideas. slyly even deposits among the special, +7007|Supplier#000007007|uB0ugMJPKT,R 0nj6y|12|22-255-627-3014|3653.92|ng the blithe accounts. furiously regular instructions h +7008|Supplier#000007008|BXISIGR033SlhlceM,XAOqjeQGIvd,iNg|11|21-914-999-5215|7458.02|ly regular foxes cajole carefully alongside of the even deposits. daringly br +7009|Supplier#000007009|Dm1A9KNq6py1ZA wf33KrRo8aXYUQwEnS9HiCwGi|14|24-360-133-3119|59.12|otes sleep after the slyly regular instructions. carefully unusual instructions haggle quickly a +7010|Supplier#000007010|eNUebieMDRu2rmc1C|22|32-860-684-8353|1510.90| against the carefully final theod +7011|Supplier#000007011|9wtidtOx1Gmmcku0zXR|16|26-417-784-5046|3713.41|uctions. express requests sleep. carefully even pinto beans cajole above the quickly ironic accoun +7012|Supplier#000007012|Tnn5YREXXb8Ues|23|33-534-876-7469|6222.52| the silent, even ideas. sl +7013|Supplier#000007013|P4rea429Dhk9Wr4HA691S4hxY8,gbGazLpr1cvs|14|24-388-558-3561|2858.30|y unusual realms around the special, ironic requests cajole blithely slyly unusual requests +7014|Supplier#000007014|QRe,woltmYU8mMk3oSH|19|29-978-161-7278|365.67|s deposits wake final requests. final, quick patterns snooze slyly furious +7015|Supplier#000007015|GTI62eAWWMHyFrMDKT0om|5|15-166-122-7339|5831.76| unusual packages sleep above the furiously ruthless ideas. waters are carefully. dogged pinto b +7016|Supplier#000007016|IZSABjCNf EQHPJLZUFhdgcwq0UMMW3zI d5Z|2|12-964-486-3174|2454.34|ect fluffily. carefully final accounts play fluffily. final foxes boos +7017|Supplier#000007017|a5W2DYj6EnoAwTEgk|14|24-509-475-2517|9057.26|ts are closely blithely unusua +7018|Supplier#000007018|mj5AQ2xqjt3dnol6PvQLgMBQLmZcod r6SeQyHU|6|16-651-560-8892|5397.19|mptotes among the excuses are about the slowly pending requests. final excuses hagg +7019|Supplier#000007019|2GQsALzRiTt2BQum6bocdeGawkOrsjNIZ|21|31-277-340-1713|2631.26|ts. ironic requests around the accounts play quickly express foxes. furiously +7020|Supplier#000007020|MWRYVzR8AasLDSuO15L TXnTdkKMpfc6XkVoFVMg|13|23-899-965-3786|1530.95| tithes cajole. furiously ironic packages haggle slyly. ironic, p +7021|Supplier#000007021|Zp,ttXxWQJoLc ke|20|30-382-376-7650|-365.21|e furiously deposits. blithely ironic accounts wake finally. furiously regular platelets sleep +7022|Supplier#000007022|leVNNqe6onwVKaUvrGHQMkZ0DnrXJ|17|27-517-322-9003|8811.64|es. excuses boost. bold foxes haggle carefully across the regular requests. +7023|Supplier#000007023|ZWkkeSLe6XeLZe8QR 1mTZRQSp|5|15-319-886-8295|2243.40|refully even accounts sleep along the bold, regular instructions. +7024|Supplier#000007024|hJD8qoxaZhd71Y|9|19-661-850-9608|5723.71|silently bold foxes cajole furiously ironic platelets. slyly even deco +7025|Supplier#000007025|MeI6qPkYfn6gzM|19|29-164-321-1034|1411.95|ts. quickly regular accounts after the furiously even deposits sleep furiously regular deposits. i +7026|Supplier#000007026|snAGzZ,Hxa55hLoy|10|20-828-194-8013|1573.68|le blithely slyly ironic dependencies. requests haggle. ironic dugouts cajo +7027|Supplier#000007027|63OZHEQLfvOuly sC7SPVf3MRqQ5cb|6|16-553-679-7517|2827.52|ions wake carefully fluffily even ideas. furiously express dinos haggle. final packages affix qu +7028|Supplier#000007028|L1cJBpLuoGXaQPE5AaSLHxcWBoRxm3dscjh8V|7|17-894-762-4422|172.40|aggle. blithely unusual platelets alongside of the final escapa +7029|Supplier#000007029|4wLP6R1LH8anLTvVYd MKHWfuqAv6FDul|10|20-559-853-5034|2405.88|jole blithely regular, even deposits. carefully special instructions above the unusual, final ac +7030|Supplier#000007030|ipRbESjBUB2tDWv8O|16|26-706-211-1738|8571.06|quickly ironic asymptotes cajole. furiously ironic packages haggle amon +7031|Supplier#000007031|zTr3Hq6ufV2a EPXW|1|11-494-218-8250|6921.71| ironic foxes are carefully. furiously regular pa +7032|Supplier#000007032|yqobYXgvTSC|13|23-264-756-2278|6153.97|egrate furiously slyly express ideas. quickly unusual accounts s +7033|Supplier#000007033|ZgjvkxLbXCLTLOIXv6GP2GFSyar7C8u|15|25-242-847-3215|5882.44|ly special packages! quic +7034|Supplier#000007034|PbTEBHgAx,Rh10cfhOfpmCKw4451TN|4|14-833-593-6359|7210.73|heodolites sleep across the fluffily ironic dependenci +7035|Supplier#000007035|,pChjEjn3Slu tkv3nlxG,g|1|11-243-514-9527|1711.34| the final foxes. slyly final requests affix carefully blithely ironic exc +7036|Supplier#000007036|scrn1 ctBJgudYPOx0tRc1EQWNIRB3A|5|15-399-535-2559|6403.06|onic accounts wake alongside of the finally final requests. pending, final dinos are fur +7037|Supplier#000007037|IA,rgzC2oCxysnqCSD73 yvYa8KGAq8hz0|8|18-449-464-3907|1291.15| nag fluffily express accounts. carefully regular accoun +7038|Supplier#000007038|Z5UML Yd8ZOMvawM6dv rQFFZr,Lm|6|16-978-492-1589|6332.07|usual asymptotes. regular deposits haggle always quickly special packages. furiously bold pack +7039|Supplier#000007039|W36dcpdHN2GyJSd4wfv|10|20-152-963-1884|7064.04| quickly regular accounts above the final, +7040|Supplier#000007040|8yV6xwzmQuu 3Rc2mqz7rOdr AW|16|26-499-346-4011|277.23|even requests detect after the furiously even deposits. packages use slyly. pending patterns nag +7041|Supplier#000007041|JvviiY7,cRNc,kcaMMwrfJyIXqurU|1|11-194-454-8672|4388.96| carefully careful packages. carefully ironic requests poach slyly. ironic r +7042|Supplier#000007042|8vYguGDS2TAwnwUf7HYyhQsbReKC0|6|16-338-868-6616|8276.01|quests. carefully final packages after the furiously pe +7043|Supplier#000007043|FVajceZInZdbJE6Z9XsRUxrUEpiwHDrOXi,1Rz|23|33-784-177-8208|8936.82|efully regular courts. furiousl +7044|Supplier#000007044|O2aV wCzz dcp|5|15-166-556-6472|5255.75|sts should have to use silently alongside of the quickly bold requests. blithely regular +7045|Supplier#000007045|NYbUSz1bQaYt,xhyrfyQgD62OaCOqIG9IC|21|31-864-135-7407|1418.29|ously even requests run always regular ideas. slow, final foxes play slyly silent, fi +7046|Supplier#000007046|qHGfPK7qr2l9zDGJXz|5|15-248-134-5861|9136.90|c courts nag carefully pending accounts. fluffily expres +7047|Supplier#000007047|4UDtZ,2vtplv74entNaraPT8Ja660w9CS|17|27-515-593-7697|1959.40| the express packages. regular accounts boost blithely accordi +7048|Supplier#000007048|1MdbjFx1cBm F vA7GBlAJh5vJKrNyOpdkFB|10|20-819-904-8165|6546.58|bold requests. requests boost +7049|Supplier#000007049|sVoraaxoGuK4FOiWQ,I|6|16-565-172-5107|4368.79| affix quickly. ironic packages haggle regular accounts. instructions sle +7050|Supplier#000007050|qGHyUjDT27KwsA6Swmyj0Ib6ynqjG54jGRJZoo |7|17-704-896-3446|-812.94| the epitaphs. deposits wa +7051|Supplier#000007051|i3DDzvJNI1P|12|22-921-202-4919|7904.97|s boost. carefully final pinto beans sleep ironic +7052|Supplier#000007052|EvvMY80H5gifhB5a82U4Q0FtttnZ3npEYJ|11|21-562-180-1409|9931.82|he furiously even theodolites. blithely ironic theodo +7053|Supplier#000007053|HztwJXj6Nej9qn62zWiISWGj6SJLRr4wuS|8|18-305-502-6678|8237.01|packages: regular, express packages beneath the regular accounts use above the slyly final pint +7054|Supplier#000007054|fJhXBKM3iv|20|30-374-655-3634|2408.19|l foxes need to unwind. furiously close courts doubt b +7055|Supplier#000007055|GEdTmR6U846kNcWPEVLqjaUVzZ|15|25-777-984-6768|3341.73|counts wake. quiet pearls cajole final frays. slyly even excuses sleep blithely +7056|Supplier#000007056|M5cAJQvW9D5zwC7o2qkoe|11|21-175-383-4727|7393.50|slyly even requests. forges haggle boldly express requests. furio +7057|Supplier#000007057|y7FptKFu3YCmDFsZf6b,6xbmWP|0|10-162-742-8020|4579.33|reful theodolites. requests cajole. blithely r +7058|Supplier#000007058|WjxB7gSPQkeNnd3|16|26-519-546-9713|6838.47| cajole carefully about the special, regular packages. +7059|Supplier#000007059|W9K7t6qjLqghDT2o|22|32-898-594-1732|4189.78|ent theodolites use slyly silent requests. blithely ironic asymptotes sleep slyly along the carefull +7060|Supplier#000007060|qE,c71OeuAclR|16|26-265-429-8153|4676.53| cajole slyly quiet pinto beans! unusual asymptot +7061|Supplier#000007061|lv7XVrUOi1PwFVNzpcNb|20|30-616-603-7046|1375.10|sublate thinly about the quickly even pin +7062|Supplier#000007062|DGBGsC97WZ uHillDcuwPuK6qYr3vw|8|18-314-204-6543|9268.80|uses are. ironic tithes among the regular foxes boost slyly express excuses; quickly dari +7063|Supplier#000007063|,3M1JeIaWh1pCT8f|12|22-236-733-5397|7832.12|se furiously pending platelets. carefully unusual Tiresias should +7064|Supplier#000007064|ep lHd3q8kDIWLG R7h,9z7AEcvqxi1GNN49|8|18-543-384-6456|6466.59|ts. carefully final ideas haggle along the permanent, pending accounts. quickly fin +7065|Supplier#000007065|JgQakP2RDDtNp8ZRpgHgSiM7mjgcyikga0hug|15|25-436-118-7920|2043.80|n furiously? furiously final theodolites print slyly. quickly final +7066|Supplier#000007066|a rdeYnN1ELxIjkkCRo4UVbJqh2lXmQB2TXBuwPw|6|16-880-494-3956|6183.73|mold furiously. regular packages are furiously. asymptotes cajole quickly after t +7067|Supplier#000007067|ceXHONkKVuCiUxjfx2cF6VyE,E|23|33-413-651-8610|-44.14|regular notornis. regular depos +7068|Supplier#000007068|vgG0WFNQvJHf9CG,1R|12|22-670-400-4945|3986.56|ccounts except the special theodolites boost instructions. special reque +7069|Supplier#000007069|II2PgkwHPix7ojcLYpyPsI gSQZlox|4|14-499-950-9694|-823.48|ring theodolites. carefully bold requests haggle. reg +7070|Supplier#000007070|p,,oEzctc j2973nbRl8Qiv6xVs84K|8|18-915-875-2146|6721.04| special, pending foxes after the pending ideas haggle final ideas +7071|Supplier#000007071|e2CiSnlQ8TUVWIn6X iwCawyJEw,HB6Wj|15|25-903-628-2776|-486.78|sts. furiously regular instructions sleep a +7072|Supplier#000007072|2tRyX9M1a 4Rcm57s779F1ANG9jlpK|3|13-507-453-9808|619.16|ep quickly permanent dependencies. pending, special forges after the regular +7073|Supplier#000007073|6PzKtod4BPf2krTkukFc EwBYJ 08cC1EebwXzSM|12|22-560-424-4848|9168.07|y even accounts integrate busily regular requests. fluffily slow accounts cajole care +7074|Supplier#000007074|kTHqfxDR,H8b9 Pra8A,jPB2Kv3|2|12-295-238-1986|8787.11| pending hockey players cajole blithely. slyly even requests across the carefully ironi +7075|Supplier#000007075|52hMohrQKM|11|21-580-554-9032|8081.62|ecial packages. pending accounts grow carefully; +7076|Supplier#000007076|DjLJ,yEKffpT,EKIbM6ZziTtzvCbE|4|14-929-221-6504|8906.10|nusual requests solve furiously across the carefully regular excuses. requests are quickly c +7077|Supplier#000007077|kQ6qLdSpTnD |11|21-279-573-5595|5874.04|o cajole slyly. regular accounts haggle blithely alongside of the even, regular dolphi +7078|Supplier#000007078|KbUybJPPmTnc0aNkcGTwnoIm3C3mC0|12|22-831-188-8290|9066.41|ajole quietly pending requests. boldly final foxes boost fluffily. careful +7079|Supplier#000007079|,LIgpCtnyPglw,HMQG3pG3M|16|26-938-943-9263|5258.73|arefully packages. slyly special dependencies use slyly alongside of the c +7080|Supplier#000007080|R1vXmHSrTAXaVu7kraZ5|6|16-957-468-4227|7466.57|furiously quick platelets. carefully regular theodolites sleep a +7081|Supplier#000007081|nqyAPCRSKI1jzPRN|13|23-965-234-1242|667.58|refully regular accounts hang carefully instead of the regular accou +7082|Supplier#000007082|9lwYCbYeQ8yywFjuL870VfHcipF3T2|18|28-577-801-5928|9030.71|. silent, regular sauternes hang furiously across the furiously iron +7083|Supplier#000007083|mzBkq7paoo|18|28-987-537-3349|-747.59|l instructions. even, ironic foxes cajo +7084|Supplier#000007084|XbDmiCojjPMhz7ULce1J4EOS22syqdsEI9P|21|31-129-979-8004|6216.07|gle blithely blithely even accounts. always final accounts above the carefully regular +7085|Supplier#000007085|eIgturKfems2G7|10|20-901-744-6553|4522.65|express, final pinto beans ca +7086|Supplier#000007086|N 9vyD6nbyUM9wlz|2|12-243-123-2380|134.00|uctions. quickly ironic somas breach after the ev +7087|Supplier#000007087|7c5GtRWIvs5pYndU41WeLHCnHG5mV|15|25-217-736-8507|6031.47|e daringly. quickly special accounts haggle carefully accounts. slyly +7088|Supplier#000007088|ngUFUKSPxKzNl3sAWbfMhja6bQD7WZOLLinhar3|17|27-472-515-5436|3913.35|counts breach slyly above the final dependencies. pending requests boost blithely according +7089|Supplier#000007089|8hF9PhLPVk,CIOzgksixLR8a|12|22-324-270-4650|710.55|ajole furiously pending deposits. q +7090|Supplier#000007090|ja16GplPqUs7rAMcEWOq12XOW2cQp4FzBurosq|6|16-663-439-9561|5164.92|e furiously. even, bold accounts a +7091|Supplier#000007091|dpr XUtSl2fA8pXLZ|15|25-992-449-4349|3753.26|s across the furiously silent accounts +7092|Supplier#000007092|,Gjn1UH1OfR9N3xs4RCDOx4DSV67YWL|22|32-400-807-1421|5041.68|ts. pinto beans along the furiously final instructions doze iro +7093|Supplier#000007093|ZkZJTAjNMSo5MhOy|6|16-131-789-8723|538.90| decoys. accounts haggle blithely. quick, regular packages are. car +7094|Supplier#000007094|AB3Ilu 4dGDn4GQULl81L7obQ3TqGZ8pPu|6|16-777-218-3967|2150.49|p slyly after the ideas. +7095|Supplier#000007095|gjDaygektlbMBE508U3M3mxeQKn mTF iEJ|22|32-670-893-5903|6304.47|uests sleep regular multipliers. enticingly final courts detect slyly. blit +7096|Supplier#000007096|ZRjdMsQ7qua7x,t2argj|7|17-682-424-6014|5248.58|iously final instructions are about the quickly si +7097|Supplier#000007097|jnG c5lMSOSSjWV72NvIkm7cQJ4|14|24-468-902-8109|1845.54|furiously up the carefull +7098|Supplier#000007098|G3j8g0KC4OcbAu2OVoPHrXQWMCUdjq8wgCHOExu|3|13-249-810-6460|6971.20|. requests sleep slyly blithely +7099|Supplier#000007099|QNPdhlsez5AjIAAozKshQgKbj|10|20-165-629-2474|2078.66|nag carefully final pinto beans. blithely silent pinto beans nag closely bold, final foxes. sp +7100|Supplier#000007100|uepAB9PA3BhNu,THXvIB|8|18-222-364-6257|9042.87|ccounts nag. slyly regular ideas use. furiously regular requests print. unusual, stealthy theodo +7101|Supplier#000007101|sDtYLeNALsXJzis02TAdBBjtlAFT63DtOa|2|12-489-953-8113|5554.37|ter the carefully bold theodolites. even accoun +7102|Supplier#000007102|ekYKpMm5f1Opp7wnr2wRyxutnzkRV|6|16-609-979-4437|4282.94|ut the stealthy, bold pinto beans. fluffily even excuses +7103|Supplier#000007103|YgK9wyZ9rtOb3p6B4bjsC|16|26-962-165-8472|8335.27|ackages. slyly regular deposits kindle carefully. sly +7104|Supplier#000007104|w63Qzq0t10|17|27-114-456-1570|3900.61|nts nag after the furious +7105|Supplier#000007105|54dVM9wEXGnNRR3|10|20-573-651-4616|6621.95|ully whithout the ironic packages. quickly even pinto beans nag slyly special packages. slyly pe +7106|Supplier#000007106|8qPuqCqQkjPorn4m5fp N|9|19-183-837-3833|6171.65|y pending accounts wake carefully above the furious +7107|Supplier#000007107|766mJ3PIqHFnZPrpcQBP u6r1 ,12P6Go|5|15-338-555-1847|5909.00|ular theodolites haggle express +7108|Supplier#000007108|wo2cXHOK5YzuHdLEl7rs0U2 kUl5O5lP|14|24-918-209-2604|2503.25|ts. regular requests for the quickly pendi +7109|Supplier#000007109|ZnKo5yyY0 klRwTy ksV1e3uz|23|33-192-547-6913|6033.81|? carefully final excuses are carefully. deposits hag +7110|Supplier#000007110|Xjim0GGIysuwBawGxl607wWZ,XaGWBRr261zYp|22|32-115-349-8168|-782.57|c deposits boost slyly blithely even platelets. finally ironic packages are. slyly even gifts nag p +7111|Supplier#000007111|C35CxrTQwtIONArKz|12|22-545-891-8947|1271.83| requests. special requests are furiously around the final foxes. ide +7112|Supplier#000007112|qYYofPf5SA4GV2 6J1ZU,pIQfB|11|21-936-591-3443|626.62| slowly according to the blithely regular dependencies. +7113|Supplier#000007113| a25fAfyWNsvBtLUe7CIIMksV2l8k 848nyMgL6C|15|25-731-762-6294|2467.68|as. express packages use fluffily according to the theodolites. ironi +7114|Supplier#000007114|s9s4YLeLWo7fLRO3rdQKFfUnZhrZUPjOC|21|31-808-386-3612|2304.57|ffily ironic packages. furiously unusual deposits sleep fluffily accor +7115|Supplier#000007115|FuocRvjqLIb9Lb4HEPfFUM|22|32-536-466-3613|-113.61|eposits boost quickly carefully permanent pinto beans. furiously ironic deposits are +7116|Supplier#000007116|dPQ y 5vLTNuG|20|30-545-741-9241|2495.39|gular theodolites. special deposits haggle. requests sleep furiousl +7117|Supplier#000007117|hzd0TYPCB,nZ2ASLT5LxU3SpsJ tablXhsDuZTGq|23|33-117-972-7830|5985.77|lent instructions. blithely unusual excuses nag blithely across the fluffily unusual deposits. +7118|Supplier#000007118|8TSZe1lL3xQRr8xZoqbJeti,R3|1|11-780-159-8657|5874.47| regular instructions. slowly express packages haggle quickly close excuses. car +7119|Supplier#000007119|XhWMs0ow6WYtZvE1gGdc|12|22-705-492-4532|5177.11| final dolphins believe. blithely express +7120|Supplier#000007120|QLaIDxZrAI2zWl2|23|33-964-286-6586|-294.62| deposits. carefully regular deposits ar +7121|Supplier#000007121|TWJo7,VGdLIic9qMp2WzYlUO peOnTBS7dN Xld|6|16-881-623-7753|7215.68| accounts. blithely express foxes sublate iro +7122|Supplier#000007122|xMKInuZNsJ,gLecTyT ROceYfx|3|13-103-963-7387|7996.84| bold packages integrate carefully. final dependencies lose. +7123|Supplier#000007123|ThPWnDOZtty6BoHvyXYJFk1AdRRj|8|18-917-122-9478|2900.87| accounts are. carefully ironic excuses +7124|Supplier#000007124|TX,pTg39EO62N|12|22-585-333-6019|934.90| dependencies. carefully ironic deposits are despite the blithely fi +7125|Supplier#000007125|xTeBXQnpesVl,D8N0gi4FcUFkIgEicmKvhP3v7|19|29-478-622-5217|2884.40|press ideas! blithely unusual packages across the blith +7126|Supplier#000007126|ic1ixYb,R2K2qb|1|11-745-331-7130|-865.65|y ironic packages. fluffily +7127|Supplier#000007127|3LlA2QAVNr2SyTnDoua2eJgMc8Nlus0atFz |20|30-922-704-3614|1959.30|oxes was. furiously fluffy deposits are blithely about the slyly special +7128|Supplier#000007128|WM478,6eTURY|10|20-994-910-9631|4870.23|blithely. furiously final theodolites boost silent foxes. car +7129|Supplier#000007129|8oILFkMpst0M|9|19-204-131-2284|1026.87|inst the blithely final requ +7130|Supplier#000007130|MC0t1fM6eEqGFjS1CH|24|34-695-712-1177|8571.88|lithely carefully pending requests. carefully special packages nag fl +7131|Supplier#000007131|idB,ZCHCaKXfMZdEGcSu0onzMnavfZrqp0lYcB|20|30-149-781-3330|9715.91|bold accounts according to the furiously pending theodolites x-ray furiously +7132|Supplier#000007132|xonvn0KAQIL3p8kYk HC1FSSDSUSTC|3|13-378-981-2173|9451.71|ffily stealthy foxes are slyly across the even, regular ideas. ironic dependencies according to the +7133|Supplier#000007133|WT4lazsXR mLEWohhl|7|17-244-746-2199|1930.97|. pinto beans unwind boldly. slow instructions kindle blithel +7134|Supplier#000007134|qeRvNaN0p9MHKs7z0hNd1Yygb7y1|17|27-827-610-3312|1827.75|s, even pinto beans after the regular deposits nag carefully blit +7135|Supplier#000007135|ls DoKV7V5ulfQy9V|3|13-367-994-6705|4950.29|instructions integrate fluffily pending, pending instructions. slyly special asymptotes +7136|Supplier#000007136|EHhfZo,xLtoM8JOsXt1p7dEugu4Jmvkvms7a9SlB|17|27-296-685-7917|4537.74|en, unusual requests. special gifts use carefull +7137|Supplier#000007137|mxQHehO5rwAP|19|29-749-311-2489|838.57|lithely final packages. silen +7138|Supplier#000007138|yTbjyvDClsKrSqMsNhj6ptg LiIju1|10|20-465-208-3863|761.49| about the quickly regular requests. carefully ironic packages kindle careful +7139|Supplier#000007139|YJoCSG0gg8R|0|10-659-563-8284|3358.22|y ironic requests. ironic reque +7140|Supplier#000007140|SBE7HKcr,c0yG,zgMF0DfHDv7ezRR4dToKwvqcJ|1|11-951-404-8797|9547.11|ly special pinto beans. f +7141|Supplier#000007141|ISEr,Y5xgSg4yyicA6wFV2d|17|27-143-796-3790|-694.40|sits haggle express foxes. slyly regular requests cajole furiou +7142|Supplier#000007142|bU733FVPdESXm3JpC50PDYF23kmrwNx20JxHBh|20|30-817-222-1916|9214.27|. bold instructions affix slyly among the carefully unusual deposits. requests use quickly +7143|Supplier#000007143|NFBf1JKFODt|24|34-700-669-9996|4288.31|symptotes sleep fluffily after the quickly spec +7144|Supplier#000007144|oZkgMoN7PRNEYDM1HBmXQKJ9kmIgj76TM8jkgZxB|21|31-693-600-3326|-362.28|l packages: final pinto beans serve? bold deposits acros +7145|Supplier#000007145|eG9ZPCfNuIuxusKl|22|32-331-188-7406|3153.90|counts play. fluffily pending accounts are. ruthless, +7146|Supplier#000007146|J8I3Pyfc2Usr2tJMJLDbsq7|3|13-146-899-2332|7618.66| among the regular accounts. slyly regular accounts haggle slyly. pinto beans cajole a +7147|Supplier#000007147|Xzb16kC63wmLVYexUEgB0hXFvHkjT5iPpq|3|13-635-690-5639|4883.31|y quickly ironic accounts. express, final ideas use carefully after the ironic +7148|Supplier#000007148|FDxytVBVJllKW|19|29-216-866-2808|830.73|lyly above the furiously busy foxes. express ideas use carefully after th +7149|Supplier#000007149|70ZpLZacEOs6kW5WG rrX79,AU 0QDd5nFT|16|26-921-629-2721|955.71|ously silent pinto beans. final, regular dependencies along the special ideas +7150|Supplier#000007150|FP7U7cbhQEybeMzPjQ2jatyYcp4|0|10-708-722-1522|635.96|courts haggle slyly. busily regular accounts sleep slyly express theodolites. ironic, pending r +7151|Supplier#000007151|3JH3tRaIkRLjTxSTgaayV kdqPIBmwMM7|17|27-188-234-9881|5459.54|en ideas wake quickly. quickly special deposits detect idly against t +7152|Supplier#000007152|jqPnZXHz4x71IWOxXYF3m2tM87Jg0Oki|11|21-419-297-2550|5440.54| are quickly permanent accounts. blithely regular requests wak +7153|Supplier#000007153|9BcR3N6LAdL91XPg2AiYdNMWpyyOexjcJ0SR|1|11-180-518-2987|5239.34|ts according to the accounts cajole fluffily against the courts. express ideas sleep quickly +7154|Supplier#000007154|VOqsdDsO4y,exyr|23|33-112-156-5571|643.24| packages are daringly. slyly ironic foxes detect. regular +7155|Supplier#000007155|aW7uJQnYNDIKWiu|12|22-878-535-8109|6871.25|ularly special pinto beans. requests boost carefully. blithely express pinto beans wake stealthily t +7156|Supplier#000007156|Pi5b77xdrKZfKVi|0|10-767-948-2849|5171.86|as. slyly unusual dependencies about the blithe instruct +7157|Supplier#000007157|N gqX7,FRjIHxiVqewfnfVAn|16|26-127-544-2483|-541.76|y express requests against the packages use furiously unusual dependencies. foxes wake along the acc +7158|Supplier#000007158|opQcsc pqJXzGKmD9Co31GehPzW1 orj|13|23-202-744-8221|627.10|usly final ideas use fluffily after th +7159|Supplier#000007159|9I3wWhd3Iv|6|16-432-124-4280|-463.78| carefully unusual instructions wake blithely ag +7160|Supplier#000007160|TqDGBULB3cTqIT6FKDvm9BS4e4v,zwYiQPb|3|13-691-385-9926|2134.95|s. blithely pending pinto beans on the carefully fi +7161|Supplier#000007161|5H9puOcj,unx4b18uig2d8zy3UhYRAPmym|2|12-970-314-8423|9148.46|refully slyly ironic deposits. furiously regular pa +7162|Supplier#000007162|FhVeZZDNEYZH MEcwstpgw7|21|31-755-574-2420|9179.69|lar deposits. blithely even Tiresias cajole quickly around the even, ironic instruc +7163|Supplier#000007163|9jzdDoHPLZ6gMt7GzSLqP Sdn10zYViXoNTT8XO|22|32-457-558-8569|6107.04|sts are. instructions sleep carefully across the ironic foxes. carefully qu +7164|Supplier#000007164|gBl2v8LJQuR0v8Ln4m84c|8|18-840-734-8244|9095.93|odolites across the final requests sleep fluffily regular ideas. express instructions +7165|Supplier#000007165|MxTb OhFEHp0CtL42lJKL3elKLcTuIhGrT7faYG|0|10-422-196-8965|6447.84| doggedly pending deposits haggle carefully by the blithely permanent deposits. fluffily ironic de +7166|Supplier#000007166|SLTRoeGAPQnhRh5gO43ibdGxApdfpJ8X4EAwv6g6|22|32-792-364-6658|5854.31|xpress packages. unusual foxes among the +7167|Supplier#000007167|O,uP7xO,SCDILkyczq4VaoeotMvBC|0|10-106-222-1832|7410.91|s sleep: furiously special d +7168|Supplier#000007168|Zbi1Gso2fuOruVZzDjeJSqX5KRZLY1mk3a3|8|18-771-260-7057|160.37|ajole quickly according to the slyly ironic theodolites. excuses wake blithely depos +7169|Supplier#000007169|tEc95D2moN9S84nd55O,dlnW|3|13-805-131-6933|9299.64| slyly against the pending platelets. final, final pa +7170|Supplier#000007170|9vABqu hZaciXSCQrbTj|21|31-132-516-6122|9088.35|es after the blithely bold acc +7171|Supplier#000007171|DXerxFIhNRpqF9dWNRw hDOlLX gEJFxh0|21|31-311-575-9302|-39.74|e even platelets? quietly silent ideas breach into the carefully +7172|Supplier#000007172|GBwCtGvazNkR0pLSqPITAiU|16|26-724-394-9371|975.12|the special ideas. requests wake blithely. sl +7173|Supplier#000007173|lyc6c1b76KDdr|5|15-876-287-1035|7804.25|vely regular deposits wake after the carefully even instructions. pi +7174|Supplier#000007174|4PP0sCXiIsntEaW7UsdEeGmubuVGkguV5oDQGX|24|34-494-520-4425|46.96|. slyly slow requests are around the furiously final accoun +7175|Supplier#000007175|6OGmajWN7b6eLeH7x4pmCsfMhDzufqCcqxmG0|17|27-187-885-5530|6115.19| fluffily final requests nag daringly slyly special depo +7176|Supplier#000007176|UlOr2sIS7jHPURS7ZVaOY5c4UE4I3WSwtM9ZosY|0|10-253-722-3725|2955.38|lar requests sleep furiously; furiously even dinos dazzle slyly carefully final deposits. furio +7177|Supplier#000007177|YnfYb6Bng,9Lk zSdG5olh1oVhmIQUrV|6|16-465-628-3315|4013.74|carefully after the carefully unu +7178|Supplier#000007178|K6fB8,JK8MB0pKKPpFg6XHsK1yYo|15|25-371-961-3709|4921.74|old deposits cajole carefully. carefully bold asym +7179|Supplier#000007179|l8eTxI1F6OGNy6p7CAq5NzYa9|9|19-737-245-9748|8756.45|arly silent packages nag across the silent requests. sauternes are furiously furiou +7180|Supplier#000007180|iN4nfTIHEPRQ2uNas9XKRxZisFyDyWtV 44Kq|17|27-918-420-2750|9661.37|final, final requests haggle quickly along the special packages. even dependencie +7181|Supplier#000007181|aIdiAjBcdilBpC78xuOWZKajbG|4|14-762-119-7999|5708.43| requests haggle carefully regular instructions. special instructions sleep furiously. careful +7182|Supplier#000007182|CL2rjkREMyf2Q txFszNy wXV3fndD2HKgdQlgrD|8|18-414-221-8067|8598.49|regular asymptotes. packages sleep quickly since the unusual, even grouches. boldl +7183|Supplier#000007183|6fW9mxnQy82BJHO,Y40C7TLo51ZsbFRbx AHr|3|13-960-355-3402|5825.00| quickly ironic accounts wake regular accounts. final fo +7184|Supplier#000007184|AKfsS0cnFq64dK|1|11-750-116-8427|9769.31|ily blithely regular packages. fluffi +7185|Supplier#000007185|3VUn39FRCVtmP2Tn9P1HLa3AgFu|16|26-715-768-3697|4826.62|haggle ironic packages. slyly final asymptotes wake against the furio +7186|Supplier#000007186|c5gPjnPZFZj1Dm9uPFCkipStYN0gZ|17|27-546-480-3566|5069.94|ffily even requests. regular requests haggle furiously quiet, spec +7187|Supplier#000007187|qVNA8Sc4tYjVcVr|15|25-337-390-3003|2514.55|lar ideas. foxes are. furiously eve +7188|Supplier#000007188|9V2cDltKfOErZ|6|16-681-517-1402|9048.09|es cajole furiously. special, +7189|Supplier#000007189|GXYyj1A6V3|19|29-203-826-7991|5265.94|ven foxes cajole slyly. blithely pending requests hagg +7190|Supplier#000007190|i9IcQxXtzr2LWc,Br2vt3epS|9|19-426-415-1610|7609.15|e regularly ironic warhorses. blithely careful requests slee +7191|Supplier#000007191|y62XYc3,bw33tqGBua2LqHCd8pIXNL3xO jH,|19|29-327-376-6773|9968.09|y bold accounts are quickly among the carefully final idea +7192|Supplier#000007192|VYQVSMUjNYlJB8CvmdM4oI9EmN|8|18-678-210-4987|1779.36|ending packages haggle according to the asymptotes. blithely iro +7193|Supplier#000007193|LOysL3v1UnfSXP,O3drFxmt2eCd2FCQa3|6|16-365-781-2094|5613.57|y regular requests cajole furiously. thinly silent pinto beans +7194|Supplier#000007194|zhRUQkBSrFYxIAXTfInj vyGRQjeK|19|29-318-454-2133|9280.27|o beans haggle after the furiously unusual deposits. carefully silent dolphins cajole carefully +7195|Supplier#000007195|8hiDLmHelzB|23|33-557-791-5721|2626.17|iously even dependencies along the bold excuses wake fluffily furiously fin +7196|Supplier#000007196|StcKAmtNNc3gPwIVtxU46MN3u2pU8RH4qnqCM30|0|10-205-425-3230|3975.16|es nag around the blithely even foxes. ironic, final dependencies wake furiously. carefully reg +7197|Supplier#000007197|ToRFrjcWLhrKm87NBYbznpivq|12|22-606-130-2366|5523.85|s are after the carefully br +7198|Supplier#000007198|Je9D8swRfBXw|12|22-906-540-4535|8420.42|ng to the idle packages wake slyly closely ironic ideas. bold accounts haggle caref +7199|Supplier#000007199|Ew4HU3ynk K5GKwyUIs7qorp6nqyA,bdT|4|14-809-367-4496|5002.14| final, bold instructions. slyly special packages above +7200|Supplier#000007200|bNY0Sik2xuZVoCQvqA1WpQb8tp|14|24-482-396-5609|7000.40|ckages cajole packages. silent +7201|Supplier#000007201|ieHlvZlpgUFWlK0xdxDgELwE0Jp1ql|16|26-821-153-3231|8830.15|ording to the slyly pending deposits haggle bli +7202|Supplier#000007202|4IO0WeGZ9h864FjV3zpbs8l6yrLnnHTmFvISy|14|24-744-766-5751|5343.09|requests. slyly stealthy requests sleep atop the final ideas. regular foxes play regularl +7203|Supplier#000007203|E3GTv728OrIJu2fG3SjkASzvGbDl4ITUCbeDSbse|22|32-985-936-5669|6320.73|ld requests nag quickly above the fluffily final packages. bold platelets boost carefully a +7204|Supplier#000007204|oYlakHfCcD,V7m8e4hVEW7Dn4kpPEou2yZdFIP|1|11-479-802-6397|3556.63|leep. ideas according to the carefully final instructions detect quic +7205|Supplier#000007205|OUp5iAull9qmfW1c31XGarGSt5pCH|4|14-170-569-1698|295.77| slyly silent ideas wake blithely in +7206|Supplier#000007206|3w fNCnrVmvJjE95sgWZzvW|22|32-432-452-7731|7871.50|ironic requests. furiously final theodolites cajole. final, express packages sleep. quickly reg +7207|Supplier#000007207|kIx1ELH,VfDe,I|23|33-667-355-7772|8307.01|ubt carefully after the blithely stealthy pinto beans. c +7208|Supplier#000007208|jW6h9jaXMGOmMI7CSnX|15|25-465-535-3420|2027.06|ly regular ideas. slyly busy requests sleep blithely across the fina +7209|Supplier#000007209| MbnHmU1lXT|8|18-850-494-6565|-233.66|slyly bold excuses nag slyly after the slyly regular instructions +7210|Supplier#000007210|G7MYkWkkJDVu,rr23aXjQCwNqZ2Vk6|13|23-560-295-1805|8837.21|en, express foxes use across the blithely bold +7211|Supplier#000007211|OahNKPQEVUeuN1qltqI|2|12-143-351-5463|3098.05| regular packages. regular pinto beans nag carefully. depths snooze +7212|Supplier#000007212|CL0Y2ehpi HVSEqUDhrFv |1|11-332-864-9431|3862.07|al instructions. quiet requests wa +7213|Supplier#000007213|2Nrby3JJHDJyWwVNiqPtm2U JGWlZpU|21|31-571-413-2799|7767.20|ly ironic packages about the regular, express theodolites hang slyly ironic +7214|Supplier#000007214|,p4JuPC07kEJlPXuabRktLBIu5|1|11-808-331-6755|4426.58|furiously regular excuses boost slyly. carefully busy packages nag after the fi +7215|Supplier#000007215|56XDMiFtYQo8aT|10|20-316-167-9209|2503.52|busily furious deposits along the bold +7216|Supplier#000007216|7rOpTpThf08CGwHBVHsVu2dn76w1xuF,fhN|16|26-654-982-9982|-344.27|ideas haggle according to th +7217|Supplier#000007217|A53luuNHtE3RTsRgVTjsfO56G|0|10-985-742-1483|8778.97|ounts affix stealthily. pending, regular accounts must wake after the i +7218|Supplier#000007218|64x6vb0UkDRkp56NRHYNKEAeED3L1RDA5O5G|22|32-706-226-5429|3284.09|ts. slyly even dependencies sleep carefully outside the furiously special accou +7219|Supplier#000007219|p5Ui3IGPcmotYu|21|31-888-210-9936|6672.57| carefully pending requests sleep carefully special foxes +7220|Supplier#000007220|,7dILJeEhKrHly2nD2K7BXNrioZ2w|4|14-575-445-3713|9889.10|ccording to the final realms are furiously unus +7221|Supplier#000007221|kS94WMnqV,|2|12-213-618-9500|5184.77|t deposits are quickly alongside of the furiously pending accounts. carefully pendin +7222|Supplier#000007222|uNRJrTpZQITrXmU|11|21-837-226-3845|7413.80|ronic theodolites about the even pinto beans cajole around the even, even foxes. quietly regular req +7223|Supplier#000007223|1jnl0HAZoFvpqs5MzooD7Alpr|10|20-986-400-4652|4805.99|y unusual pinto beans wake furiously special platelets. fu +7224|Supplier#000007224|dHIhpbSKQV8v1HTAriVLjK5Z|7|17-400-369-6821|-766.68|ccounts. busy excuses haggle blithely above the quic +7225|Supplier#000007225|F ykA n53QblHRXYMEOZr4WwJBF5T7JFLn|0|10-696-419-5791|4307.02| blithely beyond the deposits. regular excuses about the slow requests wake slyly furiousl +7226|Supplier#000007226|Aw6lbwDjwiSaI3y6PYaWt6UJXGvblZveDY|11|21-691-885-8563|6013.49|integrate ironic platelets. quickly regular foxes are slyly a +7227|Supplier#000007227|O5rzrk9evOe6IWbJrV,GIw2atEZmpS2|16|26-900-925-8777|9679.81|sly regular packages across the un +7228|Supplier#000007228|sb84OybO,0ucnpN5W8qy2uftZ4sn9MGX7Fi|20|30-541-928-2531|7617.58|e regular asymptotes wake about the slow accounts. furiously special deposits haggle f +7229|Supplier#000007229|iwNoWdaURFzLAsQHxK,BeOPpI5TOTo|21|31-503-440-8378|8798.71| the quickly final deposits wake fluffily express ideas. even accounts sublate fluffi +7230|Supplier#000007230|bVgNADApRN6dAUGrUgpuxsm XMjo2J14J|24|34-823-146-6618|5043.68|eas haggle carefully alongside of the pinto be +7231|Supplier#000007231|9eLXfCSdrrsk3n8CfDDth4qwomWktBImEyvR08q|23|33-994-796-9709|1091.98|thely alongside of the bold requests. +7232|Supplier#000007232|PwnbvnPDbyz3bcuGjdGeIoAOP|16|26-319-373-3131|2851.22|. final waters sleep blithely according to the r +7233|Supplier#000007233|ja32YcV2cAkJY|11|21-679-368-6693|2352.10|to beans. slyly bold theodolites affix blithely slyly pending ideas. furiously d +7234|Supplier#000007234|a1ouLX5lQTS0aRX3AC,M6qM|4|14-649-286-8919|6755.76|ently regular instructions. blithely silent sheaves engage furiously: even, express accoun +7235|Supplier#000007235|sjG F7NXC5wk9bZMrBXwSt2|24|34-432-304-5232|21.50|ggle quickly furiously express de +7236|Supplier#000007236|NmUSpgmcSA9eBb1kp7ycjB44jpGzjiwkmZd Qtn|24|34-752-157-4774|7291.48|use fluffily. boldly bold theodolites are blithely. carefully ironic a +7237|Supplier#000007237|ciV3sB3YZ6TSRjZGfbKgX0YMkVO,daIi|9|19-905-233-9800|6004.58| carefully final theodolites sleep multipliers. e +7238|Supplier#000007238|wZ,MnHKNnbRv7flb6VUPYrF7jOgfp1E02SaZxPL|4|14-864-807-5923|2995.20|ter the special ideas detect +7239|Supplier#000007239|knFmf6ucf8Y3KjefyQGwBdJS|6|16-810-863-5667|2236.73|hely above the special accounts-- slyly ironic excuses haggle: unu +7240|Supplier#000007240|ApzTXaYhoCBqijpuu29od3cEIhAsr|23|33-140-585-2550|3711.33|uests-- blithely blithe theodolites wake. regular excuses +7241|Supplier#000007241|8xtX4J2 GxQ,4|7|17-604-128-7288|4305.85|ke carefully across the even, pending asymptotes. busily even foxes wake slyly i +7242|Supplier#000007242|WQMe6sy53jRalzf|0|10-888-866-9595|8460.28|olphins believe blithely according to the blithely bold theodolites. busily pending depos +7243|Supplier#000007243|AbkpyP G2Bsuf1als07c3x|1|11-559-591-5678|3958.36|ructions integrate carefully. care +7244|Supplier#000007244|LsQ2MhveyvMBOalHN|7|17-771-349-2256|9170.74|ly final packages are along the furiously regular tithes. stealthily +7245|Supplier#000007245|5VOUnvxlJeOJ|11|21-663-724-2985|9775.37|ic deposits about the slyly bold requests +7246|Supplier#000007246|x16yQEGSrcdNzWFp5iNZLA5G6EFV8t|15|25-155-999-9378|7696.91| regular deposits across the carefully bold +7247|Supplier#000007247|JosmnE4uT6PQixnmkJqDKEtU1XNRp|17|27-629-215-8284|-65.16|y regular epitaphs may use even theodolites. special deposits thrash. deposits +7248|Supplier#000007248|tqBWdYeG uBLy|18|28-874-436-5507|2758.18|y pending ideas integrate carefully along the quickly blithe ideas. r +7249|Supplier#000007249|zHWfI MH5Oa79Mt 9OfkcOi11OfXKVkfoR|15|25-892-634-2039|715.10|lithely even patterns haggle deposits. slyl +7250|Supplier#000007250|k503SxMLaQUVqetp7u,QIxBYJudsgO9Ckwy,|21|31-139-450-7934|4597.83| furiously special instructions are slyly slyly express accounts. final excuses around the +7251|Supplier#000007251|WJ3 AJhmgwV56aAibT5ZLsozU|22|32-623-165-6389|9657.60|the accounts. unusual packages detect slyly according to the slyly final packages; acco +7252|Supplier#000007252|ykFg2LFMpAHHg2kDX5KtNyL2VYDWpF WXX,RFY |9|19-496-247-9423|8288.63|es. furiously regular foxes across the furiously even courts sleep req +7253|Supplier#000007253|MzACEECgTqQOokHW|6|16-895-363-7908|1126.02|nal accounts sleep special, ironic theodolites. foxes sublate regular theo +7254|Supplier#000007254|dNHRuDHLFJxlkqvgatiTXvJCR1K2 Vu|1|11-458-333-5500|2948.60|y regular packages boost furiously after the deposits. i +7255|Supplier#000007255|oOIF4BBayIivseCR|20|30-466-154-3995|140.09|lithely slyly pending packages. carefully ev +7256|Supplier#000007256|ZZZwPfP9X1xcLQ|4|14-341-828-6369|9498.87|carefully quiet deposits. even, even ideas are. furio +7257|Supplier#000007257|ag76lMmwqT|23|33-820-388-8545|2585.33| express, final pinto beans. ironic, final requests sleep. furiously final ideas acros +7258|Supplier#000007258|8uT WwVHEiucQh5PxrDdnSalPNtdoz|10|20-302-348-4764|-382.16| even foxes wake slyly. regularly final requests integrate carefull +7259|Supplier#000007259|S1U6lyVlF4HUUwbQyo5JzCxL7Y mcodVZXj hA|10|20-445-712-8250|-997.61|ole slyly. quickly silent requests are furiously across the unusual dugouts. slyly close braids +7260|Supplier#000007260|OZFzjVCfHXhBRV6Y1Sn97awPoBwWD31sDzM,Hkg4|1|11-953-899-5018|6094.29|ts wake carefully final packages. c +7261|Supplier#000007261|ut kr2BQNRm4mowfEbbpwiTu7gqZfZSote|20|30-336-678-3733|3800.21|ully express dependencies. slyly final theodolites nag alongsi +7262|Supplier#000007262|RwDXVgS6O8dmjfAJQDxlU|14|24-169-985-1533|2175.47|eposits. blithely even excuses use fluffily. silent requests haggle carefully: always +7263|Supplier#000007263|malQPdYc8xiup2MiFuKHa|21|31-733-855-9694|8159.59|, regular foxes. idly express requests sleep evenly final, silent requests. bold, regular deposits u +7264|Supplier#000007264|UKakjM2zil|16|26-994-601-6378|-761.91|ly final packages. carefully special accounts boost carefully after the dep +7265|Supplier#000007265|WZBdSXAlTmVnqJ 65UfoOlAmQfJZ|12|22-431-725-9863|7437.58|sts-- furiously regular accounts haggle after the slyly final deposits. furiously fina +7266|Supplier#000007266|tgSicj2ZHIKya|18|28-956-783-2750|8392.75|eas. sometimes regular ideas thrash slyly final frays. slyly final deposits c +7267|Supplier#000007267|wkE,TQd4XhOV9SPfgKd0igePnykkbjM|6|16-866-660-5960|1972.90|to the ideas. special deposits breach along the carefully even theodolites. permanently +7268|Supplier#000007268|ENw95gPg25Ns|2|12-354-583-7278|2504.69|pending, final dependencies. furiously regular escapades haggle slyly bravely final foxe +7269|Supplier#000007269|ZwhJSwABUoiB04,3|6|16-267-277-4365|8338.58|iously final accounts. even pinto beans cajole slyly regular +7270|Supplier#000007270|TksERECGdYZRPUjkUdDRZv5pW26cOTaA1|21|31-292-136-6841|4555.43| above the furiously regular requests integrate fluffily unusual packages: carefully pen +7271|Supplier#000007271|9CZVew4lgybKddJdduig3KrVySYs9vPKQlG|11|21-463-180-1307|1088.70|cajole across the furiously regular theodolites. expre +7272|Supplier#000007272|b7cEq4dJGRl2D|10|20-686-654-9492|2694.00|thely. finally even ideas +7273|Supplier#000007273|jpW05hgHIR1ax0pkU93nbvDrYsXhA|1|11-151-346-1839|57.93|lose blithely. silent asympt +7274|Supplier#000007274|sGXGtoXKb0qXjrdN oi,15zOP|3|13-547-947-7799|3619.79|ly. carefully regular dolphins are slyly. requests cajole according to th +7275|Supplier#000007275|eYXi,OyJ5abdL1Breiqm9,7Dg|6|16-260-363-4691|5847.95|rough the furiously express platelets. final foxes haggle. even instruc +7276|Supplier#000007276|Vi9,aBg2ychZf|21|31-632-365-8754|-484.10|deposits wake slyly across the ironically even foxes. ironic accounts haggle ca +7277|Supplier#000007277|bV0Y3ewDUbmexWbA8XhWTyL6 BzjRF|17|27-280-749-3078|1615.51|re regular, careful pinto beans. ideas wake. slyly ironic theodolites engage. slyly regular +7278|Supplier#000007278|I2ae3rS7KVF8GVHtB|3|13-459-259-9441|1793.43|s cajole slyly. unusual, permanent deposits nag. regular depths sleep sly +7279|Supplier#000007279|FJsMa6CsK,Qks25mMvs4WajrnJTn|15|25-499-550-4752|111.76|beans. slyly regular theodolites are deposits. express +7280|Supplier#000007280|c2bU0RrE0BBce1PiOaSHqhuQbQnGw7qGX|12|22-804-704-9601|6210.00|he instructions. furiously p +7281|Supplier#000007281|sDqiFy Lfa9G5sLhA1mw7fuUSFB|1|11-972-878-2238|1436.11|ep blithely after the ironic requests. express dugouts sleep. furiously close packages integrate flu +7282|Supplier#000007282|PtJlEVgkt86VvPa gfCwgq3owArvCcQ ,JmE5|4|14-456-102-1263|616.94|omas nag carefully blithely unusual packages +7283|Supplier#000007283|qSEuR1Ri4Hrk LrOLGh|18|28-567-310-8067|4901.58|hely close deposits. furiously unusual packa +7284|Supplier#000007284|,BL3APKaiZbYh4lDWl7nwMrg9cqyIVuVaC|5|15-548-268-9777|-174.09| the carefully final packages are furiously regular platelets. bold accounts hang blithel +7285|Supplier#000007285|26,fyUR52kfle|23|33-107-393-7355|8954.35|e blithely. even theodolites are blith +7286|Supplier#000007286|gLuNuDQfFqub8AcO9V6FS,vhXTXVE|5|15-280-171-3074|2836.23|nusual packages wake bravely carefully express +7287|Supplier#000007287|uC4PYwKmDQWzhHTZbLSNzDkffjryO0XoHl|9|19-579-738-3799|3495.88|nt. carefully unusual accounts wake across the accounts. even requests wake. slyly regular +7288|Supplier#000007288|SBPQKok8avYzretytWCTiU5LYnx5swhu|12|22-540-960-5607|2305.17| carefully. quickly unusual requests use fluffily pending, regular t +7289|Supplier#000007289|GoYBOvCn2XuN,7i|14|24-296-865-2694|9885.57|riously ironic excuses. furiously bold accounts doubt above the dependencies. id +7290|Supplier#000007290|afT1C khnuXPI5MGmB5k8saraF7sizHGtm,vJWc |20|30-548-854-3310|856.58|nic foxes haggle furiously special deposits. carefully silent re +7291|Supplier#000007291|kvUtgGrc rtJQXYuYU0PITXPpPvbxwOcciEvRO1D|17|27-787-234-9070|6502.36|e carefully. slyly even deposits detect quickly furiously express dinos. ideas poach blithely. unu +7292|Supplier#000007292|XfG7Vd6 JQzCm|7|17-148-172-4176|9125.21| ideas. final asymptotes are +7293|Supplier#000007293|uM3PEucMaB,xZXl9hd90bccWMQ,|11|21-415-548-4457|3517.11| bold deposits kindle carefully along the express ideas: slyly even re +7294|Supplier#000007294|EyOgjdxcfyTvyw4MIs3YQoREk4FJ9Vt|5|15-863-561-9005|7092.68|the carefully regular theodolites. furiously even +7295|Supplier#000007295|Bpw1iyDtz919YV,tAaejP|19|29-594-566-5925|8734.60| packages cajole furiously ironic, even accounts. final dolphins ab +7296|Supplier#000007296|j7oSuCaztwM3PYKGk6Nh56|20|30-893-557-9029|5567.32|es haggle furiously carefully ironic theod +7297|Supplier#000007297|XqpA5r7IRvdZP1nTOUYw|7|17-551-230-4726|2009.27|evenly ironic packages are. fluffily pending deposits integrate. ex +7298|Supplier#000007298|3lJ7mWLnn6q|11|21-603-756-9582|8750.15|kly ironic accounts. blith +7299|Supplier#000007299|3WSkVU1ME,JrAaMCJhHixzzzEMY0eTBWpvDX7Lif|3|13-546-854-7549|2876.66|y pending pinto beans are furiously even +7300|Supplier#000007300|YfbUYzk8UMpmbThCwzHSKg8xlvfx0K2ODpMiPLD|19|29-192-770-8538|1034.96|n deposits. slyly ironic requests dazzle. car +7301|Supplier#000007301|FWp9qX40Urvuf3mQ8dlI0L8I|7|17-626-730-6751|1552.52|grate carefully special accounts. furiously bold gifts use above the quickly +7302|Supplier#000007302|QAerqePJk tcgi|18|28-519-634-1060|-833.33| instructions. furiously unusual foxes caj +7303|Supplier#000007303|tUAUS4uifQ4jiW160a2T1ksgZKbaWf11eKT|10|20-238-593-8892|6108.73|y express accounts are furiously after the fluffily ironic theodolites. furiously ironi +7304|Supplier#000007304|n 2pSn2qMg85wHuzOwsTRs0G73Ni|3|13-282-100-5446|4370.50|y regular accounts wake fluffily carefully express foxes. quickly ironic ideas dazzle f +7305|Supplier#000007305|pFuKnETctQn1J|7|17-698-827-9966|5961.20|ffix fluffily bold, final packages. quickly expr +7306|Supplier#000007306|oE9FeqTebvdIWLZ6G|7|17-341-733-5105|-130.03|around the ironic, ironic packages. carefully ironic courts are even frets. +7307|Supplier#000007307|oqxuVj9loAZ|5|15-946-270-8340|1502.02|blithely bold pinto beans use against the carefully p +7308|Supplier#000007308|OFlDRSzpirdQmRXavGABZ62,LGxM2UXk1s7y|9|19-686-605-5793|8341.98|ular, regular accounts wake across the furiously silent instructions. ironic ideas after the c +7309|Supplier#000007309|BmjrvlihSVhZzs8pLNF4xW,R1Dn eiuGR|11|21-272-548-7085|6720.86|lyly. idly even pinto beans along the Tiresias doze +7310|Supplier#000007310|b70TwzmoAia 7umUDBk,mfXJWFhxd|23|33-954-918-2762|7584.69|s boost slyly toward the evenly express dolphi +7311|Supplier#000007311|UhT1waexydhU82 ,AQIIUt9lwuv5EhTs7xXHGI|13|23-173-743-1842|5888.82|efully pending deposits. slyly express platelets boost along the carefully even requests. r +7312|Supplier#000007312| 8jris8HYZXJrgVN23zMQVePFy5TUVhpPS|20|30-285-236-1487|4479.38|the fluffily ironic deposits. slyly final requests around the quickly even foxes wake quickl +7313|Supplier#000007313|meT,KifX5L2se|8|18-769-841-4284|9870.20|en ideas cajole furiously across the daring gifts. carefully bold accounts boost fluffily +7314|Supplier#000007314|Qwq4e kveE|16|26-891-235-5057|6507.36|t ideas. dependencies use across the furiously final instructions. blithely even requests haggl +7315|Supplier#000007315|nANqnt IpXjP,d4XJXMSqV74uWTSUc8A34NjW|22|32-946-129-9640|5225.05|slyly bold braids. blithely special asymptotes sleep about the slyly regular +7316|Supplier#000007316|uAhbZzA6S2VcHy|15|25-539-229-2317|2390.24| slyly final pinto beans haggle closely fluffy dependenci +7317|Supplier#000007317|nJGsPl2ruoq4Ydtv0fwWG385eOFV6 VUokbCZ|4|14-666-351-2592|9714.60|ons integrate furiously? slyly ironic requests sl +7318|Supplier#000007318|rk,XcKlao GIe35hZUcjt6dcyxHo4OBOhM4m|14|24-185-616-3920|9374.51| fluffily ironic packages. requests against the pinto beans boost regularly slyly +7319|Supplier#000007319|awgO5pCDNnL3,7NviK0fpBt2bA2bGOhEh|5|15-838-867-9396|2535.92|rious requests affix blithel +7320|Supplier#000007320|zwDCjiHUZck2qwqZ1ilPzA0N0E82T|18|28-805-742-5077|5175.71|ages above the pending, express deposits affix regular, even +7321|Supplier#000007321|P0PzqTa0LNYs|8|18-359-235-4861|9176.70|er the requests sleep at the quickly even packages. quickly even pa +7322|Supplier#000007322|wr7dgte5q MAjiY0uwmi3MyDkSMX1|3|13-390-207-8249|-177.55|ect always at the quickly express +7323|Supplier#000007323|OUFIl2BHtccFOEaT7|17|27-941-979-1583|5781.75|ccounts. ideas alongside of the carefully final ideas wake fluffily enticing platelets. bold, bo +7324|Supplier#000007324|vf9OE3AZ,Q1vLaP2 u|9|19-293-838-7328|8873.81|accounts. carefully unusual requests against the quickly unusual deposits serve stealth +7325|Supplier#000007325|49QOb4br8oFQPncoWNP4fy4F46kRc4,Tk|10|20-536-413-4233|2207.75|pecial instructions. furiously bus +7326|Supplier#000007326|1,TZ6EFFBIYbxe5j46ICs fkOLnC49ha|4|14-865-827-1371|656.89|leep: excuses wake quickly above the bold foxes. theodolites wake. express deposits about th +7327|Supplier#000007327|uZGHBo4xgx W2sT8oA5drPwqQL8Bw5YU,ZNb |12|22-154-891-1374|5352.15| to the carefully unusual ideas sl +7328|Supplier#000007328|pX2LnnOYkaVJTM70e EYctoBJkjmwxDFD0|10|20-668-147-4629|4558.44|ecial accounts. carefully unusual grouches are furiously regular theodolites. +7329|Supplier#000007329|bF,Rjakq7fwBvfzwyROUPPtSvuZkUu4MtOws|22|32-241-689-8384|7668.58|beans. platelets use slyly about the regular accounts. accounts affix blithely at t +7330|Supplier#000007330|8sLxWqHCStxmWoU Bla1JOYJ1jQmEklP|17|27-689-370-8022|7928.93|lyly unusual ideas play busily. packages affix fluffily s +7331|Supplier#000007331|x3vIcAygRX6sDMch,ANYy17mMnBweAOCRi|4|14-379-845-1562|5620.76|sits in place of the furiously final deposits hag +7332|Supplier#000007332|gVo6DrbBSAzBZaGj3VYf3gRn|15|25-612-368-7830|9135.83|odolites. blithely ironic asymptotes shall are. idle pearls sleep carefully! packag +7333|Supplier#000007333|9JJQB 6k0DJB6vFBav,|4|14-802-542-1048|5909.84|s along the slyly unusual packages are above the c +7334|Supplier#000007334|NPXYWdJ8L9EDr20tw9CZQsEMqXlgXzI2JC Y|21|31-803-710-3346|5654.21|r requests thrash blithely about the final +7335|Supplier#000007335|CrWf0Gob0n6LVIKhmGv3WawLORNjg6scbZ Dl|16|26-130-174-8524|-649.92|ingly bold deposits use re +7336|Supplier#000007336|y0BKztziBCkvpiT4FG|6|16-180-420-5844|8778.96|ctions. ironic pinto beans wake against the even, special for +7337|Supplier#000007337|nz05XGx3k7VJYEzd7Dix|5|15-236-375-7373|2996.46|ounts dazzle quickly above the blithely +7338|Supplier#000007338|FgOOm6MpqY3mE wnIrAStuyf|8|18-664-949-8603|1527.32|y. pearls haggle-- regular accounts sleep. carefully final requests according to the instructions +7339|Supplier#000007339|N8lH6VcOyzGl,O7DogZA,VM008ORQcDdD9j4|13|23-140-824-8664|4637.91| above the slyly ironic excuses nag fluffily eve +7340|Supplier#000007340|9f,1UUNu5Ug54qe17YAuNfVi1yf2ZcJYTZRyDn|3|13-864-417-5265|1333.77|ithely. furiously final requests wak +7341|Supplier#000007341|bt3rwDKuDwJIExBNm hfdYmy|16|26-158-434-3089|7685.06|ully regular theodolites kindle. packages wake among the regular accounts. quickl +7342|Supplier#000007342|JOlK7C1,7xrEZSSOw|22|32-399-414-5385|9836.93|blithely carefully bold theodolites. fur +7343|Supplier#000007343|OIlkeR2Fi,tMgvXuKFIQcqnO|23|33-586-573-8880|8296.62|unusual, regular pinto beans haggle quickly about the furiously p +7344|Supplier#000007344|4Db1i7k4kP0vVT1iRu693YXe5Zy,|8|18-543-611-3790|3349.81|slyly unusual packages nag blithely slyly re +7345|Supplier#000007345|6LOXvCNnVpleDYKB88ex1P7V4oUFpW|2|12-103-401-2717|191.28|. unusual accounts affix final deposits. quickly ironic depo +7346|Supplier#000007346|O0c816gzadoKlKdet8RacnIBdDAw|19|29-488-831-1697|781.20|refully pending theodolites wake ironic, pending t +7347|Supplier#000007347|MxOzRwSCLP9qtQcx8VYFZBhICEJdGy|8|18-252-234-1859|4532.74|icingly final theodolites. slyly bold deposits h +7348|Supplier#000007348|DWsHlBJHxcggXDAGzeCyWPTvGB4f43aqYhd|24|34-865-949-2340|4727.63|ke bold platelets. slyly even instructions use +7349|Supplier#000007349|Hmd0c9i,NamcO8k88GWi7dvVPyiGo|1|11-615-661-3009|2236.24|against the fluffily unusual pinto beans. furiously pending requests should +7350|Supplier#000007350|Sn4p2QFhb5k82dkmyjFFlgtLbDf|14|24-688-354-2976|6532.34|al packages wake furiously final pinto beans. +7351|Supplier#000007351|ze6ennNi5SKzB8f6 J5sHm57 j06f0MVQ|11|21-762-689-4646|7082.66|ess theodolites. slyly regular accounts haggle carefully. blithely final requests nag. sl +7352|Supplier#000007352|S7dBXzQa0lPuLtvXBFB,Mln,2pRgae,T|6|16-949-393-2959|5340.29|ironic ideas after the accounts cajole quickly blithely final multipliers. furiousl +7353|Supplier#000007353|ka8HKKDbjI, Ok8TBMPf,lk|12|22-418-697-4642|3344.74|e of the final dolphins. pending, even requests integrat +7354|Supplier#000007354|aeK 25BHHY2z1O 7CjrjQ32WCBuTp,XbFZAc4 TE|14|24-296-134-6462|3105.46| beans nag slyly about the carefully unusual instructions. carefully even requests could +7355|Supplier#000007355|bH1s5 4Q,2Oc,98dKQOR5HZ |18|28-392-158-7069|7382.90|ickly unusual excuses wake furiously at the regular, expr +7356|Supplier#000007356|0xNYRuPSssnEN6ypm8uAx5IybcJ,I6OrTToR|8|18-135-522-1643|2485.51|ithely even asymptotes sleep quickly bold, even depo +7357|Supplier#000007357|HR2HMaX xAvHJGkJ wOCqV9K77b5|23|33-222-585-9707|4968.15|e bravely even deposits. furiously final deposits haggle even +7358|Supplier#000007358|0U1 6bUtHv8Ndu|19|29-794-437-5593|7652.77|le express, final deposits. quickly regular +7359|Supplier#000007359|SYl0YoLC8A|21|31-245-137-6681|6230.73| pending accounts cajole blithely blithely pending packa +7360|Supplier#000007360|a3MOdQcLllvnQ|22|32-565-430-4290|8834.23|ly slow asymptotes. boldly even platelets are carefully across the regular packages. bold, bo +7361|Supplier#000007361|msFsvlfRdwjfLuk0FYdo5htVVDIKrdlTNeMwP|24|34-962-658-2126|7980.89|nts wake. carefully pending ideas wake blithely! e +7362|Supplier#000007362|8PAZ3synXExRh9H5D0KT|10|20-999-975-5790|9798.85|ly unusual pains according to the furiously regular requests sleep above +7363|Supplier#000007363|rsRXG1avYTbZZ0rQJSov|7|17-279-982-1929|6369.05|ckages. ruthlessly final braids may wake along the slyly even +7364|Supplier#000007364| IHaD3oLzh sUP4fQSpviOWk3|7|17-665-996-9676|1599.59|e dependencies. fluffily unusual ideas maintain along the carefully regular patterns. blithely reg +7365|Supplier#000007365|51xhROLvQMJ05DndtZWt|3|13-268-624-3593|9304.54|t the fluffily bold requests. fluffily ironic id +7366|Supplier#000007366|7SeKZHcFCApYBVhHh2d|8|18-909-530-8832|3479.30|kly up the furiously pending deposits: regular, regular pa +7367|Supplier#000007367|bTP8DLvaRAB0n|4|14-234-934-5255|9557.33|ep slyly regular accounts-- regular, regular excuses dazzle furiously about the reg +7368|Supplier#000007368|7idONcheAA8L26ipzDXLnrAY2OSMKV0 Mhp HzJa|14|24-170-941-8674|6174.94|regular deposits against the waters sleep whithout the +7369|Supplier#000007369|scvOOLLT4mXO6mtuaLmoYVb2,Uz 8nVUbUb90g|9|19-532-859-9127|6187.85| after the furiously pendin +7370|Supplier#000007370|42Bmm0jBt71M5PI0l8maW|20|30-382-520-5768|3289.26|s! quickly even packages detect +7371|Supplier#000007371|ouhoak,WkHQPJl|20|30-909-527-3553|5543.30|express deposits. carefully special deposits are about the deposits. q +7372|Supplier#000007372|7BY7Lpkw0W0CaC11M5jkFw7fxcOdY86wUN3R|23|33-661-701-2273|5189.97|its after the ironic theodolites af +7373|Supplier#000007373|QM5agRKsO9Jum2BMcstO9ZroT qpppshbxs|12|22-674-381-2045|6848.59|ckages sleep slyly by the requests? bold ide +7374|Supplier#000007374|hXPCDq Y,NT9x1WD3BEbGeZ8osHYds|24|34-774-530-4188|9298.32|ages haggle bravely. blithely pending foxes wake furiously +7375|Supplier#000007375|xYRtkObjcIwnZ1UV2bxQzPA|17|27-845-316-7062|8883.91|ses. carefully regular excuses haggle carefully after the blithely regular instructions +7376|Supplier#000007376|zWndValk7etbXGziKw8D1KvdjrgL8YipBT|17|27-934-409-5203|7023.84|. furiously final instructions sublate finally pending pinto beans: carefully even pa +7377|Supplier#000007377|PI3niLhLgAqt4RiVR2dQUvuEPtq|5|15-567-196-2012|6691.75|asymptotes. even instructions haggle quickly furiously special requests. bold reques +7378|Supplier#000007378|GnTGpCefwi832,fo71OMc|16|26-355-284-9378|2296.82| across the furiously silent packages wil +7379|Supplier#000007379|ei5 DaCLCraASeBfLYAaA5|12|22-133-295-1829|1601.91|tes. final theodolites breach express, regular accounts. carefully regular asymptotes sleep +7380|Supplier#000007380|MRgHoeIsZf1lA |1|11-912-946-9060|4219.92|requests cajole. fluffily ironic packages affix abou +7381|Supplier#000007381|zHlai,Ux7pD,0p8jK2kx|20|30-938-562-9811|5216.33|ncies sleep about the blithely regular pinto bea +7382|Supplier#000007382|p9BaidBFCsmn|20|30-115-989-9371|9880.70|counts serve carefully fu +7383|Supplier#000007383|ECCvFTjqtlzXlDuT6erP51Il|24|34-604-440-9988|7030.55|lithely regular accounts in +7384|Supplier#000007384|fPH27,m57mt lgutR|24|34-126-374-5859|9565.89|cies. furiously express accounts shall have to prin +7385|Supplier#000007385|bZYyLInf0Kg4n4DFks5hm1OgHWfA6K,DQvEfat,|19|29-529-645-7684|7028.76|pinto beans. regularly silent accounts about the regular, regular pinto beans are among th +7386|Supplier#000007386|7OwCfIcamhJtcgiTUcUq0lMGzVT0CWmZvq245G|22|32-799-168-3188|1417.33| courts sleep bravely furiously even asymptotes. furiously +7387|Supplier#000007387|TTTg9Q9ubNYwBrydOkvZQnifkx5JxVUhFdIUAY|20|30-563-477-1315|4055.11|theodolites. unusual deposits run quickly according to the blithely bold packages. blith +7388|Supplier#000007388|0pZLY87lg8dg|1|11-603-534-9813|7218.48|ng excuses. blithely final packages sleep blithely among the sly, +7389|Supplier#000007389|BGcj3l88CBqb|19|29-446-540-7488|107.22|s. final pinto beans dazzle. slyly regular accounts nag a +7390|Supplier#000007390|29wYY3Dz3y6GB0k|10|20-132-165-2168|6987.40|otes boost furiously accor +7391|Supplier#000007391|iX14MlXXppDv5jJMpM7mE|6|16-840-162-1413|-653.25|s detect furiously furiously final requests. regular ideas wa +7392|Supplier#000007392|JeeScPASUUcJFQbjB5BVwXb8De3VTYddzkGwBv|10|20-928-350-9877|7751.80|ular accounts detect carefully along the unusual pinto beans. final, special pla +7393|Supplier#000007393|o93 KCogRzIHG8YznDb|17|27-616-414-1665|9619.44|elets haggle fluffily. furiously express ideas haggle f +7394|Supplier#000007394|DjRSei5omHFhieDPWpx,JjFxt0btINKh|10|20-601-570-3864|5081.76|ests use according to the fluffily special pinto beans. ideas use slyly. q +7395|Supplier#000007395|oGinj4gWE9EnO6xCg4s1rMzyMv85wFOoRzemgBvD|12|22-139-602-8510|3208.95|fully. regular, silent deposits cajole furiously furiously regular requests. slyly regul +7396|Supplier#000007396|RBqY8HVamTPASc674Eqtk3Ql,|22|32-735-362-7935|8815.90|e above the ironic, final accounts. slyly express deposits haggle carefull +7397|Supplier#000007397|k bmt6UGSGnItCHgbNFKgQCuc3jfJY|21|31-614-697-7115|7887.63|ost fluffy, ironic accounts. enticingly special pinto beans run slyly slyly +7398|Supplier#000007398|V8eE6oZ00OFNU,|3|13-304-352-9285|4390.85|t deposits. even, pending instructions wake slyly. blithely regular +7399|Supplier#000007399|zdxjENOGR4QiCFP|11|21-859-733-1999|6964.04|e blithely after the even requests. carefully ironic packages use slyly a +7400|Supplier#000007400|7r9zZj8J,,hN2GRfWtDxzuGa|21|31-132-846-7821|2656.49|s. furiously regular accounts boost. ironic, pending ideas detect regularly +7401|Supplier#000007401|0bGoBtTkMpgUOyDlAXEdx6I 26|18|28-719-774-9928|7532.68|regular requests. bold, fluffy escapade +7402|Supplier#000007402|4UVv58ery1rjmqSR5|3|13-625-537-8029|764.43| final instructions cajole furiously; fluffily even pinto beans +7403|Supplier#000007403|bgyKgLn3cOZzh9yyMWy|19|29-416-279-4283|8749.11|aggle blithely busily ironic accounts. slyly regular theodolites ca +7404|Supplier#000007404|uJ6JipGjD9H2B7TFwbniGkEf5nHKHfWDDw|17|27-982-883-8616|7429.73| across the ironic, ironic deposits. blithely even excuses are furiously ironic excuses. e +7405|Supplier#000007405|dTwfunl43XW25C94xZtQRbs|11|21-638-228-1861|8552.27| slyly final accounts about the carefully ironic pains wake +7406|Supplier#000007406|eugL0NHIQ0UX8PpqeI|8|18-581-490-7546|-856.38|fully even requests sleep around the final, bold requests. quickly +7407|Supplier#000007407|ohUJnJvZ1sXrT2bWXJNx|23|33-333-594-6574|3818.72| carefully bold foxes. quickly final instr +7408|Supplier#000007408|fEZ1,SsXEwVGW1C8p6xHCoIlDWQm1tdhE|11|21-517-911-8957|5277.49|ites against the express foxes nag furiously express dep +7409|Supplier#000007409| orXUe6dWG3qpH8oo543k60|13|23-874-588-7225|3823.34| sentiments are regular theodolites. always ironic accounts unwind q +7410|Supplier#000007410|ej9in0Jk8uxXVTr2PD,0|15|25-115-755-7525|5797.55|ress packages affix furiously daringly regular deposits. e +7411|Supplier#000007411|CNyc3Jvk4jSckyj0xU|18|28-501-398-5271|-621.24|eodolites haggle fluffily. final platelets sleep. even, ironic requests haggle against t +7412|Supplier#000007412|x7v4PmfH76nwINd1hbc0IqvyY5nrDH1|23|33-519-444-2525|6668.60|theodolites cajole carefully around the fluffily special accounts. regul +7413|Supplier#000007413|JAm6CYvamQ1Oavf6Dpc2 YGt6B,K3uye LFRjx|13|23-970-853-1710|9138.40|slyly regular asymptotes. ruthlessly even +7414|Supplier#000007414|IRowsjn2vYrUOAx5z|7|17-492-201-2393|5098.12| have to grow slyly blithely ironic realms. requests nod above the quickly regular excuses. b +7415|Supplier#000007415|JJYykAlP0FsVdL|11|21-714-344-2604|3156.68|gular requests nag. regularly r +7416|Supplier#000007416|U7P6,C42ymp|17|27-612-915-9794|-778.03|e carefully. even, even theodolites wake. regularly +7417|Supplier#000007417|85gxuUKJ5JHUvSy3mAqsQ ,49DEujZB03mIUk|20|30-976-203-6169|636.56|efully unusual asymptotes. foxes h +7418|Supplier#000007418|SsWadX3k2M vjymTh4IaOrLZ0vEfxwkbmkVfyNb|0|10-114-167-6974|4573.74|fter the blithely silent foxes wake above the realms-- even foxes across the furiously e +7419|Supplier#000007419|m84ie7wCVEFturwQ,XsfIjsmXE3TzE|7|17-783-878-8183|881.07|ckages. fluffily unusual +7420|Supplier#000007420|pltGihWq iScZqq|24|34-294-186-8855|131.75| the finally special courts. slyly regular ideas haggle fluff +7421|Supplier#000007421|FSYmohAxhSu11arpuAizmXeK|19|29-644-291-7830|1216.33|n excuses are above the accounts. express instructions haggle quickly above the +7422|Supplier#000007422|scnZzhg0YdHUIdf|4|14-723-980-1437|2758.58|, final accounts haggle boldly express deposits +7423|Supplier#000007423|1t0vzPO0Lzsvz lJnh NbiEdjQFxk e8Ola|11|21-876-649-7878|7383.37|ake carefully about the blithely express pinto beans. express packages sleep. packages wake c +7424|Supplier#000007424|LkmE8pkvAyBMw3zuOXjK3EK4wlh3WYz|4|14-771-660-5924|-655.15|lly about the pending, express accounts. quickly +7425|Supplier#000007425|TAqmXEdgxyxdIDal PT7HjJPqgT|19|29-711-206-8445|8793.45|tions wake. ideas wake slyly final platelets. ironic packages wake quickly. e +7426|Supplier#000007426| 4r0HDt85zLPoM|3|13-228-593-6695|4584.59| carefully along the furiously unusual pinto beans. pending foxes sl +7427|Supplier#000007427|zu85osCcQGpY54vnEUTGMIrFGRMckk,ZMC4Ao|5|15-883-816-7527|7197.00|leep across the pending deposits. requests are blithely. final, regular theodo +7428|Supplier#000007428|70mTTnsPpkA22r8ygMPNpNEO,z,IylWA7s|2|12-796-136-4924|2087.25|y among the slyly final packages. even, +7429|Supplier#000007429|PzGXj1f6j vjea|8|18-930-524-7128|-576.69|e of the deposits. regular requests boost furiously at the carefully ironic requests! unusual packag +7430|Supplier#000007430|6XfH6NiFdely|24|34-996-657-4788|7566.99|quickly among the quickly regular requests. furiously even requests wake furiously exp +7431|Supplier#000007431| HPr3oyuG,ZblwN7GUr8Z5 6nT8qL|13|23-946-608-3412|5464.10|ely slyly even ideas. instructions hind +7432|Supplier#000007432|NgBuGog0iL5G918dWdjBVq|5|15-803-834-8325|5201.89|thely slyly regular requests. furiously slow sentiments are fluffily of the regular, +7433|Supplier#000007433|JkFXNhxBeouyBAyEPyG05|11|21-556-118-4183|9126.96|xcuses. final requests haggle slyly. carefully furious theodolites about the permanen +7434|Supplier#000007434|pveDnrbkQ4My,5dr91NwqhgbK|13|23-708-433-5099|-132.46|quests thrash fluffily. deposits after t +7435|Supplier#000007435|xPZenvNfjvw fnSNYUUTW6f|1|11-448-269-3587|4711.16|press pinto beans haggle after the carefully daring packages. carefully special packages wake furi +7436|Supplier#000007436|HVuRAb35M6j82A2hPcibeLKzo|22|32-791-422-6015|547.09|y. silent excuses with the quickly even dependencies hinde +7437|Supplier#000007437|OAXmJUfE1c7naS|2|12-742-862-9372|7916.11|aggle across the fluffily blithe theodolites. dependencies ca +7438|Supplier#000007438|LPKbazS1XiADueiEJLeoq,BucRnhhVSmaEdVqQuP|16|26-789-540-5015|-895.29| blithely final accounts sleep. enticingly even accounts wake doggedly regular platelets. fin +7439|Supplier#000007439|wYXjKhXUIwck0bSV5O,FYjh2ocBdgXleAy5SP|19|29-341-370-8531|9833.84|packages are. even dinos sleep fluffily packages. blithely ironic accoun +7440|Supplier#000007440|f3YKlu1r9 RL|19|29-674-945-7616|455.26|ent accounts mold final accounts-- furiously ironic ideas believe fluffily across the permanently u +7441|Supplier#000007441|58PkaXFO4wMIdteeUGXpuYYFf9JnQs3NiD3GNg|20|30-877-114-7161|4459.56|egular instructions across the regular, ironic instruction +7442|Supplier#000007442|DzycM1,T6kh2EutfPeFpv0Ro|21|31-999-609-6818|6895.79|usly final packages boost. deposits wake a +7443|Supplier#000007443|S,dOfluUwjy1al YenWKdknieXWTDTaS10VO|6|16-277-525-5408|-339.05|ites: packages haggle fluffily among the pending deposits. slyly ru +7444|Supplier#000007444|zaCZ4XLZump7 Pxl|22|32-970-132-1321|8449.22|gular requests boost quickly after the carefully final accounts. regular requests boost about th +7445|Supplier#000007445|TySK7rEzzGe4gB6u,,4CSDM0iTx|1|11-107-152-7059|5090.78|carefully regular accounts cajole. slyly final packages sleep. blithely ironic accounts boost furiou +7446|Supplier#000007446|sUDf3rG1Y,4XK3OB|11|21-479-697-7150|709.30|s. furiously regular dolphins integrate. regular, regular platelets haggle +7447|Supplier#000007447|d0c Qx09cNNYAhiGS19aqb,99lDaJtvszE6Xn9ra|13|23-114-669-2987|6095.15|gly pending accounts. ironic, regular instru +7448|Supplier#000007448|yhhpWiJi7EJ6Q5VCaQ|3|13-190-786-8552|7165.37|o beans. special deposits wake regular deposits. pending foxes haggle furiously. special deposits ac +7449|Supplier#000007449|aRS5s8KGcawKyM8dbBqa0l6ve8aMR8s6m|8|18-148-870-6544|9417.92|sits are about the slyly even deposits. ironic accounts maintain fluffily slyly ironic f +7450|Supplier#000007450|G ykHRKb36JjY1JTH2b45DiNKNnqU,hbrXY|19|29-890-926-9833|6127.58|ng to the unusual accounts. bold, final requests sleep. carefully even depos +7451|Supplier#000007451|wJCwChzzwCPf,nmjbgyQPh46R4nvdOd1xBr|21|31-833-133-1673|2299.53|ding pains along the carefull +7452|Supplier#000007452|sVt5kGMpNCfu0fAXV058yHMPs c BCBcsG0Y9|13|23-503-242-3661|3112.04|kly bold ideas haggle carefully +7453|Supplier#000007453|h8pnN6978tPWx|6|16-671-287-9714|8037.99|ajole carefully regular dolphins. pending excuses according to the carefull +7454|Supplier#000007454|gybI3LobLdbZeZ|9|19-315-642-4922|1519.16|ate slyly express requests; permanent foxes boost along the furiously regul +7455|Supplier#000007455|yj0sy598kDac5BYSdUsrzrAQ0s,wTS|13|23-891-692-3507|-527.06|ic theodolites sleep permanent deposits. furiously pending Tiresias s +7456|Supplier#000007456|ITYEeccPVJi0HvnAwVs2Z|21|31-194-977-8267|5664.16|s. pending, final requests wake carefully dependencies. pending, special account +7457|Supplier#000007457|5aSV83RQFpd,2y1uGvZjGQcM0bFuyT|11|21-978-479-1723|1541.46|ions integrate after the regular, even ideas. even instructions nag +7458|Supplier#000007458|BYuucapYkptZl6fnd2QaDyZmI9gR1Ih16e|3|13-631-411-3540|248.89|ding requests cajole carefully blithely sile +7459|Supplier#000007459|vdtdqoVZM3NQf0WWfN,Yz7M|10|20-247-403-8646|1095.44|sleep blithely carefully special pinto beans. express Tires +7460|Supplier#000007460|s4ypM7SV1D6ut8gtn8OtxV5|3|13-739-620-9679|5993.34|usual theodolites. asymptotes sleep thinly slyly silent courts. regular packages are caref +7461|Supplier#000007461|iXVtUhaFxwOQH6CIIwEgiz0e5ojKNxF58 j|6|16-615-287-1465|1106.37|ide of the blithely special packa +7462|Supplier#000007462|2yTDdBgIlGN0wwhNCERIgmdHFKkbA|13|23-756-545-6036|2716.77|nic requests. even packages according to the furiously stealthy platelets will have to wake across +7463|Supplier#000007463|U7O 9NmpIuTsFtziT1S2z1xe|24|34-145-831-6422|8686.15|r the fluffily express requests; carefully +7464|Supplier#000007464|Q9WBcTjgT1okf3sTwJGmnYU3QAc|0|10-613-422-9919|5229.29|ly regular excuses haggle carefully slyly bo +7465|Supplier#000007465|xwiFNsrmy,8l,KlUd1wINdB|9|19-651-621-8978|7675.65|ing accounts wake. carefully ironic packag +7466|Supplier#000007466|Iskr9uYWmA|22|32-748-898-1030|5637.23|oxes wake carefully furiously even dependencies. slyly special platelets boost slowly +7467|Supplier#000007467|D3VjgzrcOrMRi5x8WDqQ|10|20-820-641-5313|-66.19|. blithely ironic pinto beans cajole furiously even instructions. +7468|Supplier#000007468| E9dKc7,wBKDQtwEapLWicUwxs|18|28-262-785-5924|7427.47|even orbits. slyly regular theodolites detect carefully. final accounts across the iron +7469|Supplier#000007469|KKE9Mc1RMtPHJwcLuR7Ces4PZOqcVr|5|15-769-400-5716|3568.57|ccounts. furiously regular asymptotes haggle blithely fur +7470|Supplier#000007470|yHBwSk3Thj9qFmWF P|19|29-727-676-4971|2927.96|t the accounts wake across the carefully bold platelets. blithely even e +7471|Supplier#000007471|mpHjg ZMKz,AzIov|22|32-546-505-9261|5264.12|ly express deposits poach furiously f +7472|Supplier#000007472|Qe0h2mZMj,HZDDzDqAx5q6MDRE29Rb,M|10|20-506-646-1828|-949.12| fluffily about the dinos. pending grouches at the express, final packages boost furious +7473|Supplier#000007473|figThe,wXbLaTV|23|33-411-872-9153|7122.77|he accounts; deposits wake furiously. regular accou +7474|Supplier#000007474|3DmFgq7Gryc|19|29-953-127-5706|8397.50|, unusual excuses wake blithely +7475|Supplier#000007475|6u91e8jXtf1OZ50d53Jyzp8GFAaHF v2dP6hy3c|5|15-819-523-4605|4789.92|pending courts. blithely even asymptotes are blithely across the blithely unusual f +7476|Supplier#000007476|z8RgUxhZSR|24|34-331-692-6417|4217.09|refully pending packages. even, express theodolites use. quickly final pinto beans are +7477|Supplier#000007477|9m9j0wfhWzCvVHxkU,PpAxwSH0h|3|13-583-330-2709|1571.25|l asymptotes cajole caref +7478|Supplier#000007478|g2NN3fXFMc4uJ|23|33-864-385-2033|6000.95|uriously. sly foxes cajole blithely blithe +7479|Supplier#000007479|ROwWDiSbA rsvex1Ptz kV5Ft|23|33-358-933-6532|7907.49|ackages use furiously alongside of the blith +7480|Supplier#000007480|ubM2xuLS0u |4|14-851-119-1838|8262.82|ly final requests; furiously special packages +7481|Supplier#000007481|YFrAFgdH5cYR7sFEt|11|21-506-229-5173|7353.23|sits after the furiously final sheaves cajole slyly carefully bold ideas. final, +7482|Supplier#000007482|7MSkjk6Z0e|14|24-502-520-4698|9784.29|the quickly special deposits. ideas among the slyly regular deposits are amon +7483|Supplier#000007483|NYifFQDtYm7ybNBfACN|17|27-924-602-4011|3581.17| fluffily express requests thrash according to the furiously slow foxes. regula +7484|Supplier#000007484|mOM9ggPwQSI1SX5lpumMLWc85u|21|31-500-504-7115|5713.13|regular requests according to the ironic tithes +7485|Supplier#000007485|TnjabJ6xAZ9n|4|14-340-351-6780|-418.26|ts cajole. quickly pending instructions wake. ironi +7486|Supplier#000007486|AjfdzbrrJE1|4|14-970-643-1521|8159.13|ld accounts. enticingly furious requests cajole. final packages s +7487|Supplier#000007487|,J6ak95pJEOgkXlwC3Iih RgRpoIE3rhucZL5M|5|15-183-399-5255|2588.98|ependencies. fluffily unusual instructions against the pending, special accounts w +7488|Supplier#000007488|cATpp9zHpsa4Y1wdpbbxHA6tZm,gI|0|10-149-832-5333|5980.34|ithely regular depths. regular requests haggle blithely about the deposits. bold, iron +7489|Supplier#000007489|NYxeEtz UBTtnJozcAkTAG1P N9qAqqpWRl|6|16-504-669-7466|2531.11|aggle? quickly bold pinto beans cajole. final, special requests serve carefu +7490|Supplier#000007490|4E5gwkoxUw5EPHxUxeP1a0 jVrR jCwR|23|33-581-252-8292|1266.47|ily. carefully quick dependencies cajole. fluffily bold dependencies kindle +7491|Supplier#000007491| rgpN2KQE6A0gul0q4ZVpD5S|20|30-642-433-1399|9445.20|packages wake blithely against the carefully da +7492|Supplier#000007492|j4GTlT,MQlEPWsuSNUx0k7p|19|29-660-153-6870|2577.83|y even pinto beans. blithely eve +7493|Supplier#000007493|w DN7Uo5X17u7G9PgNRpsUr92bE6Y|2|12-364-475-9599|5499.77|refully. furiously bold t +7494|Supplier#000007494|vmCUgh60da0HG|20|30-946-217-5679|-342.33|azzle furiously. slyly express ideas are quickly among the pending deposits. slyly special p +7495|Supplier#000007495|3jQQGvfs,5Aryhn0Z|11|21-953-463-7239|8351.75| requests. carefully final accounts after the qui +7496|Supplier#000007496|mMAkGHqZaHQHfdReZ4m|15|25-161-303-6074|5956.08|re during the furiously ironic accounts. express, bold +7497|Supplier#000007497|zBBC5 Uiazuu7E awhFEhPiqsmjq3YoL|20|30-691-617-8205|1864.36| above the slyly ironic requests. bold packages boost about the +7498|Supplier#000007498|hrMVIZ H9Aqrtpl9J4,g2tKLz|4|14-497-243-1560|4256.71| packages are quickly close instructions. furiously i +7499|Supplier#000007499|6 5mk6a8fPyliCLSLh0trLNTOk|19|29-440-620-5955|7534.87|oss the unusual excuses. unusual, special deposits around the ironic dugouts cajole among +7500|Supplier#000007500|Y7twNc5h0E9zD55Dnyra|9|19-468-400-3864|4563.28| regular, final platelets integrate fluffily ironic packages. slyly unusual accounts wake. in +7501|Supplier#000007501|AsB2Fd8oZg9FeNH4VrCVqA|13|23-821-356-7066|1064.42|ajole quickly slyly regular excuses! care +7502|Supplier#000007502|,rSo5R1i654jwDP8DIUT1W1YW1TJ3onUPaXa,2|18|28-622-664-8208|1627.80|iously at the sometimes ironic deposits. pending platelets detect beneath the fluffily unusual +7503|Supplier#000007503|ZsYDRAa2yZzSCBrnVDKW,xOwrzswX3QH |22|32-499-180-9030|-640.53|packages nag carefully regular accounts. stealthily special dependencies hang furiously bl +7504|Supplier#000007504|Jd0UZtAgCGSTqPiyj614n|11|21-608-908-9154|1613.81|even dependencies boost blithely carefully ironic dolphins. final pinto beans doze slyly. s +7505|Supplier#000007505|HrpvrWPPfiNgNUUZIkZQWsMN|5|15-512-324-2294|6330.67|ons affix fluffily among the furiously even excuses. pending, quiet accounts above the slyly fina +7506|Supplier#000007506|swINMWWwcXRSltVzV|5|15-536-477-9412|5640.74|fully blithely ironic excuses! carefully unusual theodolites cajole quickly unusual requests. care +7507|Supplier#000007507|vUfH613Y2e8lxIEaVMbMqRsvcTT|10|20-946-110-2952|3908.77|tructions are above the special ideas. even deposits are carefully? packages cajole +7508|Supplier#000007508|YnSNa1CMGO|15|25-545-852-1929|643.93|structions wake above the silent accounts. furio +7509|Supplier#000007509|q8,V6LJRoHJjHcOuSG7aLTMg|3|13-514-440-5268|7514.33|ajole. express deposits beside the slyly even accounts serve furio +7510|Supplier#000007510|ixO6595C3ZwVRbBzPW1Pu|1|11-714-273-4551|387.76|usily final accounts. furiously regular dolphins ha +7511|Supplier#000007511|W4OiKH,gzMTq24 xGsnwVy|20|30-770-235-3547|3094.73|carefully ironic instructions use above the regularly special asymptotes. flu +7512|Supplier#000007512|pXSnLcEj dzzxyUNG|11|21-146-481-5250|3044.80|re blithely accounts. accounts haggle furiously quickly regular pinto beans. ironic, iro +7513|Supplier#000007513|RdlsM32B779IxkrX|9|19-136-395-1425|-29.40|r dependencies. fluffily final pinto beans wake furiously. +7514|Supplier#000007514|VRv6icraoDzTBpx LaNbxf|17|27-893-116-7923|9804.55|ven ideas dazzle blithely final pinto beans. slyly final account +7515|Supplier#000007515|c2ILRnap2LiYJt4y4rmsiL,u4Tcu|15|25-697-670-1343|3153.80|lyly express requests boost above the pending pinto beans. quickly unusual accounts alongsi +7516|Supplier#000007516|MXUTfa53VDJlZZz ET JU5Doou|24|34-943-403-5413|5889.63|sts throughout the slyly even instructions cajole blithely special, regula +7517|Supplier#000007517|A0jDz3SUnGe7 riLcwD4akLvDAqV8GtXm MM2r53|20|30-548-567-4623|7029.00|lly final foxes. bravely even p +7518|Supplier#000007518|JbzpdeQoBLXf,2MURMK7l,3e|0|10-480-739-8890|5480.52|gular accounts. requests cajole unusual fox +7519|Supplier#000007519|oSTDJIojIOdTWYYMp Sj1QBT|1|11-511-610-5276|6133.96|uests boost for the slyly even forges. ironic foxes atop the regular deposits +7520|Supplier#000007520|at1CTEqdKfGubHRPnxVbyFLhQ|14|24-700-592-7490|-433.20|ackages along the closely even foxes boos +7521|Supplier#000007521|XKZS,1Ns,5jFevpAk43oiGPzdvsguV |14|24-277-624-7516|-879.74|ilent theodolites. regular, special platelets haggle above the furiously fi +7522|Supplier#000007522|JjF gKOuyb1gy8cdcqql|6|16-675-888-4837|7155.75|e. silent packages wake thinly after the quickly even accounts. carefully ironic a +7523|Supplier#000007523|XvPm ScJP u4|20|30-546-972-2854|1730.64|ruthlessly pending decoys nod fl +7524|Supplier#000007524|CEZxK,uwb4GSB,ELczTZZeEzP|10|20-867-118-3156|764.08|ajole. blithely ironic pinto bea +7525|Supplier#000007525|AA4cOZUA8Cuinq1T3MpNfbLycABkpV|8|18-100-531-8345|2777.91|ular packages boost blithely blithely final packages: deposits haggle do +7526|Supplier#000007526|ec OCBW1Amcz7umPbB|13|23-196-113-9571|2358.90|ckly final requests. regular packages are carefully into the boldly express fox +7527|Supplier#000007527|Vi2GCsbNCmbhoFETM4bT5p5k YOU|10|20-481-318-6812|-84.34| regular, express packages above the furiously final asymptotes integrate blithely against the fluff +7528|Supplier#000007528|JcOP4l7dM5l YFXZ,eeRKu05kfG31|17|27-942-960-8499|-561.03|arefully unusual warhorses. slyly ironic theodolites according to the ironic packages shall have t +7529|Supplier#000007529|AzrFZw3qFTtUcw8QX1r0 X8tVh3moVymM|15|25-384-805-6049|6727.26|fily express instructions boost ruthlessl +7530|Supplier#000007530|ZkAmi98 D80jbq,z2GcN8|11|21-911-231-5040|5176.69|special instructions. instructions cajole unusual platelets. furiously express package +7531|Supplier#000007531|z F82q xzMeav4D9JMi|6|16-739-788-7770|3662.95|osits. express, ironic pinto beans sleep fluffily e +7532|Supplier#000007532|27CkgqXAFkXRoiY013ZCIQV9nD|20|30-571-776-8216|5173.16|ly regular accounts sleep furiously even accounts. slyly final dependencies a +7533|Supplier#000007533|ucjeK PHikh0uTVT|18|28-889-266-4311|-607.04|arefully unusual packages sleep carefully daring, fin +7534|Supplier#000007534|RCtYZug8JZGO|14|24-601-575-4324|3376.71|unusual accounts are blith +7535|Supplier#000007535|baYGUM1V96ULxAjkFF5 XLc6cNJ32LqDKxHVr|23|33-814-473-6588|1393.36|cial foxes. instructions h +7536|Supplier#000007536|c9HkDsVtkhjKrW|3|13-649-721-9921|8182.89|ously unusual requests. furiou +7537|Supplier#000007537|HktVUlyzFBQ4bqM4puig 76Gzy1EjahmdVGRtf7|4|14-236-947-3108|5836.20|ronic packages haggle. even dinos about the evenly stealthy accounts cajole fluffily eve +7538|Supplier#000007538|EARYNgnmahFaNRPK4cLuseJN0ppf98E1|16|26-785-980-3802|84.53|gular theodolites run furiously regular, bo +7539|Supplier#000007539|LkQVe q9XKuQwc1voX2m2|18|28-364-651-7211|-452.51|ses. special deposits cajole f +7540|Supplier#000007540|MA,sS2ik,ud48C5QR3SAGyNl|8|18-829-622-7156|4514.23|en, busy requests sleep blithely pearls. requests haggle sly +7541|Supplier#000007541|dVh8pcsand|22|32-245-142-6812|1747.00|ole quickly quickly final accounts. ruthlessly pending c +7542|Supplier#000007542|Pz IGM,Qrc5OV0 lETgInzHbIr6Uw|0|10-947-960-4456|723.60|ts cajole blithely. accounts against the pinto beans nag slyly slyly +7543|Supplier#000007543|fC3tBKvOvoCuhc8ZtU1fkwG|23|33-310-975-5260|4411.42| quickly sly foxes wake fluffily regular +7544|Supplier#000007544|W0Ga,g4,AbfqOk|16|26-348-368-1674|805.04|sits; theodolites haggle blithely slyly even foxes. quickly unusu +7545|Supplier#000007545|lRLYS1M8CXnOYJ9HITCcqk uZKPFjdmS|17|27-596-939-3829|9284.30|egular theodolites along the fluffily ironic packages use blithely duri +7546|Supplier#000007546|aFLE2fFcE 1hBWZrrZYkbShlvH|7|17-156-434-8041|4236.22|y unusual deposits haggle slyly final excuses: blithely regular asymp +7547|Supplier#000007547|FmcZf4dgulcZkMWETmVmH2OfSSu6AlwBg|5|15-947-904-6471|4174.02|ests. carefully express accounts sleep. +7548|Supplier#000007548|orY8R6 tWFsTLgAV|23|33-656-960-6126|3756.57|ix. regular, regular pearls are after the th +7549|Supplier#000007549|BeFGcALqutrV75vC1XwVxtaOv33c0QiNOr4unlX|9|19-232-712-5516|5706.22|nwind even, busy sentiments. thin ideas hang blithely blithely bold packag +7550|Supplier#000007550|sQ8,IPz6Ev0CzUyjQlRTW1vv6GOIdRQ9qTbo|6|16-634-657-7800|4507.10|cial foxes boost blithely. final foxes above the furiously fin +7551|Supplier#000007551|V4HF5GzVsdw9Hl LUMXWYzPkmofaSQy|8|18-729-196-3963|2890.26|of the even pinto beans. slyly unusual courts haggle slyly packag +7552|Supplier#000007552|yV1BjDrL0DSLzJuhSNMkpQ|12|22-747-741-3776|9555.51|its haggle along the requests. express, unusual foxes along the instructions was alongside of th +7553|Supplier#000007553|wAU2Lui w9|10|20-663-409-7956|8594.80|old, stealthy accounts are blithely. fluffily final +7554|Supplier#000007554|Huq0k qKET hByp3RcMcr|11|21-787-637-9651|9825.95|ending, final requests thrash pending, +7555|Supplier#000007555|iRfBs3APb0CLDswKdNj0,JNCbDOcXZdb97OvD|18|28-348-356-3747|-253.86|ously regular theodolites integrate furiously +7556|Supplier#000007556|iI1FclAmBLde PCl6d|23|33-974-496-5278|-748.82|yly furiously regular packages. ironic platelets cajole along the slyly slow +7557|Supplier#000007557|udLvpjNvIx9qeRNdjL1ZAO0OZNOBo6h|11|21-629-935-9941|7575.12|ally special accounts nod; f +7558|Supplier#000007558|6bAWEpC3vt8cqJ6oj v5BbqR |8|18-392-526-9580|-978.28|eodolites wake according to the stealthy, ca +7559|Supplier#000007559|Wmzx1vskciC|21|31-565-694-7012|881.45|requests: blithely even platelets sleep around the furiously e +7560|Supplier#000007560|ys4A6CGsNL K,9gNPx2MNnG|19|29-721-141-7530|6519.54| carefully ironic hockey players. blithely permanent accounts wake slyly among the clo +7561|Supplier#000007561|rMcFg2530VC|3|13-520-876-3751|912.09|e regular dependencies. blithely bo +7562|Supplier#000007562|R5hsqznicQaiAGIxq2cg2J,d9yrdoVUJRJGSdh|4|14-329-695-6722|4510.89|lyly across the unusual, regular theodolites. even packages sublate furiously? stealthily regu +7563|Supplier#000007563|53kPvJgMXys sH4pWy2PI9qy8EmD|16|26-853-236-9890|575.90|ly ironic theodolites are blithely after the blithely regular packages. brave packages dete +7564|Supplier#000007564|hCfkBoEdgehPDK3LwO7jTYp|2|12-965-161-3745|8103.58|ithely even instructions. silent, bold deposits boost slyly a +7565|Supplier#000007565|Dksx,qjHkCWqpZFxdF5D4 |11|21-709-524-7729|3872.78|mptotes are blithely. slyly express pin +7566|Supplier#000007566|SBuR3PRFvnBZuKYfA5L051M9k6JgCgs8SnRkt8f|18|28-605-996-4940|882.86|instructions affix slyly express deposit +7567|Supplier#000007567|7W4k2qEVoBkRehprGliXRSYVOQEh|10|20-377-181-7435|8111.40|gular foxes. silent attainments boost furiousl +7568|Supplier#000007568|g8Q, 3C6pR8MSmV0t3fcP0Oj5,rmUhwrbO|10|20-668-273-9709|3341.44|g to the carefully silent dependencies. ironic dependencies detect about the furiously bol +7569|Supplier#000007569|K1xgmI4bhbgpZNaj8lvg7kPAjav|16|26-193-398-8891|2483.36|furiously carefully ironic de +7570|Supplier#000007570|TINQf1CEpSZA4ZLYteDXYOpKCqBpzh mO|19|29-399-692-5807|1744.12|y final pinto beans sleep. fluf +7571|Supplier#000007571|KaJ0IHl4soF4qSNmjVvPPuI7DITQT|17|27-285-338-2698|7393.40|ng packages. theodolites dazzle fluffil +7572|Supplier#000007572|i6AATMhkMkb244zmrl,,6GY|22|32-609-773-5192|2976.77| should have to haggle. care +7573|Supplier#000007573|C2xSsp4M5SFGQo|16|26-754-576-8528|503.29|inst the regular accounts. carefully pending pear +7574|Supplier#000007574|u3JmTCGgom5RD4YLhf2dpDVU6783k|0|10-375-719-8227|8665.98|ntegrate. ideas hinder blithely after the ironic courts. fluffily final deposits nag. slyly p +7575|Supplier#000007575| jyBL3LXVnChHCUo82sBc|6|16-929-131-1821|7182.26|unts. finally bold pinto beans boost across +7576|Supplier#000007576|6KzBGPBzNqnl ixFx11VZxKSqwe5zxgv MxHEc|12|22-915-376-2026|7291.73|ke furiously furiously final requests. even depo +7577|Supplier#000007577|1UtrIecA68zVCgTtI,xlz3Txi3,E7ZGAE|13|23-275-725-5449|-908.98|y even theodolites. quickly even requests integra +7578|Supplier#000007578|JILaAHgOZHwDw7wOBD|19|29-432-333-1258|-258.71|ns haggle slyly according to the +7579|Supplier#000007579|BBT4MfzsHUeGDY6eME4 |20|30-574-243-2676|5732.84|n pinto beans. pinto beans serve alongside of the ironic dolp +7580|Supplier#000007580|xWVlA8twBv7qYkUijhaK8Rr5N2fwjMMP8RTc|17|27-613-651-1393|915.26|lly slyly ironic accounts. slyly final de +7581|Supplier#000007581|v1X2SeuCw 9RKtM|8|18-564-519-1085|2792.89|ntegrate slyly. accounts promise carefully along the b +7582|Supplier#000007582|XhM0z6DsdvpXzo4Xe67e|6|16-246-633-4829|1152.96|nticingly even pinto beans. furiously ironic packages above the thinly +7583|Supplier#000007583|tY1A0g5IAXMAYxhe45QIhjm5e|23|33-127-114-8415|4693.14|quests. carefully final requests use +7584|Supplier#000007584|huGf5x6lxo8OwHoXAephYfb7TH|2|12-954-925-2810|5557.42|e fluffily alongside of the special ideas +7585|Supplier#000007585|6kpn41RCTreOmCNM2ykYoc4TqazalUr9m5AXm1Wu|24|34-990-303-7174|562.83|ggle express dolphins! express requests haggle blithe +7586|Supplier#000007586|Pq1 SpXCLkeIw9ISg2Hs5jo6L8ySmrInx|18|28-165-802-8314|8441.51| quickly against the carefully even foxes. slyly expr +7587|Supplier#000007587|bjs0ZkBH7B6p0|4|14-637-776-3429|7559.07|e quickly final pinto beans sleep quickly regular packages. carefully regular accounts serve +7588|Supplier#000007588|nl9tvg87Zu0u629XtHL6QZAItY9lhLVh|2|12-593-727-5767|4264.44|ing, express requests haggle carefully regular account +7589|Supplier#000007589|GUTp8apKI6hoOTlZ1DXSZHqPhKGjgskV0xZRjf|3|13-839-534-1103|5468.02|side of the carefully ironic requests. stealthy accounts cajole caref +7590|Supplier#000007590|2Wd3NCI8YFjJZVskPHI4 RrkY9vju1|24|34-605-148-4420|4975.90|thely. quickly unusual accounts th +7591|Supplier#000007591|UZ0XaTY1BmcDuetGs8d5ACvAWwsiiO8VSE7,jYo5|22|32-433-724-1715|5955.34|ole. slyly blithe pinto beans hang +7592|Supplier#000007592|yXDhufXHnFunWqRlLd|6|16-716-881-4288|358.49|cial packages wake after the slyly regular theodolites. fluffily slow pinto beans a +7593|Supplier#000007593|K7CDCFlvUpsVJk8ypLGXWD0O6BD|13|23-934-730-2446|6051.97|r accounts. quickly unusual platelets believe; bold, bol +7594|Supplier#000007594|,MDKhwTsZPA1SiXYj,KXdJeru3j6S|13|23-516-704-8928|7456.45|er the furiously final requests. +7595|Supplier#000007595|dION7CYxDKcGUJXDlXG0cReNjjGPoHmylMFSjIe|13|23-824-887-9329|6637.73|n asymptotes wake above the de +7596|Supplier#000007596|hH6,rbnfRopUcXb8BKuJXq4Ab1nXGOnJ|2|12-281-883-7492|7304.24|de of the ironic ideas impress +7597|Supplier#000007597|DyGVYOJf0YXcazbYP9|6|16-802-291-9574|459.58|g requests sleep quickly alon +7598|Supplier#000007598|YclVmTiWlL1gp6eXcUgM2,MtbsVtMUj2t|1|11-858-435-3889|61.56|s around the furiously final deposits cajo +7599|Supplier#000007599|HZFlVd9P9hPkrsDoOQb1tv2|13|23-515-832-7179|9499.75|platelets haggle slyly regular requests. carefully special packages +7600|Supplier#000007600|6F0nOKOIIv7REou0wYk|6|16-958-947-4987|6414.34|. slyly fluffy accounts wake even warhorses. blithe +7601|Supplier#000007601|RTiDfo6aTuo1d IwsxnZak3gjjNwms9fQWr|20|30-600-354-6597|3841.92|unusual asymptotes. always express ideas boost about the slyly bold ideas. requests cajole carefully +7602|Supplier#000007602|JZXAzMCUL3,QQGuLGL8|20|30-504-663-4024|-813.38|inal packages poach busil +7603|Supplier#000007603|afAJNTKJPBb rwxa6SBbkY|5|15-799-960-1671|431.76|ly pending requests sleep +7604|Supplier#000007604|,O,SnMfEmNTLFEo6J5tBgQa9xtY WJn|18|28-297-318-3423|5523.30|are bold requests. pending, ironic packages wake around the deposi +7605|Supplier#000007605|0aWoD lZmEwNfcg8cj60u2hp|7|17-497-385-9346|-906.23|bold deposits. bravely sp +7606|Supplier#000007606|WvQk,AYhITiI,7WZ7cGLmb0Yt TxU|18|28-173-974-5741|1946.17|y even theodolites. closely ironic somas nag quickly. quickly special dependencies ar +7607|Supplier#000007607|7bILPe40CyXTg2LxLlxylFP|0|10-814-344-9836|4160.37|t according to the quickly perma +7608|Supplier#000007608|EUzXTeHOSCKruCLNL2EedQmkXoxIS5oJljEM|9|19-754-784-9957|5302.54|regular requests across the blithely unusual deposits haggle quickly +7609|Supplier#000007609|g0BgPne9HMFspF|17|27-519-129-1933|6757.33|yly ironic braids. blithely silent dependencies around the furiously bold accounts nag sl +7610|Supplier#000007610|6rHKTuNXVP dEM NIAYDeRo|7|17-916-883-5873|4534.22|riously. ironic tithes wake asymptotes. pending th +7611|Supplier#000007611|YPNYx 1KJF0B0,mSQUPXIPH1EjB3yYe,|19|29-260-804-8693|7675.66|se carefully final packages. furiously special packages haggle blithely carefully bol +7612|Supplier#000007612|5NLvvoXI3eSo6mNxSEC9g8Jz|17|27-694-727-6479|5928.13|ic pinto beans across the ironic requests maintain quickly silent ex +7613|Supplier#000007613|FQPIblYPeKqOkQuwE4F5PqHFGGwNRoVGwasy,Bdc|24|34-305-830-8861|5061.85|into beans. carefully final requests use fu +7614|Supplier#000007614|LIoiHPKlog0vYRCb4lciK6U|8|18-926-882-2449|9521.69|latelets along the carefully even requests cajole blithely +7615|Supplier#000007615|nhgM5 SXC95svxqbo57h|1|11-988-945-1706|1135.32|uests? unusual requests wake according to the even grouches. bold requests haggle +7616|Supplier#000007616|R IovIqzDi3,QHnaqZk1xS4hGAgelhP4yj|3|13-830-310-3200|5353.41|ayers cajole carefully carefully even asymptotes? +7617|Supplier#000007617|eX9oIlq qy74YfKC ,H6W4|11|21-496-746-4893|3758.12|y. bold deposits haggle bravely special requests. final packages solve quickly slyly spe +7618|Supplier#000007618|lLb8,1p07ZseCSxYpYt|22|32-642-503-8109|9652.21|equests are blithely; doggedly unusual packages haggle furiously about +7619|Supplier#000007619|LF6stms3r,fk8Ap1tRFABWX1oaB4DD9s,eO|20|30-561-375-9120|5845.43|bove the furiously ironic accounts. bold pinto beans slee +7620|Supplier#000007620|BE1BA,3hW StHY7gwhDq6YFc6MTd ERt|15|25-170-118-1298|1165.61|ic platelets haggle furiously dolphins. regular accounts haggle quickly +7621|Supplier#000007621|MAqy4THMPElQ|21|31-227-850-3395|8390.17|rts sleep carefully regular multipliers. thinly regula +7622|Supplier#000007622|eOp0qBjoSvXqZf8Ucghr5ZAQfgL|2|12-685-832-3085|2605.81|usly bold foxes sleep among the blithely re +7623|Supplier#000007623|d9cXb8a3pGjdlc,DOkl,I8,aUr|0|10-460-945-9319|2758.39|s cajole furiously slyly pending deposits. regular, ironic requests are slyly. regula +7624|Supplier#000007624|budqY6iH7JGY0ZAPM2K|4|14-455-556-2580|4151.90| the carefully sly excuses. carefully spec +7625|Supplier#000007625|AJYIKaliIfJeJ|23|33-144-323-8280|5918.27|at the sly, ironic foxes. regular, ironic accounts sleep carefully. pending pinto b +7626|Supplier#000007626|a,l0vqnWdjZgZwJWAp,krdNplMivgB|20|30-743-483-8978|4378.34|ld Tiresias cajole slyly requests. final pin +7627|Supplier#000007627|CiBhgStvuY3eopiZkdC63LWs|8|18-981-431-2170|5779.72|refully even, silent packages. +7628|Supplier#000007628|p1a0OAMN7JA3SB0c 84tE3mADYjl3CH1|0|10-717-400-6276|7735.74| furiously unusual excuses wake waters. ideas cajole caref +7629|Supplier#000007629|Gxgg8RLTzAnWvf|24|34-500-770-6458|3715.05|ites play slyly. slyly bold accounts according to the fluffily fi +7630|Supplier#000007630|a6VgzRIz8IVlpryoN4s8Ea|9|19-754-193-9017|889.46|refully slyly even packages. ironic asymptotes cajole carefully along the blithely regu +7631|Supplier#000007631|eSImoUDXAQp|0|10-860-346-4466|9040.85| requests. furiously final foxes according to the even accounts detect alo +7632|Supplier#000007632|I8LD6sDUTS8YCB,MWy6TnIV,ZGIr|21|31-978-847-2307|4695.32|ly close instructions use carefully boldly quiet requests. blithely bold packages play slyly +7633|Supplier#000007633| rsl6lh6tEzOKpo6r8ph3A1OuqK zEOBs5Y7l|11|21-473-187-3309|9243.92|g ideas kindle slyly? carefully ironic deposits boost furiously special asymptotes. furiousl +7634|Supplier#000007634|TkmKScI3gO|10|20-699-145-4483|7653.17|ous packages. blithely final accounts sublate blithely bold instructions. bold package +7635|Supplier#000007635|GgAHVl1YHhPiBcPvYlsos3|17|27-117-312-4916|1925.18| the even, special packages. special, final accounts sleep furiously according to +7636|Supplier#000007636|AAFlIZIKyklodxLKfI0x50ZE0O|7|17-630-607-9411|3443.62|quickly ironic packages boost quickly blithely +7637|Supplier#000007637|,T0w7vPN9DG4N0 vWAt90qBj|10|20-696-827-6247|3860.94|regular, express dolphins wak +7638|Supplier#000007638|eQfz2uMuiR4BefKLZF18rX2ZJeL6|6|16-997-248-3176|-179.15|ss accounts. furiously eve +7639|Supplier#000007639|jyUH89VrU62pYflx4S2iQn1cIITnXKvgG|15|25-981-204-7774|1013.68|es doubt quickly packages. furiously pending packag +7640|Supplier#000007640|5q8,fgMPTbyNIESBILSGTHkp0lwgbV|17|27-921-220-8313|6937.76|uffily regular foxes after the bold requests sleep across the packages. even theodo +7641|Supplier#000007641|XDnXtprXzolM|17|27-166-455-4918|904.96|ironic, unusual accounts integrate furiously according to the carefully +7642|Supplier#000007642|eg,Zn9STku6kl|13|23-978-491-9976|5671.03|ages. furiously ironic instructions breach blithely above the idly ironic requests. accounts +7643|Supplier#000007643|dJhovIhUlwbXkPlb|16|26-179-585-3150|7003.29|e furiously silent accounts. bold, bold packag +7644|Supplier#000007644|nosGdNo,Eb67350eNSOIWh|8|18-910-115-7443|2372.03|deposits. furiously ironic instructions against the slyly pending forges haggle furiously final +7645|Supplier#000007645|JtOvr9g04Yu|7|17-858-909-8241|743.38|press requests along the ironic, regular sauternes play quickly throug +7646|Supplier#000007646|hcI56 dJu4yGCUv DlzkPHd6cVtK|2|12-321-253-3386|6990.85| ironic requests use about the regular deposits? slowly regular a +7647|Supplier#000007647|QLqxLzecWn7SZ7Ub1Vcv7E6oRypl|19|29-696-857-6510|2202.37|as cajole blithely express requests. slyly final foxes use furiously bold packa +7648|Supplier#000007648|xijpSa3xolGv2fpo28oXUxABzv5B9FM|18|28-466-670-5238|9170.72|refully express requests after the pinto beans cajole above the s +7649|Supplier#000007649|ifREImqBy8|9|19-179-565-3935|9380.82|old packages about the ideas sleep doggedly unusual, special packages. evenly ironic pack +7650|Supplier#000007650|9D 0JiGBiod|18|28-266-641-8300|9482.39|pths according to the slyly pending dependencies af +7651|Supplier#000007651|HHs2bSBnZKmhrEzXF Tj4o6B|23|33-761-193-8351|603.73| even asymptotes wake blithely after the pending instructi +7652|Supplier#000007652|JUpb3Yu0,CCLYnjVN rVWhPrvaqdZ1yQbOSVymEy|23|33-160-417-2646|2829.08|fix finally furiously brave requests. blithely bold deposit +7653|Supplier#000007653|0CyERxqI,m,WcKPj6sJnFhRk xLjUudBDO|22|32-633-287-3050|4320.36|ic accounts sleep carefully sile +7654|Supplier#000007654|GmS Z,Kw43pcnPlIyQREHBEB,0H|18|28-916-271-7928|-791.63|ly special dolphins. final packages use bl +7655|Supplier#000007655|puv4FgaAA1IK 1PSUFo9uF|2|12-864-706-9176|2997.36|deposits affix. unusual, special asymptotes cajole bold dolphins. +7656|Supplier#000007656|EM vHZtCNnjNIqIaNLI2ny1B,c5B|20|30-407-816-8421|5681.66|uriously even foxes poach blithely along the silent +7657|Supplier#000007657|z91wl9KLdbdTEUaHIy Fze9oy cavNW|4|14-694-576-1183|9756.30|ts. asymptotes cajole by the fluffily express +7658|Supplier#000007658|QFP,X9ZqwsG Oa|9|19-402-271-4966|6275.64|bout the ironic deposits. fluffily final courts integrate furiously between the regular, unusual a +7659|Supplier#000007659|tD9jb Gs6iupLlr5aKoJC,X7yvtzymyvg|9|19-107-670-3911|4649.86|packages according to the express requests cajole slyly carefully final d +7660|Supplier#000007660|q0w4EwU7MseVbhDR1A|12|22-819-312-9279|4161.31|ss packages shall have to use ironic, final accounts. quietly p +7661|Supplier#000007661|OOWmCewsPSL|7|17-325-301-9747|-870.66|posits use above the fluffily ironic ideas. regul +7662|Supplier#000007662|VGk9AgllnOQ2,yPPk,pfHXJ4NgF2|20|30-162-843-3198|5734.03|arefully express deposits. carefully +7663|Supplier#000007663|LVDAB8vrsB9R7RO sq6a40,aqy9BBVm|23|33-823-738-5604|9840.97|ges. deposits wake ironic accounts. furiously regular dolphins haggle furiously! daring, regular id +7664|Supplier#000007664|vxDpY eMwmQVAvSzvUu9vL9|14|24-817-684-7844|-681.55|st the regular dependencies. furiously regular +7665|Supplier#000007665|Jk1i, VCplzH21|7|17-191-303-3379|5146.31|he carefully stealthy platelets. slyly ironic foxes use. blit +7666|Supplier#000007666|qSwWpsWNOrPGl3,Ly O5SOv1Fs3|10|20-135-256-9102|-913.18|ly pending foxes sleep quickly slyly unusual packages! ruthless accounts hang according to the +7667|Supplier#000007667|rJasq6 x4rVnc|7|17-804-110-6206|6026.39|. special, regular foxes wake blithely pending instructions. ironic asymptotes wake. slyly +7668|Supplier#000007668|3Ux8croaXj40iasWgdYf6gjBieSz98F7RD6CIgsg|20|30-790-273-7652|6520.36|until the blithely express packages. even requests wake about the +7669|Supplier#000007669|YEpR82xmb6BtV5fQ1LWpVOH1q9OKu|23|33-736-794-5796|3717.15|ts. regular pearls haggle quickly against the carefully express deposits. regular, bold +7670|Supplier#000007670|2Ucw1ITgpK4|19|29-412-494-6534|3120.00|thely furiously regular foxes. furiously regular dependencies haggle quickly pend +7671|Supplier#000007671|IZ4jEzEG0hX5F6aHtU|6|16-297-940-9391|9893.40|s cajole. slyly special asymptotes across the blithely final accounts wake slyly bold, iron +7672|Supplier#000007672|GBNVlTf8,1VWZeEmazMtU|15|25-242-730-5044|409.69|y unusual deposits. regular dolphins run along the bold platelets. bold p +7673|Supplier#000007673|DBlWbg eBbXRdWzZclPj9j,GjA5VW4oZc|7|17-941-588-4206|8527.13|iously bold pinto beans haggle carefully re +7674|Supplier#000007674|M3PcJmk8WBZEl5NXTYUn0,KGSY7962Tx36r7FM8j|8|18-275-808-6179|5494.96|quests cajole after the carefully pending accounts. slyly unusual deposits +7675|Supplier#000007675|LMt0fLfNb3c62Tb5bzJCYoKmMQp,Rai|3|13-572-777-5847|6862.65|regular packages. furiously final dolphins haggle furiously upon the foxes. platelets according +7676|Supplier#000007676|tOm,1t5wY dbSPigBt5TewAvyIXHMktM|0|10-459-599-8523|3677.35|sits are about the ironic theodolites. blithely ironic decoys wake around +7677|Supplier#000007677|OoTYQdxQyd7NukSaSRv|21|31-794-496-1198|5282.35|lyly even frets are furiously. daring, even deposits haggle aga +7678|Supplier#000007678|mcyyz2Gm12MnVDM6 PTJ 0NOb |9|19-923-359-9505|8498.35| pinto beans-- fluffily quick forges wake above the furious +7679|Supplier#000007679|iof985EIANf1XPdezBY8k|11|21-427-242-2928|5843.92|final ideas. express instructions haggle carefully. ironic, final instructions sleep c +7680|Supplier#000007680|PfdxT3eMtorxsDMGoaqPsHkt2w|18|28-769-698-7237|8826.11|rts detect. carefully regular requests cajole quickly. carefully regular pac +7681|Supplier#000007681|akGJAVkSL4xefhXaMFQitshNbgl33,wn,|4|14-516-840-8296|3572.74|ke around the express, even accounts: even pinto beans against the bold asymptotes cajole ev +7682|Supplier#000007682|wxyhu1i81gTuKAoZKhovS,atwP37OcmwR22jN|6|16-417-532-4728|6964.89|ending packages. quickly unusual deposits are carefully around +7683|Supplier#000007683|sAVwu4kVTl3gEpkpFOvq0WYSmlAdB3TM3U|7|17-808-119-1143|6015.70|xpress accounts use carefully ironic pack +7684|Supplier#000007684|jr7FgaRd4 v,5gfU1xlbpLOk,zLBzMIpmnYJK|0|10-821-183-1969|2937.96|iously ironic instruction +7685|Supplier#000007685|4OCzF9LfDLYGF0pcgW9Z|1|11-251-650-9168|8227.98| regular accounts breach. carefully express dependenci +7686|Supplier#000007686|kD0dozapw04G|20|30-912-212-6366|4695.11|nic ideas cajole. slyly final sentiments c +7687|Supplier#000007687|rDVqhUFbrOX|8|18-925-876-6611|5733.86|depths hang slyly: permanently final packages ca +7688|Supplier#000007688|d6lpYBVfO6wowoX6P1w7ssjXGWjgm|23|33-598-549-5765|7007.49| accounts haggle alongside of the regular pinto beans. blithely regular patterns wa +7689|Supplier#000007689|lHQzLQr9Ej0tCHqbrIHVynT|9|19-944-980-4845|6983.01|ckly ironic deposits sleep carefully slyly bold foxes. carefully silent ideas print slyly sl +7690|Supplier#000007690|3mxdk0kzKF6tmyIvEKRnfROqExG A,Zyt3 ab7|18|28-573-152-9604|2528.65|y pending asymptotes against the bold deposits haggle pending instructions. ironic dolphins prin +7691|Supplier#000007691|,cdI7V0bBLssomToIjIYDMJi5X6DUULBOjc|5|15-798-957-9943|4017.10| platelets. blithely regular theodolites after the always even dinos sleep except the slyly iro +7692|Supplier#000007692|GjRF,y0Drd|14|24-830-130-4500|6377.45|nding platelets wake blithely special, even multipliers. carefully pend +7693|Supplier#000007693|imxeuo8vnSW4gLT|1|11-898-179-8103|-120.33| foxes along the ironic deposits h +7694|Supplier#000007694|vvblmqLSXjvx7mm9WBhaZ6M4muFpXYM t|9|19-182-788-7695|-456.83|le furiously furiously unusual depo +7695|Supplier#000007695|p5djXFw2hnJuaFW|24|34-905-937-5355|8901.31|y pending deposits. ironic, +7696|Supplier#000007696|GprwYOyLWe7nZ8KHcBLthIa,4jiOk|4|14-474-176-5939|2384.75|sleep furiously about the slyly final pinto beans. +7697|Supplier#000007697|4lzQlXRJgy 9s1oDSf7yMDlotyh1qy1Wx4vLXd|6|16-123-112-2857|5389.56|its haggle fluffily. final packages +7698|Supplier#000007698|JROKl3gzqZiY8vJKqiEOjjFzv2wLUd93DKVq|1|11-577-530-7950|6284.49| deposits sleep. requests according +7699|Supplier#000007699|fnbn5zDZD9wc6oD3SW2g9fMQNwEZk6T|1|11-168-845-2149|-566.62|longside of the warhorses. silent courts haggle across the bold, regular deposits. quickly f +7700|Supplier#000007700|d4JPROrfIeY8bL|6|16-435-671-9407|675.86|final foxes. slyly regular ideas along the ironic sentiments cajole iro +7701|Supplier#000007701|J7,8suS2TWxrNTVDHn3noD|1|11-767-398-7074|5086.74| dolphins are fluffily quickly ironic excuses. care +7702|Supplier#000007702|mzgoSrhKy3rd|9|19-394-466-7780|9871.61|s. furiously final instructions since the ironic theodolites eat final, bold theodoli +7703|Supplier#000007703|odtP Txyybda|24|34-595-906-4948|8251.26|ress deposits. special asymptotes wake blithely instructions. packa +7704|Supplier#000007704|mUcN2x,yodhb|24|34-319-784-1592|-92.53|e. carefully final deposits sleep. bold, special courts about the pinto beans doze furiously flu +7705|Supplier#000007705|w0I035jqGIat|17|27-315-741-1666|4751.69|xes haggle blithely according to the fu +7706|Supplier#000007706|BlHq75VoMNCoU380SGiS9fTWbGpeI|23|33-481-218-6643|-379.71|lly pending packages doze across the evenly final +7707|Supplier#000007707|x0q 8pDWEg7sFFOGlUJ8x|24|34-312-340-7340|7200.70| unusual somas sleep blithely ironic depo +7708|Supplier#000007708|TXCbF4BwDio7F7cU1O4KJU|8|18-228-676-1304|9141.05|cies believe. even accounts integrate slyly across the final p +7709|Supplier#000007709|SmHoGfrBDxzZR2I9K|0|10-422-603-5883|5684.85| silent accounts detect carefully furiously regular dependencie +7710|Supplier#000007710|zFDRlSRWLZ,y3S Jmn4RnN|8|18-428-344-2569|8357.27|ideas. pending, unusual packages use bravely. quickly bold foxes sleep furiously silent, regul +7711|Supplier#000007711|ywvKl2FmWFH4QV8Pdw7hxN9t9V4IQ65Bxg1lFkB|8|18-432-745-4227|-324.75|eodolites nag slyly above the even, regular sheaves. carefully even theodolites sub +7712|Supplier#000007712|DyTQD 3ajuOtHQTpI4LsWSF kSd2SE6U4COgYHQ|21|31-394-117-4728|6934.32| carefully even foxes boost ab +7713|Supplier#000007713|8jILJdOR7UPc08ZRC HC3 q5FTyK2Pj4IjXIe|21|31-373-803-1471|9132.07|s. express requests grow fluffily requests. instructions use furiously. unusual deposits gr +7714|Supplier#000007714|KsKIPwzy7oojl3 jaifCvdbFuc0,AlkT84Fcv|8|18-793-375-2899|8738.07|sts sleep across the fluffily regul +7715|Supplier#000007715|gZHd7Yzbtv7yb7DYCCAQPJH8FRHTqi6T4w|21|31-668-501-2822|8745.78|longside of the ironic, express foxes impress blithely ironic accounts. +7716|Supplier#000007716|qJM57NQd5ZzpBQb18WRDQ|13|23-520-894-6759|6361.94|he fluffily express theodolites: Tiresias p +7717|Supplier#000007717|jXkaR6a 1imfl|14|24-225-157-2216|3231.07|ses breach slyly blithely pending packages. final, ironic packages mold slyly. ironic e +7718|Supplier#000007718|3tad1IVdPCxaLN1ctl6SL xHw2UT cJi7o|16|26-612-724-1360|6595.23|ven requests nag carefully furiously +7719|Supplier#000007719|eX8YGrnYR4PRdIXXaQh|4|14-999-658-6468|8078.28|y pending requests. carefully silent requests are furiously pending foxes. quickly final p +7720|Supplier#000007720|ERjXnwOJ1vSklF,yOm66|4|14-733-939-9211|7632.13| against the even packages. slyly special requests sleep. regular f +7721|Supplier#000007721|TP8cl03NxBsYJtMg|22|32-257-333-8840|189.81|excuses across the evenly ironic accounts affix slyly blithely +7722|Supplier#000007722|6YZe90,pRYjhIxqq4mS9vlr4Am5 BInGPdF|10|20-165-286-1721|9095.81|thely ruthless requests print according to the sometimes enticing theodol +7723|Supplier#000007723|3DxoZeApBqwKU7qOJCch4OLl9jKK1NW9IvDsM5bu|7|17-728-781-3387|4380.56| sublate sometimes. packages are. blithely final waters cajole. slyly ironic dugouts nag +7724|Supplier#000007724|6IpJsjVfwtw9j|11|21-313-729-9222|-931.66|he blithely ironic excuses. slyly pending requests along the depos +7725|Supplier#000007725|XOi C9s28HqjETNXQBB|6|16-302-143-6532|3554.29|s haggle along the furiously bold deposits. ironic pinto beans about the slyly ironic pin +7726|Supplier#000007726|y4EoxCVx9ewM EDjn0TYWRzCZQUEU6SkdGe1ftT|13|23-759-433-2430|4938.57|elets boost fluffily. courts wake quickly. blithely ironic pinto beans wake idly +7727|Supplier#000007727|p6rs44lEg4AfmlK4oi4fLqEQJLcIfaGvGBy|0|10-222-775-2979|5563.25| haggle blithely carefully fluffy ideas. pains haggle carefully. thin, silent d +7728|Supplier#000007728|TXH8rd7zkawQPgqNlazTUFrLpZcpNYgEsTX4viFb|1|11-478-665-9769|-448.00|theodolites must have to hag +7729|Supplier#000007729|UmPNWP6AjuhUsQCvF,R5CWG8I5oN84XadAvLc|24|34-777-718-1562|3454.19|boost furiously ironic asymptotes. regular accounts wake carefully about the +7730|Supplier#000007730|Wfzhuu6C,iEggQZBX11jT|19|29-385-237-4982|5612.21|r deposits about the carefully final deposits hinder careful +7731|Supplier#000007731|f,iNRXApoi,hTD5nXRjNDsy1Yi6bYuTprIie|22|32-680-537-8379|8413.72|ts are carefully blithely +7732|Supplier#000007732|QsUb05sYrP2amzq63xfTs9MKNTX ju0hIcC8 a|19|29-294-874-5655|2461.34| bold pinto beans cajole quickly after the foxes. quickly regular +7733|Supplier#000007733|uaSH7NosamEsJ gHMxxs2hZHDrPEjVNi5b,|1|11-739-165-5524|5725.93|s. deposits sleep about the even frets. daringly regular platelets could have to haggle slyly. +7734|Supplier#000007734|ZLpKKs8an9GYEAw|6|16-828-216-9755|5756.61|pending theodolites. silent packages haggle slyly along the slyly regular deposits. depo +7735|Supplier#000007735|Fss7BcxB,HlKksNHkmaHP2gHhH6hEWA0Mai|5|15-174-432-4420|4334.32|telets wake after the slyly pending ideas. blithely express tithes haggle furiously ironic packa +7736|Supplier#000007736|BcMatI7xYRA G4lv|1|11-464-772-6240|2940.75|ithely. slyly ironic platelets nag fluffily c +7737|Supplier#000007737|V2Hc 1hH206n R2eGF68 OSLe|17|27-217-636-8480|239.79|nusual asymptotes are furiously carefully speci +7738|Supplier#000007738|vq4STgP6D50u1mC8sFr|5|15-278-716-8834|2063.21|. slyly ironic excuses above the fluffily regular +7739|Supplier#000007739|sJhH0TQQyGbrmEaXPQ,LOpwgdG|12|22-952-157-3033|2296.67|s use quickly final, bold deposits. slyly ironic requests along the fu +7740|Supplier#000007740|pRf1UWgkERuvA8sqNCIiEEyGajtxo|8|18-282-428-3507|-823.51|the express foxes. silent, pending theodolites boost slyly slyly final pinto beans: blithely e +7741|Supplier#000007741|kQDmLOO5Y8R|5|15-487-996-7263|6264.76|beans are around the bold theodolites. furiously unus +7742|Supplier#000007742|LUJhL0c,O0oiBmkN,vwVVk NHEwGxjsg5EM|23|33-854-351-4834|8860.16|o beans detect slyly regular, final accounts. +7743|Supplier#000007743|u1cZhSek7yVc7iKLIbxYSfpb|15|25-918-418-1752|3828.87|lyly final theodolites according to the slyl +7744|Supplier#000007744|o koI8Mp1em5NliKRnjm6rAtUgSIe8mo6J|15|25-867-154-4522|-134.61|s. fluffily special asymptotes use fluffily final instructions. furiously +7745|Supplier#000007745|jVuyRb0vzpQU yQRSd9bup2qxxmIGX|11|21-521-889-2916|1457.18|s nag after the carefully e +7746|Supplier#000007746|x0y0FHENyMa4xzCYG90wfrNK7s|7|17-795-266-6043|2761.07|ts. blithely pending deposits sleep +7747|Supplier#000007747|2monYxu,U6,R cCd wz|14|24-911-546-3505|5204.98|lly. quickly special foxes sleep slyly ironic ideas. furiously special deposits after t +7748|Supplier#000007748|FJL7Pf,GdyKWCqTNwQHnOnNo,yd5k65yw 3|19|29-535-184-2277|6061.05|requests cajole furiously across the ideas. bold decoys believe quickl +7749|Supplier#000007749|yS8 YrUHktkqqqCL7Od9S|8|18-225-478-7489|6085.95|sts haggle blithely. silent dolphins cajole slyly carefully ironic requests. sl +7750|Supplier#000007750|wtDhjsvnoWBD01XH7ydI|18|28-680-484-7044|6662.89|long the furiously unusual accounts. regular courts cajole fluffi +7751|Supplier#000007751|k3eAPMx7dqlYNOF2C4b8p8yCjTZ5PN|10|20-990-606-7343|1295.46|ntain blithely regular warhorses. ironic pinto beans wake slyly instructions. furiously express +7752|Supplier#000007752|Rz1JMCsGrIGvAJ,FkwTy ydL KwH|2|12-936-258-6650|1693.09|refully silent pinto beans wake alongside of the fluffily special packages. carefully bold inst +7753|Supplier#000007753|x5 roiz0vYoIN9F8LIzAjLW31|12|22-394-329-1153|7751.04|nts boost slyly ironic foxes. requests boo +7754|Supplier#000007754|5B,Btw C6Gl|16|26-941-591-5320|7698.36|ronic dolphins are fluffily furiously regular pinto beans. ironic accounts haggle ironic packages. c +7755|Supplier#000007755|uS2xiG6MSzd9c0aR21QSRufA3YVF4E9|22|32-138-467-4225|2402.75|g the slyly special pinto beans +7756|Supplier#000007756|I1EnLnLYmgi6AOZerijr|19|29-860-205-8019|1.38|above the quietly final ideas +7757|Supplier#000007757|hpa7LMuLVDxBuhuCrhW,DuEuBJyBuVqfArV|16|26-212-263-4961|-875.09| furiously regular packages. regularly express requests haggle doggedly final decoys. +7758|Supplier#000007758|Tp78 E4pgf,6gDpGWoupbatAWm2oc|7|17-640-600-3116|8227.21|ly regular packages. fluffily express packages acro +7759|Supplier#000007759|QN KKYqvCRDNsHXjsLXX1TklSRX1t1i5dGJsmAT|5|15-656-197-6064|2780.91|es. ironic, final requests boost. furiously regular mu +7760|Supplier#000007760|JsPE18PvcdFTK|3|13-639-678-5341|8808.10|nd the final, final accounts sleep fluffily +7761|Supplier#000007761|Edga,Z4HCtY|11|21-114-854-4871|8732.60|ording to the final, regular pinto beans believe blithely carefully bold courts. carefully specia +7762|Supplier#000007762|MBTZAvob3ACY9yAbjfBHB2MW|16|26-753-591-3947|4904.13|ending deposits wake blithely. carefully ironic accounts integrat +7763|Supplier#000007763|2GMsRD6rTZ6LDljKoF0ng72WJrkWCgYpHaA|15|25-448-307-2887|9703.40|hely express requests are ironic realms. ironic +7764|Supplier#000007764|,ZHHIzEdpRlfnfnTXRv30qFnzhWg1rw|17|27-102-984-2540|8091.25|even packages. furiously regular theodolites will have to aff +7765|Supplier#000007765| qjlJI,ztseRFnn|11|21-737-931-3023|5620.77|es after the dependencies cajole flu +7766|Supplier#000007766|yzh0KSi92Mx7QmFq|24|34-815-966-8428|9356.73|eans-- furiously special platelets are slyly. carefully +7767|Supplier#000007767|IenQ,N7kCm 66lJHO QA9|8|18-150-170-5440|695.45|es cajole. deposits run after the blithely special as +7768|Supplier#000007768|8FDJ DvbSiN sGowNWsUMw8p|24|34-932-957-7741|4370.04|ly final packages across the regular d +7769|Supplier#000007769|rsimdze 5o9P Ht7xS|19|29-964-424-9649|9208.70|lites was quickly above the furiously ironic requests. slyly even foxes against the blithely bold +7770|Supplier#000007770| KYJF1DiojM,63gvVhywmy|18|28-511-103-4423|9667.00|onic packages alongside of the instructions sleep about the quickly final +7771|Supplier#000007771|h6q24mvxjkMLH,iyST3mHGYanLooc0my9oOx|2|12-429-403-7801|1338.46|even, ironic accounts are +7772|Supplier#000007772|AiC5YAH,gdu0i7|23|33-152-491-1126|9408.65|nag against the final requests. furiously unusual packages cajole blit +7773|Supplier#000007773|t0RW0,VHzz3XPEPgt2dkVVd0H1EA7I|2|12-674-725-6080|4273.41|furiously special packages. final asymptotes are alongside of the asymptot +7774|Supplier#000007774|Lquwk2q1ksIRnYdMXs3LR3nCqed|16|26-476-530-8706|2154.12| requests above the fluffily pending dependencies are thinl +7775|Supplier#000007775|rmASNxMyMyalTy0WNUR7uIC|16|26-347-515-9936|1326.75|s. furiously regular pinto beans cajole thinly daringly bold pinto beans. slyly ironic +7776|Supplier#000007776|jIk7IyJ0futtV7Te|5|15-836-662-2866|-433.43|accounts haggle furiously against the blithely regular packages. carefully even request +7777|Supplier#000007777|nAyZoZF,GfnpEtFRV1yf|3|13-513-149-1172|6343.14|st the special grouches. special, ironic warthogs was. sly +7778|Supplier#000007778|eHG8kfOA9PRfMA2DHOCcGf8UEzP8 |4|14-502-949-1661|6177.66|usual packages are furiously ironic asymptotes. blit +7779|Supplier#000007779|OUJsuYS8xbd5q|19|29-486-478-5290|8023.66|out the furiously final deposits +7780|Supplier#000007780|Vt6L80UlNf6cFi|14|24-388-983-4855|2789.31|express pinto beans serve quickly across the asymptotes! blithe packages detect. silent orbits cajo +7781|Supplier#000007781|iAcDHfNh0L2tsZoS 8y|15|25-813-120-1308|6850.39|permanent platelets! special requests nag. carefully even platelets alongside of the deposits are fu +7782|Supplier#000007782|hv,IaLTCWOcEACJFFyeukRJMXiPR5DDJoZp4UTfe|0|10-594-245-8826|6590.14|refully with the quickly pend +7783|Supplier#000007783|BcgnR6Touk4jKmTtHPurLD4wW2cEx|10|20-953-753-1051|-474.75|ilent foxes. quickly ironic dependencies nag about the blithely regular excuses. furiously even fret +7784|Supplier#000007784|Oh6ZsewFySTCL7L 2SM5c1ZHDx6EiUnRICe|10|20-352-664-9249|4822.50|ular courts wake quickly slyly final accounts. furiously special packages brea +7785|Supplier#000007785|RPbPiePwd6GeI 6p8|15|25-397-519-7485|1771.66| theodolites. quickly regular tithes integrate slyly. fluffily final ideas haggle carefully fur +7786|Supplier#000007786|rnIWukHu6i2q5BzE5PlBmtP|14|24-577-644-1896|8299.41| slyly pending theodolites. slyly final +7787|Supplier#000007787|jFzI0KxRKWWPkojrmk5pkU8z3bS0|12|22-225-237-5680|6191.32|eep ironically regular requests. slyly ironic packages s +7788|Supplier#000007788|Edo,P6LzKKkmFlNSZ6 mx8 A Am|22|32-700-969-1048|6560.17|r asymptotes cajole regular deposits. ironic deposits detect fluffily across the unusua +7789|Supplier#000007789|rQ7cUcPrtudOyO3svNSkimqH6qrfWT2Sz|3|13-518-202-7692|1883.41|ously even packages haggle. ironic packages sl +7790|Supplier#000007790|O,TZYEySEeLn|22|32-973-981-1466|5392.16|ringly regular, express pinto beans. slyly silent requests sublate stealthily after the unusual inst +7791|Supplier#000007791|46e5140mT1HnBpOZ9mG7f5i78vOqtL1N5FYw|4|14-659-595-6324|6059.82|osits. slyly special packages are regular, bold requests. slyly final th +7792|Supplier#000007792|ovNXKqOaK4YOYYZ4mmO2URULVI WE0bwvr39EQ|15|25-340-568-7901|7453.78|carefully special accounts use carefully. packages nag among the furiously special platele +7793|Supplier#000007793|sUQBstvYjstKdUYn D7XL8yJTmBC64et2|17|27-484-620-7389|5747.36|ecial requests are blithely across the slyly unusual foxes. express senti +7794|Supplier#000007794|uIrh 6AZ3u25ZBB9sqT3R0gCjR9cuDEjH9gQ|23|33-284-453-7930|2832.08|cial instructions wake sometimes dependencies. final asymptotes snoo +7795|Supplier#000007795|00e WVRxo H0pzA H|14|24-525-507-8150|-213.39|gle after the carefully silen +7796|Supplier#000007796|CCT1dU6BWEuQqkx2ncjUv0Pivw6Dn5V1,z,tA|11|21-378-272-7968|8514.61|e at the regular accounts. carefully regular a +7797|Supplier#000007797|TbytYWpsXmK0q7rGCO,|13|23-155-428-6355|4170.92|s cajole alongside of the slyly pending foxes. quick +7798|Supplier#000007798|ZJj16yxfA1GCHfrwZyeSdL83dpY|19|29-800-160-5305|6597.79|ss the carefully ironic instructions sleep regular p +7799|Supplier#000007799|3C4Q,c1DdaG4c80SMqqB2JqJzVhGFN0WFb|22|32-149-260-3351|7037.91|ar, express packages. blit +7800|Supplier#000007800|G7YB8hTAXI2go,2LXlUhG23OEnUKzs0WX|12|22-872-305-1125|1126.58|y even requests wake blithely +7801|Supplier#000007801|69fi,U1r6enUb |3|13-235-541-2569|1296.61|lar, even deposits boost. final accounts serve after the furiously pen +7802|Supplier#000007802|L6A35lJd5KOBZm|22|32-410-504-6173|-968.84|need to nag according to the reg +7803|Supplier#000007803|ud5MQ,DhfHSfxxvBJ8yr3K0yx1W,JF,nyksdo|5|15-744-596-1467|5630.23|ly ironic foxes. carefully regular ideas haggle quickly alon +7804|Supplier#000007804|Gbfvxk7,biVOlG9BhJHklUcHHUew|9|19-702-469-2351|3310.50|ic requests. carefully final pac +7805|Supplier#000007805|6LxaWLIsA9kAJn97S6aR5WMfowcg,n7i,Q|11|21-938-602-1709|-407.38|carefully pending foxes maintain furiously pending packages. +7806|Supplier#000007806|fFT hBBnrI4vnjJQGrXvyEhaS|22|32-304-212-3169|4109.56|ake blithely regular foxes. pending, regular pinto b +7807|Supplier#000007807|oGYMPCk9XHGB2PBfKRnHA|22|32-673-872-5854|9312.97|ecial packages among the pending, even requests use regula +7808|Supplier#000007808|njeU4t0Vqf1sbSyybCs8Lr3QhyWFo|4|14-738-885-1849|2110.80|counts. final foxes eat even depen +7809|Supplier#000007809|rg4oHdVO5 nKR2Np eQbeRP47tu6i4SjzyImCMBc|16|26-659-492-6308|-894.28|tegrate carefully. fluffily ironic requests affix. fur +7810|Supplier#000007810|mQN,Ov8kZQ1IYo,nwTO|20|30-915-744-5763|4787.13|ffily ironic requests engage carefully furiously regular packag +7811|Supplier#000007811|ohb7JDdYwuHZ3Gwgh0ycSj|1|11-148-611-9104|1323.49|arefully bold requests above the fluffily unusual foxes cajole carefully along the bl +7812|Supplier#000007812|Q1MxuD0miQqNYh|11|21-937-225-8778|-181.63|en theodolites. slyly regular foxes haggle. ironic sheaves sleep against the +7813|Supplier#000007813|W2fHvGh7zpl|24|34-507-168-8411|3345.32|r accounts use never across the packages. ironic d +7814|Supplier#000007814|FmY35HR6AilX,FeB9VYY0mIMcC|1|11-802-580-9519|1731.45| packages. slyly regular courts alongside of the final dugouts cajole blithely ironic, special re +7815|Supplier#000007815|pIXH,lXMVPMknhTIXb4owWLtOvOmsdb|12|22-137-294-1053|3364.67|ly even accounts are among the special, ironic dependencies. ironic, express accounts sleep e +7816|Supplier#000007816|1ejcJ545bwLWLuY6Qq4qyEExZIsp0SG|21|31-710-857-3724|7962.30|theodolites according to the bold pinto beans print bold foxes +7817|Supplier#000007817|OcRGcwTmBeaRd3|14|24-368-918-1957|5487.35|ns. special accounts sleep fluffily even dependencies-- unusua +7818|Supplier#000007818|yhhc2CQec Jrvc8zqBi83|3|13-952-431-4864|-12.93|t theodolites are furiously even, ironic ideas. quickly regular ideas wake according to th +7819|Supplier#000007819|hw 95lkhhkNUIq3cYuctCfjlUOU|22|32-111-417-1890|549.24|ngside of the final dolphi +7820|Supplier#000007820|KPPWF8P1I4fCtScXCB1UYG|4|14-933-162-2364|30.64|ng the busy deposits-- slyly careful deposits nag blithely blithely dogged grouches! slyly +7821|Supplier#000007821|cXfD, N6NyIc5lkyw78HFkIG7LF7bClLF|24|34-579-661-4258|6887.35|according to the quickly regular deposits. furiously express requests cajole f +7822|Supplier#000007822|2Hm6mYC1LuZ|10|20-647-729-8545|658.14|ong the blithely ironic ideas-- slyly fin +7823|Supplier#000007823|i2 gFooiCz69Xb|22|32-905-204-8095|4353.56|l theodolites use slyly accord +7824|Supplier#000007824|oAJ01q0n9BwODTzQuGzJe9FKrSIn 0518gf3 hEj|1|11-359-394-9378|7295.28|ly even accounts will have to affix carefully alongside of the deposits. +7825|Supplier#000007825|kc8mL5To13YGHGxGu7 qqx6D2X|24|34-231-129-4654|3835.46|lar deposits wake blithely slyly express ideas. carefully special asymptotes wake slyly according to +7826|Supplier#000007826|fibA02vvRyrad6kpuFFE86gT2cuPa2avvpCD8Jrn|9|19-450-310-4326|489.99|s are against the fluffily ironic pinto beans. doggedly final packages +7827|Supplier#000007827|xVuJR6toTDWvYtJ,q91ZlidhvF57T|12|22-847-639-9409|5282.68|ep. unusual deposits haggle slyly. blithely even packages wake: regular, bold foxes wak +7828|Supplier#000007828|64Q,7Mg48bp7dnO1pd|7|17-598-100-7050|2938.73|nag slyly silent deposits. even, final deposits cajole sp +7829|Supplier#000007829|8cnmZw5MzyOmoKuRB9AH57SYf|10|20-303-785-6251|-767.25|regular, unusual platelets. special excuses about the slyl +7830|Supplier#000007830|mB80tCjcieN0LtbprpGMzxAyWcAcHSZtSrS|7|17-442-518-4753|5776.16|onic requests nag carefully. carefully regular asymptotes among the final pin +7831|Supplier#000007831|cGB0mb6mHTrwHu56nNU,aAcowe LfN|4|14-393-422-1733|2873.54|s. blithely regular deposits snooze carefully quickly +7832|Supplier#000007832|W0Pm,l1sQ,uMwhFqd|1|11-875-310-2765|3585.71| foxes cajole blithely according to the regul +7833|Supplier#000007833|Rb2MOPTOPhZ0bB3 qCmt pI8ubPga1|16|26-196-264-6425|4990.73|the slyly even excuses. slyly even account +7834|Supplier#000007834|RBIXpztLHmjVrF3OnD7YT8,VuZyLqk4G|14|24-726-110-2610|2250.21|ecial ideas are alongside of the +7835|Supplier#000007835|3MFYBunQEcXt YYp|2|12-713-543-2873|-825.92|t requests. busily regular deposits ar +7836|Supplier#000007836|jQDML3J1qW79xUKXDs|16|26-232-396-2087|9722.88|es. special asymptotes doubt carefully-- fluffily express foxes against th +7837|Supplier#000007837|tMhT9m4e3qgs|11|21-874-773-1649|5545.39| excuses. quickly ironic accounts haggle slyl +7838|Supplier#000007838| gla4bzJkQP7mcmDfq54ajUJD4Pi9OMDzGdDP r|11|21-182-126-9287|7340.63|arefully ironic packages above the special packages affix fi +7839|Supplier#000007839|iprNV,S9Tgrt9EaHH9OWUaP|7|17-385-707-4568|6963.39| deposits across the ironic, ironic deposits haggle among the slyly pending foxes. slyly specia +7840|Supplier#000007840|nQs5o8p,m, 2vxJ6o|14|24-324-826-7162|2793.56|ffily pending requests after the carefully final asymptotes are carefully instead o +7841|Supplier#000007841|ztTSmFVjcII1|6|16-276-111-8473|1266.58|kages along the carefully special patterns boost furiously accounts. slyly +7842|Supplier#000007842|F1MV32ZAwYPR2jVH26jvKIFnfAFcignQ|0|10-578-478-9082|377.67|arefully special foxes around the even requests haggle quickly according t +7843|Supplier#000007843|RcfSaB8NTKOeHQD 3xNGvUI6v7Qf,s9R0|5|15-985-681-6603|9493.86|n accounts integrate quickly. qui +7844|Supplier#000007844|0zCap22lnO0Z5|9|19-876-293-2395|6299.11|ackages around the slyly express ideas are about the even packages. carefully r +7845|Supplier#000007845|agwGVTzLyRKOsZxLVi,mPWZ08Qxb|21|31-261-512-4950|3172.29|y ironic ideas. carefully ironic excu +7846|Supplier#000007846|oL1Hg d51q1rG,hf52 EPPEuQPFEU491B|10|20-294-169-7889|8639.71|side of the pending, stealthy ideas. carefully thin accounts +7847|Supplier#000007847|683Ud QZz0vWvaE3rY64Y|20|30-143-576-9014|5701.49|along the slyly bold requests. silent dolphins use slyly. packages against the fluffily +7848|Supplier#000007848|3xG5VqbMBECCCX3hPe8 TvXF|9|19-347-560-9526|1979.86|ully ironic accounts sleep alongside of the carefully bo +7849|Supplier#000007849|DuvTpT8wi2yGTQ8gUGSZsm7DhWOjyYV5D0Y|23|33-953-595-4092|9496.00|slyly about the carefully regular instructions! pinto be +7850|Supplier#000007850|AynI1tvQ0MDmkvi150qu|5|15-879-894-8543|9365.51|ly furiously regular deposits. re +7851|Supplier#000007851|vHtuwe0c8warc864AGRhMugJJrNZEQXSDCz|23|33-479-940-1329|5129.07|ar packages wake about the platelets. packages detect +7852|Supplier#000007852|8kw8Ns 4unCuU1qT5kt|2|12-843-592-3862|6246.88|never express accounts haggle carefully. qu +7853|Supplier#000007853|MK40bQXnu0Z, zDH2q9n0tRlCrTTerJGAWy|0|10-929-858-6792|5360.14|er the regular accounts c +7854|Supplier#000007854|RAwjGaqKjgYiV6LhI4SBOnaQ8BtGR0C BfnaFmq2|10|20-144-639-4321|6829.83|ven, regular platelets. slyly ironic +7855|Supplier#000007855|u FDeIpb3pxD9ChYiIi0bb8QLijGQfbUiZo|18|28-265-425-9975|2055.63|unts. quiet requests are. final accounts hinder carefully. caref +7856|Supplier#000007856|HQF50pKPPgtI76wepLiCRT|12|22-865-714-4687|6917.21| bold ideas sleep around the pending, silent pinto b +7857|Supplier#000007857|rZBpZTtgaSH7|14|24-950-862-7706|7623.37|es about the quickly careful asymptotes nag slyly after the blithely final deposits. careful, sil +7858|Supplier#000007858|cKtSTHEiyIETZf8kx8bts0xFm|6|16-838-419-9654|7101.23|ly express deposits. carefully even instructions +7859|Supplier#000007859|vNCd3TFRbJAOP1Gyeki|16|26-652-585-4258|9456.97| instructions. carefully bold foxes against the deposi +7860|Supplier#000007860|Qm5jTdAGzrSFO9S4XUvLEubDavOpuY|10|20-976-207-4061|2356.57|ts. ironic deposits cajole carefully. fluffily regular requests are st +7861|Supplier#000007861|BAbPS4aWeDHKehCNlaSt|14|24-532-772-5730|4872.52|fily about the bold, bold packages. even packages sleep of the blithely silent instructio +7862|Supplier#000007862|rvQsXb ezTrQX5G qLFC2NegdP,LpxCJ|17|27-218-346-1316|6479.08|gular accounts! carefully quiet theodolites slee +7863|Supplier#000007863|2gfNOSwT2e2AW6nlsT7tg7mS0fU7g|11|21-550-486-6331|2902.62|symptotes. final accounts wake blithely blithely final theodolites. thin accounts +7864|Supplier#000007864|j77x8tp9jK7ET8dW4NiowgEsH6oEm|17|27-980-682-8808|8316.32| finally bold instructions nag ironic ideas. express +7865|Supplier#000007865|5cDGCS,T6N|3|13-316-336-4749|4392.52|nic requests around the final pinto beans wake slyly fluffily careful pinto beans. +7866|Supplier#000007866|7DFuXLW8cplC,Y2BZJcBWPP4qa PcE1|19|29-922-399-1133|2143.90| after the blithely bold fo +7867|Supplier#000007867|2He6FjkcW9mnyvn0XsUqUnfuA108nDPiUQg|1|11-506-283-2588|5562.61|s impress unusual foxes. slyly express instructions haggle slyly along the carefully regular +7868|Supplier#000007868| 20H775tZOj|10|20-852-512-9114|-276.06|ic, express accounts nag. ironi +7869|Supplier#000007869|fc,7Qytzz4MhuQkk2ayBEqyUjxF0YF|14|24-441-351-6208|188.94|al packages around the blithely final theodolites cajole fluffily express pack +7870|Supplier#000007870|Yg9njBC12zKL3SDTjLXiqSKNgHDWSZxTUd|3|13-201-636-9672|5447.16|ing accounts; blithely regular accounts x-ray slyly slyly even theodolites. carefully final ac +7871|Supplier#000007871|mfB1B8A6xIFJX|19|29-601-998-2408|6444.80|uests haggle doggedly in place of the slyly unusual packages. carefully special deposits cajo +7872|Supplier#000007872|wNlvR0u0jwFd3wvWdRGpnN0G9g3ArZyrTuX|6|16-117-200-6865|9808.75|inst the express, bold ideas. furiously final deposits haggle carefully around the sl +7873|Supplier#000007873|nqHeHayWL6GV4yrpCaUhESHiWlB75HXGxeP7Z|11|21-910-520-4230|7809.11|even ideas wake quickly across the accounts. final accounts w +7874|Supplier#000007874|pyghw9iRdkGvkF0Yf5HEqJvmSOohAPFx6U|3|13-297-807-1753|4836.47|ets during the bold multipliers are quickly ideas. finally bold foxes poach slowly pending accou +7875|Supplier#000007875|E0CkoBYngcIoH|21|31-165-274-3168|5559.40|foxes? regular epitaphs wake carefully regular instructions. even asymptotes wake +7876|Supplier#000007876|Mbu8yBOyJuBTnRgrydVRPdaULpLJENSpAQHVfCM|2|12-846-219-9347|908.61|refully unusual accounts haggle +7877|Supplier#000007877|rCq0Pz7 dpLDT|11|21-961-125-7491|969.31|g requests nod furiously final account +7878|Supplier#000007878|aq6T3tUVq1,|10|20-861-274-6282|9132.92| waters cajole ironic, ironic requests. furi +7879|Supplier#000007879|MXMejJ89ve|2|12-146-373-1024|1046.93|final deposits. ironic, ironic dependencies detect furiously. asymptotes mold slyly. theo +7880|Supplier#000007880|sKOPFlOG,h7JL7Gs5sb3i,lXL5|20|30-205-213-2655|4705.39|kages. accounts mold blithely fluffily final pinto beans. ideas integrat +7881|Supplier#000007881|sxyfYqVVNWeU,cUBYav44cOSl1ozzLDb|16|26-878-885-8667|2505.25|cies solve quickly above the +7882|Supplier#000007882|JDv8BZiYG0UlZ|4|14-111-252-9120|6763.46| the silent accounts wake foxes. furious +7883|Supplier#000007883|lHEVjn RqfnL4Lk,bPLs75RonEqP|16|26-136-979-7523|8542.18|ackages-- slyly even asymptotes are +7884|Supplier#000007884|d35SIqy9frreEhCKx9xaWfTc7sqr77|5|15-194-649-4415|5531.13|efully inside the slyly ironic requests. furiously regular platelets cajole blithely +7885|Supplier#000007885|u3sicchh5ZpyTUpN1cJKNcAoabIWgY|3|13-347-523-6960|9739.41|nts grow quickly. pending, even +7886|Supplier#000007886|sFTj5nzc5EIVmzjXwenFTXD U|11|21-735-778-5786|8405.28|ven dependencies boost blithely ironic de +7887|Supplier#000007887|Ee40AjRtmPosrTS,hTJ3tTRYbDpvnxfl|19|29-222-572-2267|3563.05|sits haggle slyly ironic, final somas. furiously special de +7888|Supplier#000007888|nq06Y48amPfS8YBuduy1RYu|4|14-462-704-3828|9185.89|ole slyly-- blithely careful foxes wake against the furiously ironic accounts. pending ideas +7889|Supplier#000007889|Y5MRvWgfmCp3 21MlN98j4L22bnPswG5JVKe|20|30-987-656-7239|314.59|n requests. final, bold accounts wake furiously +7890|Supplier#000007890|qOLT eUaoROSSk5IUrsHkLdzvke7M|2|12-395-871-7093|5274.34|romise blithely ironic deposits-- final, +7891|Supplier#000007891|TNRiO4REOsgcOM0LThVlBGaKWr|4|14-337-166-5000|5909.28|ely ironic courts boost carefully. requests among the silent, ironic +7892|Supplier#000007892|OmsJ,xFBoYLVBh8Xwc38uRpb,CZJMbl5TCM0|11|21-987-659-7099|7338.06|egular theodolites. doggedly ironic requests against the careful +7893|Supplier#000007893|1pZDjCVpoIhJGYzRT1ArcyQipV|22|32-879-860-4012|7817.81|courts. pending packages are sl +7894|Supplier#000007894|YA0K2vTH477wz|6|16-825-146-7043|7961.26|alms detect busily. packages snooze across the even, ironic courts. regular ideas cajo +7895|Supplier#000007895|NYl,i8UhxTykLxGJ2voIRn20Ugk1KTzz|4|14-559-808-3306|9977.32|mptotes near the quickly sp +7896|Supplier#000007896|s5au3AXlF uwmEGeUAlTPW|20|30-121-893-4987|2915.20|. blithely silent packages cajole furiously ir +7897|Supplier#000007897|5iEWbfP8msH2aAUFz7LcaZwgLwgw21l2bA|18|28-672-947-2172|-305.37|ronic requests. final, final accounts unwind. slyly ironic fo +7898|Supplier#000007898|dAqAM3otCo2GkjyjtKp07BPYCF31,rJ0B|4|14-604-858-1037|2586.68| accounts. furiously special ideas boost +7899|Supplier#000007899|ikL4hIysHS53rWTz75XXLJZz5n7AJGs,JkldS|15|25-404-602-1786|353.88|ckages affix quickly. requests are carefully quickly regular foxes. regular platelets sleep +7900|Supplier#000007900|B7 7NHK5uwQ7GRYBCiRwgHqjhYYo62cvO6so|12|22-196-744-1968|5643.06|he furiously close requests. c +7901|Supplier#000007901|aS6VO,6XvwM9|13|23-777-524-1341|9869.16|lites. quickly unusual dugouts must nag stealthy frays. foxes haggle. speci +7902|Supplier#000007902|aGB2xiccVNPHFnfWCsegYkRZ1EsPHvrda1Oxf8|12|22-971-786-2474|117.86|ets cajole carefully foxes-- even accounts wake slyly blithely even patterns. bold packages wake. re +7903|Supplier#000007903|lYp 8NwPsgD59y0T5w06RVy|10|20-445-158-6409|9835.11| the slyly unusual packages. unusual realms could have +7904|Supplier#000007904|CROVP3A970tLr lDT|9|19-199-240-4636|-142.41|, ironic ideas. quickly bold ideas use. blithely special packages are q +7905|Supplier#000007905|xJcTxkuamI|5|15-155-649-5726|2482.69|y bold pinto beans grow slyly bold packages. express excuses cajole n +7906|Supplier#000007906|nyaAJ,u2Nx6MjFr7k0I3 ZuIvItZxx|21|31-720-373-2830|1456.51|above the carefully ironic ideas. carefully final deposits wake blithely +7907|Supplier#000007907|OCwT5z7KZVKF 7MPVfJWCxrm|15|25-194-672-8309|2492.31|ffily carefully final excuses. ironic deposits sublate slyly. regular +7908|Supplier#000007908|ghhHapj7GK|21|31-559-796-5693|9178.40|sly regular accounts boost carefully among the fluffy accounts? quickly ironic packages be +7909|Supplier#000007909|kcbOiaTa0YE0vDSQYwCD2R92xp|20|30-668-590-3883|6515.91|se carefully about the carefully even realms. furiously ironic packages wake alongsid +7910|Supplier#000007910|nGpgFqT87UJ|7|17-867-434-9210|5986.76|dly against the slyly final accounts. unusual, regular pains about +7911|Supplier#000007911|ifcV,O,0 a9UI1T|10|20-966-522-4066|419.95|aggle furiously. express requests eat care +7912|Supplier#000007912|zlg9yNp7xHdJ0vHBHyR3t|16|26-367-444-5750|5005.81|riously about the quickly final packages. unusual deposits nag +7913|Supplier#000007913|DkiUqg8AVUilpH66MvLPB7lBGkyXOuOFjur9Ha,O|9|19-255-694-2591|2737.66| slyly. furiously even ideas nag furiously. furiously special theodolites +7914|Supplier#000007914|0vDzSHC sJwHKkhLCdUu2|4|14-669-382-9669|7782.78|encies. fluffily bold foxes integrate fluffily among the patterns. accounts sleep qu +7915|Supplier#000007915|I,nKaalo6vCMqv4oWlLm17qeM0RAFZZO5TyKvf|20|30-463-259-7096|2164.90|ins. sly, ironic deposits use f +7916|Supplier#000007916|h0yS1ULM2u53tqGxDg2L4eYtxw|11|21-673-623-7279|6509.99|. quickly ironic asymptotes about the blithely +7917|Supplier#000007917|,mWX0LQc80V9OzEk|12|22-255-581-2696|5452.61|notornis sleep slyly even pl +7918|Supplier#000007918|r,v9mBQ6LoEYyj1|3|13-615-426-2832|-941.33|sleep carefully final, regular escapades. furiously regular deposits sleep bold deposits +7919|Supplier#000007919|D,yno5gMBfuFMfeV24WgMHlz602tnFDx|15|25-160-798-4025|5928.44|eposits sleep along the reg +7920|Supplier#000007920|lTxKUF2dFL84P6rc5BiqY4re9OZKDF|22|32-767-307-2688|-699.56|bold requests. carefully express Tiresias about the furiously pending foxes haggle quickly even +7921|Supplier#000007921|MJfvL3B1stDjGBFcZGRipxkln|16|26-495-903-2607|-659.81| busily final packages nag express, final pinto beans. platelets are slyly. bold requests +7922|Supplier#000007922|VWdpSpCHq,KhqVWh5CaDa,ktVKflNGTikEJe|3|13-126-582-5449|7557.73|s are unusual deposits. ir +7923|Supplier#000007923|JnxkRctr77k2aWKVIy8o 84l|16|26-862-106-3554|3819.21|nts are furiously against the +7924|Supplier#000007924|99jR7jC3t47M0wFaUvEX|13|23-363-479-2415|2508.48|he carefully busy accounts. carefully unusual packages haggle quickly final instruc +7925|Supplier#000007925|ZYse9qPjpNXWBzksL46jXHa|5|15-630-577-2259|5983.59|ins. boldly final pinto beans integrate fluffily slyly bold deposits. carefully even instru +7926|Supplier#000007926|ErzCF80K9Uy|3|13-829-609-6904|2159.30|re blithely. even, special braids sleep furiously. deposits boo +7927|Supplier#000007927|ndt8aw tJNiPrn7vGV9r0bJg9IX8d|17|27-937-416-1053|263.85|uffily after the slyly special requests. quietly express frays +7928|Supplier#000007928|nHaEdI 5SQDgEolYw|0|10-160-150-8622|-560.58|ould cajole blithely against the furiously regular acco +7929|Supplier#000007929|Zrv8c59BS9Hd|17|27-837-215-3993|3295.70|e the carefully ironic accounts sleep against the slyly p +7930|Supplier#000007930|vofDxBRAvejcwdaWvU0wmYHWbiJt|18|28-930-650-2467|3873.95|ic deposits nag carefully. final accounts along the bold theodol +7931|Supplier#000007931|nOT6fqQOJpmLcJjh3Dn9iQKmrOZfGNs MMWqHng|1|11-904-868-7162|9500.54|latelets sleep quickly theodolites. +7932|Supplier#000007932|4NAtwsPVHVUmuaFXA|8|18-138-224-6208|8572.69|long the deposits wake quickly care +7933|Supplier#000007933|yo7nyC8Tj Z0sVvFk,iGQO6O7HH|14|24-161-804-6338|1220.53|even courts. fluffily silent pinto beans wake upon the +7934|Supplier#000007934|V4mz6vAwVGH6j3izJ,dw8DnqWu3FN|9|19-825-315-2126|7325.13|furiously unusual courts u +7935|Supplier#000007935|N2vTacDfwSUl2DP6h0,YyV8|0|10-833-595-8002|-439.84|ickly. carefully close deposits doubt fur +7936|Supplier#000007936|ZvvJ9sz2igbHBsMzrnPesv1QptwzqO 2ty2zV7|13|23-252-906-2144|8555.28|endencies wake slyly permanent ideas. asympt +7937|Supplier#000007937|sXQr5CxeOi3HqGa9nxIf7m0kWFZI|7|17-586-381-1686|5670.39|ncies cajole slyly: final, ironic deposits cajole slyly daring requests: finally +7938|Supplier#000007938|D1wWVGx8nvgw|7|17-983-979-8240|7306.69|re according to the express decoys. express, bo +7939|Supplier#000007939|i7ua2798FbUx|21|31-738-305-1815|8547.00|ake unusual, special deposits. special pinto beans haggle about the ideas. blith +7940|Supplier#000007940|t15LYMuZRsuaZhEoqHrJiI,GFgBbC750Dm2ivDy3|9|19-609-664-5458|9480.92|. quiet excuses across the blithely regular accounts boost fluffily regula +7941|Supplier#000007941|Tf7nhtroLcf|24|34-564-866-9116|8968.19|quickly along the slyly fin +7942|Supplier#000007942|vgBd9c9KUhLfkbhU|18|28-874-122-3833|4441.07|sual accounts. quickly regular deposits are fluffily above the quickly final packag +7943|Supplier#000007943|87sm8qgfLXUhrRvvaPQdHDElKll|2|12-796-457-6146|4111.36|ts haggle carefully final packages. even accounts +7944|Supplier#000007944|2KBDUbE8kmUJLE1xSAXYVRlXNhr3g4v|17|27-917-699-8552|6669.10|rets across the thinly pending f +7945|Supplier#000007945|HDt7oHA Sf5zz|2|12-594-162-1136|6598.12|gle bold packages. express asymptotes wake according to the carefully final +7946|Supplier#000007946|FUd4 jbvufK3cdff4ApgvtUkIBhIYa3y|8|18-662-165-9885|1713.31|ts mold blithely ideas. furious, pending deposits about the pending, even dependen +7947|Supplier#000007947|zWZTLm57m4Jm8uaes4|14|24-825-721-2993|6595.17|ully even braids. slyly final requests hang among the bold packag +7948|Supplier#000007948|d8aJtC7KA4AlOT|17|27-628-539-2506|7074.47|s sleep among the quickly express pinto beans. furiously bold packages h +7949|Supplier#000007949| b86VKFmew29PylyHMFY9DBwMM|22|32-962-789-2975|-404.57|ilent ideas? closely pending ideas are. quickly even court +7950|Supplier#000007950|L78Xu87DT,m2AIKHV7pFvc4JGjSEfedQP|20|30-367-904-5686|7251.18|ular asymptotes affix according to the pending instructions. eve +7951|Supplier#000007951|t7 GQxWwp3vnEmxJbj929uDco|18|28-723-888-7848|-370.10|uriously even platelets. carefully bold dolphins boost across the slyly pending pack +7952|Supplier#000007952|q Pt6xJ0o1loOB8a1TZ55NQD2SYI3OGodslBE6,|2|12-942-520-6124|3654.69|sts sleep fluffily unusual accounts. +7953|Supplier#000007953|0qNL 8Z858aPLRY|2|12-313-889-2000|9363.09|nic deposits wake bold, express accounts? bli +7954|Supplier#000007954|b7,8HFIMQGf7ciShy,F9VJYiUjLB|9|19-311-769-3690|8611.81|long the carefully ironic requests. notornis +7955|Supplier#000007955|s3BxzmLdgbdQgcfv10ot|6|16-930-614-3740|9749.98|pendencies. final packages haggle among the accounts. evenly ironic packa +7956|Supplier#000007956|vmAYh95Lr3f2kKHl6kRX2G7|6|16-947-231-8954|-993.76|ly among the blithely regular requests. ironic +7957|Supplier#000007957|ELwnio14ssoU1 dRyZIL OK3Vtzb|3|13-675-523-4067|4841.17|n, ironic deposits lose carefully. furiously ironic f +7958|Supplier#000007958|pe3M4qaP5I4bo5iXAYe2RLOgCa2v9KyIJPzCZ|13|23-210-451-1058|7477.73|lyly furiously special requests. express packages are beyond the express, regular deposits. sl +7959|Supplier#000007959|KBPlhWwBEUDld1s MDl|10|20-856-411-5901|1268.33|nto beans about the unusual, bold requests breach fluffily regular courts. careful +7960|Supplier#000007960|aCBNGE9PUQZldJxGe3N|17|27-531-635-1898|7798.43|ns should have to engage along the carefully final accounts. ironic packages across the entici +7961|Supplier#000007961|1,lqo6bk3JyHFBDK jm2SLfJKQmAIptBaIhpCPVG|19|29-211-655-2854|1242.45|final pinto beans. regular, final platelets after the furiously final p +7962|Supplier#000007962|ANn,KqJw0Gv,ShjR6Y6WA1epvnlYMndQ|7|17-885-453-3934|1736.62| haggle? express courts across the regular, silent reque +7963|Supplier#000007963|uuOvxqm37CvfJ0IEGxKo988RD|16|26-587-366-5667|2409.81|deas. express, ironic packages boost furiousl +7964|Supplier#000007964|nUniZd0m1bfEdUdmW5JGP y|0|10-827-895-1786|8714.32|counts. deposits nag. blithely silen +7965|Supplier#000007965|F7Un5lJ7p5hhj|3|13-484-222-4032|5571.39|ugouts haggle furiously grouches. fluffily permanent accounts alongs +7966|Supplier#000007966|hD,5E4m 8AlYs0IyGFPh,ecPthXMTP|21|31-367-113-9243|4348.42| ideas; furiously unusual instructions sleep boldly. quickly unusual theodolites against the slyly r +7967|Supplier#000007967|A3pi1BARM4nx6R,qrwFoRPU|6|16-442-147-9345|8877.82|ously foxes. express, ironic requests im +7968|Supplier#000007968|DsF9UlZ2Fo6HXN9aErvyg1ikHoD582HSGZpP|3|13-577-306-7622|3545.59|ly carefully express foxes. furiou +7969|Supplier#000007969|xNL7 WZwtNToE0AMy2QcA5PJXzIyLEqh Gaaij2S|0|10-363-822-9857|1708.41|are among the blithely special dolphins. blithely express requests dazzle express theodolite +7970|Supplier#000007970|69clC3kCCI8V05ZE3uMm7Hacpm|19|29-301-635-3896|1120.55|sts wake! even, ironic dependencies will are slyly +7971|Supplier#000007971|BqsyRLtbkfo4LeXBKaM5HHmXN2|9|19-430-737-2328|-924.64|ses are doggedly. carefully even foxes use quickly. bl +7972|Supplier#000007972|WW0GuiWP2N3kUo4f|21|31-958-906-9709|463.50|nt deposits are fluffily regular requests. furiously silent requests ar +7973|Supplier#000007973|R5WVAqNVOkgM9|19|29-300-869-1121|-19.63|und the platelets. bold theodolites across the slyly permanent ideas integrate +7974|Supplier#000007974|KGJzSlNuT7GQLs1TJstDR3p d9|4|14-510-748-8398|9040.28|ly pending asymptotes serve blithely express, pending courts. express, unusual p +7975|Supplier#000007975|qBAFcyc7T3oUEE4TjoUmw,7vRHlNGCDwqLr2|24|34-696-546-8861|6114.96| affix: pains cajole fluffily. final excuses are. slyly ironic instructions cajole car +7976|Supplier#000007976|OnVlYEc,j5iCcU3FHtvx1b|16|26-885-884-9454|-872.53|ronic packages. final deposits against the fluffily special +7977|Supplier#000007977|3aJgblFc9kQmZrWEGqkKuRSFqpLYLL,lJEaYAd9|10|20-124-657-6831|7355.41|the quietly even packages. unusual instructions against the epitaphs detect carefully a +7978|Supplier#000007978|oIS8K3nHt2ijkH3gwh7|14|24-353-613-1723|2336.24|en dependencies. furiously even accounts alongside of the carefully ruthless deposits b +7979|Supplier#000007979|aTz0O9HIdoStxpU,8D|9|19-787-181-5082|4142.33|nts. slyly express packages sleep blithely permanent sentiments. packages believe a +7980|Supplier#000007980|xTPaeTWz5YaE0w,MXQ8|22|32-838-254-1780|-925.51|counts wake bravely slyly ironic accounts. carefully ironic +7981|Supplier#000007981|NSJ96vMROAbeXP|7|17-963-404-3760|7894.56|ic platelets affix after the furiously +7982|Supplier#000007982|vP5Pn9zm LQLAtqjxTCg5w|15|25-955-823-5832|-111.47|cuses. final theodolites dazzle fu +7983|Supplier#000007983|7S1i4thGBqF6MIye|10|20-700-903-3227|5376.52|al pinto beans. quietly special requests engage quickly past the carefu +7984|Supplier#000007984|hIrcB8j90bREU|14|24-825-384-5312|7180.03| unusual pinto beans affix slyly. slyly express +7985|Supplier#000007985|e1riCRgsQmaY7hHkxa05aiLaJeoId|16|26-323-149-9345|2792.45| furiously regular instructions. regular deposits affix. pending, special fox +7986|Supplier#000007986| m0VcPrxNk3sdFv|13|23-915-184-8068|4669.83|c tithes kindle slyly about the final, unusual dolphins. carefully regular deposits +7987|Supplier#000007987|NZXfuWO7nKQDj4xyO31N29|3|13-178-212-4133|9889.30|ickly. furiously express courts cajole slyly. fluffily special requests hag +7988|Supplier#000007988|J83phr8A69yR21vLjWjbaRYWteRnJJOKsTj|10|20-855-585-8262|8573.44|ress, even deposits nag carefully. slyly even patterns about the slyly fina +7989|Supplier#000007989|3z5sDcQOPC|7|17-614-344-2946|2908.39|lites haggle around the quickly express foxes. unusual pinto beans nag quickly. carefully even p +7990|Supplier#000007990|wyikYITvgrLywA9kT15pmXELS|9|19-382-228-4053|6309.60|riously alongside of the carefully final instruc +7991|Supplier#000007991|wyIhlzBWdu2H6vs4QhlHO13Y7O19BLblPc|10|20-438-716-5483|3484.74|s. furiously final deposits cajole after the furiously ironic pint +7992|Supplier#000007992|3yeHFPics4wruw7CX5n|16|26-926-975-6279|1931.01|lent packages haggle carefully across the final packages. blithely +7993|Supplier#000007993|r,FoztdN1 ,YCDgea5o|0|10-409-862-8648|2399.90|ly above the carefully ironic ac +7994|Supplier#000007994|wi95xMemaeC,Yj,npbYkJeKWd1D|10|20-779-329-7750|7151.83|s would detect blithely along t +7995|Supplier#000007995|2fs43WlGi3|17|27-362-148-4667|1820.43|refully bold packages use boldly. slyly ironic deposits haggle across +7996|Supplier#000007996|AWRSTDvlVk,DyyzIVQR32499S4n1G1hob|17|27-110-414-5569|2915.11|ets are fluffily express requests. furiously pending instructions sleep ent +7997|Supplier#000007997|qQ0UDsf8ak8DcVK DV9sqXL4LTG|17|27-875-656-8454|9243.14| pending packages boost according to the regular pinto beans. evenly pen +7998|Supplier#000007998|LnASFBfYRFOo9d6d,asBvVq9Lo2P|3|13-832-953-5827|5588.49|ly even theodolites wake blithely across the carefully id +7999|Supplier#000007999|R8OX2nsjpHwLU0E3MH,Gvu15AGCG6alrW|2|12-491-290-3228|5790.82|lithely regular foxes according to the furiously regular requests kindle regularly quickly special +8000|Supplier#000008000|zYtb gUcE,2oeUsUE5JSLBWeDwmX|9|19-875-478-4736|7389.59|ounts breach blithely along the even t +8001|Supplier#000008001|gGU5ucMbIv44xIiypL5KK9ZDgbMFpS4JmJQn7qg|0|10-499-117-6546|4857.32|ly. closely silent deposits mold. busily final dolphi +8002|Supplier#000008002|,ii0NoZP4EBx|0|10-514-404-3403|3947.16|uctions haggle packages. regular requests a +8003|Supplier#000008003|dkknyQx2opL|23|33-849-525-2924|7940.59| among the quickly express deposits run carefully regular d +8004|Supplier#000008004|aEzP7RdbXVfiuAg X2PwZhIM|13|23-297-541-1438|3607.20|ts. regular accounts haggle slyly ironic requests. closely brave pin +8005|Supplier#000008005| GlOJFny2HFIMfee798hod5hM6TTJXYDiqYTA0l3|13|23-798-711-7660|3171.26|jole quickly furiously bold acc +8006|Supplier#000008006|,8,a6tucbKSGeU CYs,LGFPeCpfMDAaT,qQ6MMX4|9|19-102-545-3561|2312.03| ironic packages cajole. regular courts nag car +8007|Supplier#000008007|Myg1TwxpviQUDCOVOEBJntWIDYfo|4|14-503-620-1300|4197.01|l sentiments. blithely bold requests nag. carefully quiet requests +8008|Supplier#000008008|7Sfw60mdAZ6|17|27-156-369-1991|5218.04| accounts: regular asymptotes alongside of the express package +8009|Supplier#000008009|sFHIdVSVGrziHMFvRSAvq44JiYFvJZhO6,zBTYi|24|34-353-316-9703|5682.82|y regular deposits. unusual, regular accounts wake among the qui +8010|Supplier#000008010|D5fYYc7e5sDm0YuAuPCBvi96zBfkMyd8aW4rhH|13|23-627-623-9740|7243.14|al epitaphs haggle above the slyly even deposits. asy +8011|Supplier#000008011|HqvL3WplaOkS|23|33-542-262-8612|7425.11| fluffily ironic deposits. express, even accounts +8012|Supplier#000008012|q RVLC5ggK7vJLk eF,73tkFu8SkxWH|18|28-205-915-4656|7807.96|slyly even pinto beans grow furiously according to the fl +8013|Supplier#000008013|ygzSf,I8c5s,v5RrV9|21|31-887-459-7971|7372.88|sts use evenly around the special, ironic fo +8014|Supplier#000008014|ZKrjV7Shs5|17|27-384-475-6847|-922.83|quests above the slyly iron +8015|Supplier#000008015|2QO5akJCIWqYNq9 3oXrZoehcFnYp,5id3Vzf|17|27-211-904-6964|8898.31|larly about the quickly pending requests. furi +8016|Supplier#000008016|30sTEtBf1 |1|11-819-551-4538|6948.62| foxes haggle stealthily ironic packages. express packages are quickly qui +8017|Supplier#000008017|Ok7auFnA9NTR PN6h6WcTlMFOy8A|5|15-863-981-9311|7503.79|quests detect slyly special accounts. slyly pending packages detect-- care +8018|Supplier#000008018|CMJxs0L 2u27R0Rs6J1oLdpMGL|21|31-191-993-9051|-808.57|sly express platelets. packages eat slyly carefully special de +8019|Supplier#000008019|mCLo M9G6yfyeJ6loXFlZrnm0ULI M9G1Jwk8CU|14|24-238-272-6799|4320.26|olites use slyly from the even, regular pinto beans. pending, regular deposits haggle against +8020|Supplier#000008020|M5v6sVUveGK3vsHe0bwHyPAEPsQ5|4|14-500-236-4745|8548.11|ang. carefully ironic requests cajole fluffily across the regular, unusual requests. quickl +8021|Supplier#000008021|D89xZmYSAMzW|1|11-923-670-6983|6228.67|carefully slyly final decoys. +8022|Supplier#000008022|dCtvuTZupoUigncw9d|18|28-304-834-4786|7346.64|unts. quickly special ideas at the dogged accounts sha +8023|Supplier#000008023|C3KhKukssqX10hzkQUu59vtARTtqyCECo|1|11-610-522-7569|9010.57|lar dolphins: regular instructions after the bold, +8024|Supplier#000008024|hC5Iv7v9tUzm|13|23-804-857-7141|1697.73|le after the furiously regular deposits. carefu +8025|Supplier#000008025|26W8ar5EmIVbc,kOg7pu9Xa2|21|31-411-226-5921|8742.86|gle blithely according to the blithe +8026|Supplier#000008026|8a6hfBj9Di53|19|29-701-123-6976|1351.65|onic dependencies. regular, special ideas cajole bl +8027|Supplier#000008027|Zn8wSn9eSmkvTz,jXB7DiIx3BLsU8D0dFBTy|24|34-917-831-9173|162.63|y regular foxes boost furiously alongside of the theodolites; re +8028|Supplier#000008028|sxe trUIxkh81syJGwHG5gRVQ|10|20-996-515-7969|3190.98|ckages wake blithely according to the quickly pending deposits. foxes boost sly +8029|Supplier#000008029|kiqxSOLZYBKI,Hu2a0gV848p0peMk6CdRa|11|21-622-338-6482|3763.50|aggle quickly. final accounts use carefully unusual courts. slyl +8030|Supplier#000008030|ASJVWT93FXZ6ky6UdLetqHeod0TI|20|30-523-787-4255|1105.24|uriously express, ironic packages. slyl +8031|Supplier#000008031|xNciw3S sb19NjYOsrkmhXzP|11|21-381-164-8246|5754.22|nticingly furiously even orbits. slyly final ideas wake blithely about the +8032|Supplier#000008032|ukHUdwGMAEyNlp,cV6EnefnlHUUq9u,u|16|26-600-545-6757|8253.11|sly unusual accounts boost even, final packages. carefully f +8033|Supplier#000008033|Ar,1f6IxTjR 5rxcydlgeo9lNcAPh|6|16-692-974-3845|9071.72|ites snooze blithely. special requests acco +8034|Supplier#000008034|rCfxdUWSblVtfexVCO,|22|32-538-362-5392|6436.41|inal pearls. silent theodolites sleep never about the carefully +8035|Supplier#000008035|,zLgenfT0jr16MbjJt 9WfCU6xm2N3hQy79Migc|22|32-321-763-7850|766.64|ously. regular packages kindle blithely furio +8036|Supplier#000008036|Jqrts8t4bPk4VZ6eG03NC9jBP56ZJQHNGP1|1|11-461-650-4235|1845.41|xcuses. slyly special dependencies impress carefully around the furiously even pack +8037|Supplier#000008037|xaAnic,DwJs4EGY1daJfhrsUyneUnrA|1|11-402-790-6442|4838.67|slyly above the express, ex +8038|Supplier#000008038|YkkdqEh9lvZyRRE4nmJ3vsZM0|24|34-568-253-8725|9530.60|sly special deposits boost bli +8039|Supplier#000008039|FZ T0P2ZR9kOgv20bhQvICsLtbkY|9|19-829-475-1177|7949.07|lithely ironic accounts. blithely quiet attainmen +8040|Supplier#000008040|TJUSGy72qAbW6ynoKp|11|21-850-239-4091|3107.77| packages. carefully special theodolites lose ac +8041|Supplier#000008041|U4KCK9OHWG1kCMarASaGrC1|9|19-620-898-7831|2333.32| multipliers sleep? regular packages on the special ideas wake carefully after the blithely iro +8042|Supplier#000008042|KikST1B3N6WnLd3gaoqhbgFZAl3|1|11-732-273-4839|-882.77| bold requests wake quickly. busy, silent deposits wake slyly e +8043|Supplier#000008043|xpjNT1nf4yV6yt7|17|27-621-359-8967|136.55|boldly regular foxes. ironic dependencies +8044|Supplier#000008044|uVZlh XFXzpo|12|22-952-525-1962|4885.04|pinto beans. even sauternes about the finally final deposits cajole slyly unusual, ironic theodolite +8045|Supplier#000008045|xlJ6zQhjNVCFJ KM2Qqm XCvvYbe|11|21-700-845-5940|7807.69| blithely carefully ironic +8046|Supplier#000008046|AfgBtGOCODl|0|10-893-233-9673|2854.02|s nag furiously after the fluffy, fi +8047|Supplier#000008047|QFuiPZDZOwOMS6NA|8|18-766-969-2724|5473.67|s around the stealthily final theodolites are against the quickly bold accounts. frets integ +8048|Supplier#000008048|LE7PAf3y5q47XLyaHvUxmT3M1Kp,MVESX0uZBP|13|23-397-248-8524|941.33| slyly according to the carefully unusual packages. carefully permanent attainments ought to +8049|Supplier#000008049|foQ57pmI4dMmhxf6SIkFcsun|24|34-383-176-4342|948.95|. ironic, regular pinto beans cajole blithely pending ideas. accounts above the slyly ironic reques +8050|Supplier#000008050|9odvS2iZM 5gJq, cZC2J0hZet|9|19-573-848-7065|8028.58|regular somas. final deposits sleep amo +8051|Supplier#000008051|3XAJrrIVHDnqGIG|4|14-541-202-3850|8346.83|odolites wake carefully except the slyly ironic requests. even foxes wake. fluffily thin re +8052|Supplier#000008052|b rrieCRzxNb|4|14-554-998-9079|229.30|, ironic accounts. regular requests detect. special depo +8053|Supplier#000008053|Jq9bL 57m,CK5v7xOltQv9b9H|14|24-908-874-6734|1849.25|r packages. blithely regular pinto beans nag unu +8054|Supplier#000008054|lmPKW5sjsabSLPeQ4Cb2bTw2S1WX7QjIIfWQe|21|31-953-691-8310|3276.91|e about the foxes. pending deposits alongside of the carefully ironic sheaves +8055|Supplier#000008055|cYfooK5a68F3 39AmTM6Sl|20|30-166-145-4303|7015.45|ronic requests. furiously reg +8056|Supplier#000008056|d9x0700R,o,KTYA,j02vM1,v7E|11|21-584-491-3991|8746.05| deposits sleep. quickly idle requests are closely before the closely +8057|Supplier#000008057|nELhdFGUYN 4QRtB0ZFE3ab2jAYU,vVYoe|24|34-974-613-2392|-121.33|ts was furiously final, special foxes. quickly +8058|Supplier#000008058|Uc791Nb820fPbOJSG1 |23|33-579-328-2172|4661.67|gle furiously according to the final excuse +8059|Supplier#000008059|j1r6orfJxZ9Q|19|29-557-610-2029|4760.75|c accounts. ironic requests boost slyly quick asymptotes. regular accou +8060|Supplier#000008060|pZjj0ghxyfltSeN pR5u5PKgiVGhXGd|16|26-552-123-7988|8949.16|ly regular theodolites according to the asymptotes integrate e +8061|Supplier#000008061|6pgoLkjGTMX8|17|27-282-225-6689|3596.35|ntegrate furiously excuses. slyly special requests cajole slyly. blithely e +8062|Supplier#000008062|lARUW6y2IlUZ5PBMO3fV|2|12-361-329-2413|7820.62|nal requests. carefully final ideas integra +8063|Supplier#000008063|Ns4l5Oiu3wzdp8LuZMkHWfbg5sm0Vq|8|18-132-649-2520|249.51| about the foxes sleep slyly blithely ironic +8064|Supplier#000008064|v2BgGmLlQiL49 Q|21|31-502-367-9169|391.64|tions haggle furiously after the carefully special deposi +8065|Supplier#000008065|zKFXk1TFX7koovd 4nWGMwJCM43sCSXVARl6cn|8|18-526-379-9871|2213.59|ake about the grouches. quickly unusual deposits w +8066|Supplier#000008066|OBZime2WRhTb29JcSUs6gV9vgiu|8|18-444-669-6796|-155.07| ideas. slyly regular packages ar +8067|Supplier#000008067|OjjTPhvOmI73nG,CyMMj3lR8PcGe|17|27-711-979-6932|9740.02|s integrate against the blithely regular braids. regular accounts among the f +8068|Supplier#000008068|suaMcZxYQ21wvMDFOd7|23|33-135-280-3856|7542.81| the bold packages cajole slyly regular instructions. carefully regular accounts engage blithely +8069|Supplier#000008069|tO4xgYvwPt46B28leZxxjJz|1|11-736-652-2346|6117.72|gle accounts. instructions haggle +8070|Supplier#000008070|XpcN4NuGnpW9trpxil5UW9MpC|22|32-916-779-3048|1537.76|gly unusual packages poach blithely unu +8071|Supplier#000008071|Udo,oIzBLGVkEtDMnJf1P0vq8FAc6|14|24-716-167-1865|4123.72|ress deposits. furiously express requests wake slyly special +8072|Supplier#000008072|xrYQHxc2neZ4x13HTt|1|11-742-643-5639|5340.80|ages. slyly special packages doze at th +8073|Supplier#000008073|gdE1NN4kHdh,MbLCAqPA4K9NCtezqz|21|31-669-638-6225|838.13|even instructions are carefully according to the furiously special foxes. +8074|Supplier#000008074|ri8Kci7sEHhwWlRzXlYTU|15|25-178-310-4315|4464.67|ding to the regular, final d +8075|Supplier#000008075|vRm,,T6Z9nl8ZkWeUX70v|14|24-446-829-9557|4679.89|le final deposits. carefully pending requests use carefully. blithely silent asymptotes s +8076|Supplier#000008076|x9NnZNQwNnG78eiif|13|23-492-570-7101|2992.23|ut the furiously unusual accounts? furiously even theodoli +8077|Supplier#000008077|Z5qb1VrjMqGc2C9kY9NECx3lNJHJgS7ixnafZ|12|22-255-537-7601|7358.90| special, bold requests wake against +8078|Supplier#000008078|Efbxz tSftmeMIMO|10|20-470-282-5491|6073.68|. furiously ironic ideas +8079|Supplier#000008079|5AeAxE5FnSi0KIA31NidOiKc,sdMnFI7,W774X|19|29-844-542-3727|9318.44|y ironic pinto beans. packages wake carefully regu +8080|Supplier#000008080|1EvDBAhbdkqJMbgt8eK4O|10|20-784-526-3544|6154.33|use against the furiously regular pinto +8081|Supplier#000008081|a1xOE WQzn,nGC5kBzKRdWxI2cl0D2q|6|16-623-963-9556|781.51|along the carefully pending packages. slyly ironic depen +8082|Supplier#000008082|h5y59I2jUbycESOyonzpRUEhmkZjjR0vey9|0|10-293-680-6976|9766.23|e. regular, enticing dugouts haggle slyly ironic, regular platelets. +8083|Supplier#000008083|i5yvxpWnKvLsFPhxyySs20lfY 7Wj4|16|26-482-326-3165|8178.09|requests doze quickly. ideas sleep bold deposits. quickly bold theodolites use b +8084|Supplier#000008084|9Xof4U nuA1Wl|13|23-184-394-9728|2270.80|lly ironic deposits. fluffily even dependencies thrash. carefull +8085|Supplier#000008085|2Aukcd6dFlLAklZRs5ihM5ml Oa1s|10|20-752-548-4276|4334.21| bold accounts around the slyly regular accounts sleep furiously ironic co +8086|Supplier#000008086|FdwOQHWxgDPGZZcJTv,J9l|0|10-784-981-2463|1067.09|ackages cajole finally. even dependencies haggle furiou +8087|Supplier#000008087|wp6fVHmipj0u0zlVV4gs1c8X|8|18-577-263-9175|2567.15|ges. regular requests affix blithely ex +8088|Supplier#000008088|NM0PY,XgTAPeZGqREwzhl19k43P|3|13-809-992-3191|2126.65|t pinto beans alongside of the idly regular packages haggle blithely silent accoun +8089|Supplier#000008089|s1ttVqhyLcrm5Tt9305O,lTfBhrilkuLNpy |11|21-247-194-8015|1620.49|the furiously express asymptotes. special acc +8090|Supplier#000008090|eonbJZvoDFYBNUinYfp6yERIg|3|13-268-820-6097|8568.78|rts are fluffily. quickly ironic accounts haggle furiously +8091|Supplier#000008091|QqiLAtu2QGjDatQEXpiZvK3sjbnCqI|8|18-820-880-7277|1559.72|ctions-- even pinto beans doubt ironically after the blit +8092|Supplier#000008092|wlPDh5 R7RO8X98djBfFjeXiBzCb5TN4XI|1|11-567-871-6576|-809.13|ironic packages sleep carefully a +8093|Supplier#000008093|NSXCBrSry5eoO3mcGpE0mqqlT|19|29-470-701-4141|5900.19|rhorses. quickly regular depos +8094|Supplier#000008094|SNS6FCscBNZBFecA|20|30-465-665-6735|8588.63|ake quickly blithely ironic theodolites. quickly ironic ideas boost. furiously iro +8095|Supplier#000008095|aOhWtUYd,U7PQH8xA3DSDDJ5Tag|9|19-977-950-8899|5239.12| theodolites are carefully. fi +8096|Supplier#000008096| DEta7opwLcO7Db A|22|32-126-568-5282|7829.70|ronic accounts. accounts wake blithely. slyly final pains about the fluffily even +8097|Supplier#000008097|xMe97bpE69NzdwLoX|22|32-375-640-3593|9847.83| the special excuses. silent sentiments serve carefully final ac +8098|Supplier#000008098|IVt2aJgxZdqUGgtAjKS8tDXCAhuwztpTD |9|19-922-131-3314|4633.33|efully regular Tiresias. slyly final +8099|Supplier#000008099|Pqy18znGQbk|8|18-983-205-6959|1384.41|ix quickly. blithely even acc +8100|Supplier#000008100|ZUQ4uU,6D9UuL|8|18-756-778-6868|733.62|odolites. furiously express deposits haggle aroun +8101|Supplier#000008101|kkYvL6IuvojJgTNG IKkaXQDYgx8ILohj|7|17-627-663-8014|7950.37|arefully unusual requests x-ray above the quickly final deposits. +8102|Supplier#000008102|Uy zIV06io FkESw,NM40O,5Mr|0|10-313-738-3949|2845.95|across the accounts are carefully according +8103|Supplier#000008103|mRDFHLzmdMCL3vEiwfk|16|26-424-806-6121|1774.47|eans. furiously regular deposits use enticingly af +8104|Supplier#000008104|8gnTXvubyQ86LOyHvlr4Bx|11|21-278-672-8988|457.91|ites are blithely. furiously sp +8105|Supplier#000008105|Eb6N a4qQvPoRf2IYEjgif|20|30-955-663-3339|5103.72|fully regular deposits-- fluffily even requests detect. carefully even excuses wake clos +8106|Supplier#000008106|fr9i1v,cE4PDKiwEJ5sqX3q 6s8|13|23-981-593-2596|9189.14| fluffily. silent accounts nag +8107|Supplier#000008107|Bsx KttrbwpEiop0lN8V,BDopTnigk5mT8|16|26-758-284-7007|-9.52|cies. pinto beans use regular instructions. quickly bold accounts sleep ca +8108|Supplier#000008108|7CfpSkaAi6OjQsUmAvO2SR5G|19|29-560-695-8617|2863.01|kages sleep carefully after the furiously even deposits. ironic theodolites haggle against the sl +8109|Supplier#000008109|aWXR91ghUYjN4xaSINKO5FBKbRrNg2g3HH|3|13-660-760-8834|2562.33|uriously ironic pearls impress. quickly bold deposits across the slyly ironic reque +8110|Supplier#000008110|rsgtLhjl9jmUPTSuHXaHje,8K0Y2|13|23-629-103-7093|9069.84|uests cajole up the slyly +8111|Supplier#000008111| aQx5,WBgSa8gxF|5|15-482-752-3234|7875.53|iously enticing courts. carefully regular accounts cajole furiously. carefully +8112|Supplier#000008112|27EyHjRPpYhNu59pqunZUn O6q4XkgKyAF|22|32-152-656-5692|-858.57|oggedly regular ideas. furiously final forges sublate. unusual, final excuses gro +8113|Supplier#000008113|mmeYi0fc3cruQ1q R62oEivlhgImcyC74vqw|4|14-722-676-1270|-111.84|s sleep carefully ironic packages. carefully thin excuses sleep slyly across the fluffily pending r +8114|Supplier#000008114|drU1viKlfGpQNl9O4voVguy8PqzQkJBs WxbZ|13|23-550-951-4959|-818.77|usly even requests cajole accounts. final pinto beans use slyly. carefully regular packages wake +8115|Supplier#000008115|Q0pSKUUA9WOt51kzug2EW63QfQ|7|17-874-542-4311|8787.70| final packages against the orbits cajole furiously bold courts. blithely bold f +8116|Supplier#000008116|crxlJN4DXlUcX,zjjs|9|19-814-663-2270|6908.63|ng theodolites affix along +8117|Supplier#000008117|YKxTys0SBFcdjS2B9Dk8D GWwE4u|21|31-774-904-6432|6435.95|into beans against the quickly even accounts +8118|Supplier#000008118|6Z5HvonPjPm4 s02GJ3dqJyleY77iWwqUchwEsUh|22|32-442-636-9572|4073.54|, pending accounts thrash slyly instead of the quickly s +8119|Supplier#000008119|DsrX61kRbCIv1 oxhG4sqC30goVvQZXJofjKNcD|2|12-161-275-9492|9014.26|nto beans. even pinto beans are blithely regular instructions. slyly regular wa +8120|Supplier#000008120|R2WCFL3A2qo VhrjfAG|24|34-681-588-2364|8719.50|l patterns sleep bravely. blithely express packages wake furiously. even pinto bea +8121|Supplier#000008121|00X,v3WwAfxILD41|2|12-115-265-5557|4751.78|dolites are quickly carefully final depos +8122|Supplier#000008122|k4bZiP3oc53kf2Q26n3vDOKAGx|10|20-504-181-7411|1204.95|e. fluffily regular reque +8123|Supplier#000008123|RVD,e4jMiFVi4gCAa0oH,enjjIBuV1LL1Wzm|13|23-305-520-8530|9577.42| fluffily ironic requests boost blithely quickly ironic fo +8124|Supplier#000008124|d hPR6,ULpVJ bVqQprQ|8|18-610-876-7103|2839.88|sual instructions cajole. unusual +8125|Supplier#000008125|eKfrrdSQ1g|0|10-734-137-6224|9787.61|g foxes. express, careful ideas haggle bold r +8126|Supplier#000008126|sSdZzhtmGCaBsmEEtCK5b,k rF|6|16-198-508-4967|4469.20| pending excuses. deposits across the ca +8127|Supplier#000008127|u8tbsOPpf4v2R6EDD3p46Bv9gcc98IjUsAQVj|10|20-119-331-1918|3874.42|ndencies sleep fluffily about the gifts. quickly ironic foxes so +8128|Supplier#000008128|9GzISvSlOsHAsOJfvFo83FJY4J0yCJ59DkfTo UM|14|24-956-544-7533|6387.08|al instructions. furiously ironic platelet +8129|Supplier#000008129|,CvEmgUUKFaIDcaJJDlXeC2W|4|14-890-181-8886|7623.35|sits sleep carefully against the doggedly regular foxes. special +8130|Supplier#000008130|CGEPlXPSIQbwHvxQBMvaf|15|25-133-727-5012|6636.43|ss accounts. slyly unusual instructions cajole slyly. flu +8131|Supplier#000008131|BDWHW37vwkj8L|3|13-447-288-7295|7559.27| theodolites boost dolphins. quickly unusual +8132|Supplier#000008132|,3WTBTxiuJblYjk5y1f5vmqrHLZ5AiM|17|27-245-108-8714|7712.09|s are blithely special Tiresias. quickly final pearls hin +8133|Supplier#000008133|AB65mzDW8mtEj 3H2EzOjn0Kkxyy84jdkBuLLH4|5|15-767-207-4031|2099.84|nst the regular requests. blithely +8134|Supplier#000008134|LdwTvnRUjKNKM|20|30-972-451-1483|1965.62|lly express grouches above the fluffily bold theodolites cajole blithely ironic theodolites +8135|Supplier#000008135|mKPh z1XL4whRyf67Hq8D6UQo3yU|21|31-501-571-4779|1135.51|lithely alongside of the blithely bold accounts. slyly regula +8136|Supplier#000008136|kXATyaEZOWdQC7fE43IquuR1HkKV8qx|20|30-268-895-2611|8383.60|er the carefully regular depths. pinto beans detect quickly p +8137|Supplier#000008137|XQGrEhMwmAHLDSVjD9MDDyNn3HRupsNjFQC|14|24-810-371-6779|2295.86|ccounts haggle furiously bold +8138|Supplier#000008138|Fae9ymufCzbEDgm5NixFHfqbdg0apKo45i|9|19-981-528-7386|7409.65|aggle. final accounts doubt carefully. unusual packages nag fluffily across the +8139|Supplier#000008139|9vEm uTIDSQU|19|29-983-832-6440|1086.21|packages use quickly. unusual, +8140|Supplier#000008140|X2m71gmpJK4vv21hikqQE|15|25-198-149-1934|4919.06|e blithely special theodolites. ironic, special theodolites was finally ironic accounts. sil +8141|Supplier#000008141|u4oIjr7XbM|10|20-777-745-4221|7570.01|ckages snooze about the furiously regular accounts. carefully regular epitaphs mig +8142|Supplier#000008142|jChB0xH r3zey1Z,|5|15-250-408-4638|1246.22|posits. silent, silent dep +8143|Supplier#000008143|hvJ1Z1yinSBvw35fW6askj|12|22-902-774-3334|-758.26|requests. quickly even ideas about +8144|Supplier#000008144|Ps5Xk30eWJwIGp15JSA,VuzbN0Q8j|6|16-687-949-3464|2775.72|s! special accounts use carefully. fluffily e +8145|Supplier#000008145|M7SygdLkUNK7XTG|4|14-525-780-1695|9444.33|beans. furiously final platelets cajole quickly. furio +8146|Supplier#000008146|P4Lt0mMYW4ffCqv|5|15-526-243-1532|9945.69|uriously ironic theodolites haggle blithely ironic deposits. even Tiresias doubt carefully. th +8147|Supplier#000008147|yKh EpomKQcTuuAVwPZCMFFgySH3Q6WBQunQXZf|20|30-652-567-7912|-628.17|cajole warhorses; foxes haggle finally final requests. slyly pending requests alongside of th +8148|Supplier#000008148|O91VIF7M4v3|0|10-208-821-4600|236.53|rges. slyly special packages sleep furiously after the final, silent notornis. silent excuses haggle +8149|Supplier#000008149|2nexcjxZ1Eulp96j,gsdzGqMU CxlwGqp|23|33-179-797-9460|2275.66| deposits. unusual instructions along the warthog +8150|Supplier#000008150|dfFFxptWU9|3|13-815-421-9289|9058.38|ular theodolites detect furiou +8151|Supplier#000008151|jWufYDMz2IblV0WvMdmHhoWVY6kx,YzKz5PBPo|1|11-693-539-5693|2302.31|eans serve quietly according to the regular instructions. blithely ironic pinto beans integra +8152|Supplier#000008152|smw,crFy8U47L6NXee4bZ1O|9|19-728-273-5586|5971.55|te sometimes. slyly final packages among th +8153|Supplier#000008153|2XdXx OeLcw630eWmuJJxXkBJrqv6j39gz|14|24-838-589-7861|8812.26|ial accounts haggle slyly around the blithely ironic theodolites. fluffily even a +8154|Supplier#000008154|amCtx9qD7Dw0D4HkCoBNIpW9ZmM|11|21-895-596-3834|1758.01| beans. final, final gifts detect quickly ironic, regular dependencies. slyly pending dep +8155|Supplier#000008155|igMmKy1Dw4oaBJjw6GCMIsQCEeX15syOE|11|21-974-552-4133|9769.01|iously. deposits use carefully deposits? carefully un +8156|Supplier#000008156|jATP,fwaSoz|14|24-214-151-5679|-171.76|beneath the carefully careful gifts. furiously ironic fox +8157|Supplier#000008157|4Bvf6EF0Ne|6|16-453-869-2487|4146.17|ess ideas. slyly unusual pinto beans cajol +8158|Supplier#000008158|zCtCwYnWirA5cB7y09LYPoDdIRl9Em436kf|24|34-760-936-9540|8619.47|ect bold theodolites. furiously unusual deposits +8159|Supplier#000008159|z72u3aX9Z3|8|18-589-957-7347|6554.83|ns sleep blithely unusual requests. special pa +8160|Supplier#000008160| 741VGNtwIZgzKDMmVlvSqKfOn7RIF3V3uIP|2|12-957-291-3005|142.10|ans. quickly even accounts above the quickly silent pinto beans +8161|Supplier#000008161|hzp mnLVMLO8MX4XfWbNWQjPpD|21|31-554-844-1241|-864.79| solve blithely. furiously special foxes +8162|Supplier#000008162|XASpbn08mRV0kgHRmUSKx|21|31-869-421-7286|5453.50|ggle furiously regular accounts. orbits s +8163|Supplier#000008163|S1ET Bn umG|22|32-950-993-6015|3039.28|quickly express ideas. furiously final packages believe quickly ex +8164|Supplier#000008164|KmsTsj5juF7hhAVaVzphwTW fSPJEHyHT73ykNU|11|21-332-931-5327|6171.02|special accounts. unusual, ironic requests among the bold dolphins haggle quickly quickly +8165|Supplier#000008165|zVzaJslxtnVo|17|27-335-167-9725|6315.35|osits are according to the regular deposits. regular packages sleep deposits. blithely +8166|Supplier#000008166|VB3HlzNsC R9rUO5GQ|19|29-661-385-7757|1189.36|osits use quickly even ideas. closely unusu +8167|Supplier#000008167|Al4yNWbZwgzhGF2dY8alK5YQp4D|9|19-321-364-7058|514.98|ly accounts. fluffily final theodolites cajole after the reg +8168|Supplier#000008168|aOa82a8ZbKCnfDLX|3|13-305-682-9014|7195.90|press deposits x-ray deposits. bold, even excuse +8169|Supplier#000008169|Bob,3K z64MS1DlDao33mhc,G3XrKvFQPhykDOa|2|12-939-306-9972|5596.31|blithely even deposits. pending, pendi +8170|Supplier#000008170|bhdq6 Emicz,TUCBbQAj3dRLOKQ|19|29-619-637-7123|5194.30|he regular deposits. blithely even packages play slyly bold, regular instructions. slyly unusual r +8171|Supplier#000008171|3 QuVd1b4T|8|18-220-139-2382|2674.93|beans. furiously final platelets cajole +8172|Supplier#000008172|nkVf4sERrJKgUGkvamajC|0|10-463-235-9233|9564.51|ounts play carefully unusual packages. fluffily brave instructions lose. furiously f +8173|Supplier#000008173|PJqdiLWUB1gVSSf1twWpxzy7q6up|11|21-130-371-5053|5819.48| final packages. doggedly iron +8174|Supplier#000008174|QMLkO0TFlhZ kJhR|13|23-874-834-6213|5081.22|fily regular requests cajole carefully. slyly special accounts poach +8175|Supplier#000008175|oQI,qqyotirXMmymmKlc8Zg5x9 CGIVh3zp|5|15-135-641-1169|5093.22|refully above the final, regular ideas. regular tithes run. carefully express epitap +8176|Supplier#000008176|cQBg1Bxn5LxxR h ZnF SCIZO 0XZpEsP0vkfc|11|21-209-627-8129|8703.31|e carefully even packages detect against the final, unusual ideas. blithely bold +8177|Supplier#000008177|4QfZyUMRJ8nZAO8vlnU ,48vk8z|23|33-490-646-7141|7611.02|ly pending accounts. slyly even accounts was. slyly even ideas a +8178|Supplier#000008178|Kzuv9PrThE56fWmNR3UtwZAz|11|21-201-455-3427|8484.29|s. ironic, special deposits cajole carefully furiously final requests. pending packages sleep quick +8179|Supplier#000008179|R3uzpK70GMYBHHj,ZA|5|15-787-639-6185|455.34|e slyly pending accounts are ac +8180|Supplier#000008180|dHrm UdN1,LMt|20|30-595-896-3672|5820.04| slyly evenly bold ideas. blithely express requests about the slyly +8181|Supplier#000008181|m68VENaS vYdKj9ZtSkuI,tNUoN1zP|4|14-223-117-6427|3504.12|final ideas cajole furiously. fluffily express accounts haggle blithely. re +8182|Supplier#000008182|KGMyipBiAF00tSB99DwH|4|14-658-338-3635|8800.91|thely even excuses integrate blithel +8183|Supplier#000008183|NrwlwxBuTKL,hHok2UEkZQa15uLu6R,1aSva2|22|32-452-571-9926|8977.60|about the pending deposits serve across the final packages. slyly pending request +8184|Supplier#000008184|xeu,fDpPqAztNRAppFYm3wL8rdDJToHtZspsWy|23|33-458-474-7324|8617.96|ts. silent requests detect furiously. carefully special accounts haggle. accounts along +8185|Supplier#000008185|o5TsjX28c4 sSkcD4|16|26-842-197-5842|3041.53| against the carefully ironic request +8186|Supplier#000008186|y8mMog8pG6RD1kIAmcVA|13|23-874-297-4226|1.92|s run slyly above the carefully regular packages. carefully express accounts +8187|Supplier#000008187|hcgjLa 41gFrMl9t2kDPnRfB6Yo8cBW2e|18|28-281-514-1886|8255.70|side of the slyly silent accounts shall have to wake carefully according to the final dolph +8188|Supplier#000008188|GrdcssHqraa,9,AKACnH4JztJuTkfkdMQP4yja|15|25-339-751-3166|8550.04| bold ideas haggle carefully about the furiously express courts. furiously special pinto +8189|Supplier#000008189|5jpw7RetjcSszx9tXG a4RegDJzmFZj2Z|7|17-284-559-2583|5597.76|aggle. blithely bold theodolites use furiously. carefully final frets detect along the bold warh +8190|Supplier#000008190|6TOnkameTTcaq|2|12-996-476-1238|7616.16|ress deposits wake across the regular account +8191|Supplier#000008191|N,Rnn8gBGVGmi|0|10-617-686-3837|6818.24|enticing pearls. blithely final courts wake slyly. instruct +8192|Supplier#000008192|WBaEm7WMwk3iomHxE|6|16-137-666-4879|5237.00|old packages. even theodolites sleep excuses! thin, b +8193|Supplier#000008193|g1IFVx2xk 2HoL7yafzOT9|17|27-111-470-5762|5292.85| pinto beans. bold, slow accounts wake careful +8194|Supplier#000008194|kZzNz1UY4OOn5pt5bGhBipEwG|22|32-566-988-9483|8869.34|yly silent packages play furiously bold, final theodolites. furiously unusual dep +8195|Supplier#000008195|s9TsiDVPstck|14|24-383-556-7475|4037.87| express requests sleep furiously ironic instructions. carefully bold dep +8196|Supplier#000008196|yghzflIa9Slg3r8N8OOSoJ4V|15|25-322-540-7522|3452.34|l dependencies. final instructions are blithely against the care +8197|Supplier#000008197|u1PZvjDFfDzxU3F1ZlSA03gFDP11s|8|18-550-473-1416|7447.69|. bold instructions use special, silent requests; quickly express deposits sleep furiously theodo +8198|Supplier#000008198|xQS Ge363wHc|11|21-186-724-1911|2388.26|cajole furiously. carefully unusual accounts boost blithe +8199|Supplier#000008199|mDuP66PNn,oiex1ssUJWkPqKoFvuGc0y7l|8|18-751-742-1146|9380.01|y regular excuses. fluffily final packages doubt slyly according to the deposits. ca +8200|Supplier#000008200|KhbY27dqsfm2AAIEU9xCt8Y|4|14-179-315-7099|6945.76|thely even decoys haggle blithely since the ca +8201|Supplier#000008201|F6SAIQpPEMTfF0t3|6|16-757-981-2245|3477.67|he quickly final packages nag slyly among the ironic, ironic +8202|Supplier#000008202|iNrnLVPL1q9qiY4nTcIVSh|13|23-831-880-9272|4262.91|e blithely carefully ironic reques +8203|Supplier#000008203|nXB7udOM3wq|19|29-780-346-3309|1796.48| final requests. special, regular excuses are +8204|Supplier#000008204|bxXHyCjF89VN,t14tfsH1q|21|31-504-959-7868|7404.72|sly final requests boost furiously quickly ironic deposits. slyly final ideas +8205|Supplier#000008205|M6l7kygwSgJHhTv9yQKO3dmjnmWVGpwXN,sHKH|15|25-727-515-1949|6247.42|st the carefully pending packages. carefully even accounts s +8206|Supplier#000008206|9HpptwDlEUbrXKmNd0A8ugvrIU|8|18-603-201-5658|3466.66| blithely bold excuses. furiously silent deposits boost quickly carefully unusual pinto be +8207|Supplier#000008207|z2X5MtjZpeDWyT6dgmt|10|20-919-562-8925|6122.45|quests. even asymptotes sleep evenly along the ideas. packages +8208|Supplier#000008208|cc6IsCnCP6aaAez|18|28-521-874-4048|3984.77|he ironic, ironic courts. quickly even instructions detect fu +8209|Supplier#000008209|VMqlFV9CGuHmnSe1o5mI35zt2ilZB|4|14-297-573-4300|1992.57|nes against the unusual, bold ideas +8210|Supplier#000008210|R1fbPmLFnNt5YsAV8wImEy1j7F|20|30-112-400-5670|3065.05|aring packages after the regular, ironic requests sleep +8211|Supplier#000008211|3KbqokDH,3NtaxcS|6|16-479-596-6995|-777.28| theodolites. special deposits cajole. fluffily ironic deposits cajole furiously. rut +8212|Supplier#000008212|5,7CjHr2y7F8MHWN4orESPcfkKYbMzK70Y9|24|34-168-546-8403|2114.02|es. ironic, special theodolites haggle slyly furiously unusual foxes? slyly regular deposits nod +8213|Supplier#000008213|YhdN9ESxYvhJp9ngr12Bbeo4t3zLPD,|13|23-197-507-9431|9296.31|g to the blithely regular accounts! deposit +8214|Supplier#000008214|ooZVeXxMjxCk6PfCCaDVt AX9v94J52Mi34W56Q|2|12-745-685-8032|8112.16|accounts affix stealthily against the carefully final deposits. quickly unusual p +8215|Supplier#000008215|o2ereW6g3ev98WS2oL|4|14-256-501-5320|5157.32|nts about the idly enticing dependenc +8216|Supplier#000008216|jsqlyr1ViAo|10|20-224-305-7298|8031.42|to the carefully even excuses haggle blithely against the pending pinto be +8217|Supplier#000008217|,XXvKzyN,KdMbMo9Uy9sEt8Fethskq|5|15-856-254-3708|1104.95|unusual, regular pinto beans at the furiously quick platelets use fluffily at the quick +8218|Supplier#000008218|7hQ5Yk,vC6a C1Kir7YhdN9E8n2t8K70|0|10-985-426-1248|909.34|oldly packages. bravely even packages promise ironic, idle requests. furiously unu +8219|Supplier#000008219|HQO,s0Jm6URO9TO3hdovC P3e1kRyd|10|20-377-311-2163|2179.98|ously above the furiously final +8220|Supplier#000008220|GgePLLEUcy8,1TEct|13|23-943-742-4081|6863.83|inal attainments. quickly f +8221|Supplier#000008221|yjDAMaDwWgyUfQWT3bys4rz6iipm0SeL6|17|27-539-310-7067|1448.20|tructions. slyly even theodolites cajole fluffily. silent ins +8222|Supplier#000008222|sjwxxzvfFA,ByaTlqG6oFGbN1|15|25-641-945-4668|5806.62|blithely quickly regular requests. special, pen +8223|Supplier#000008223|5TX5hx5CQMLXmGQ5cGndcdKe4pcHeV6|1|11-512-882-5007|8817.99|efully pending accounts sleep fluffily ideas. blithely ironic sauternes along the s +8224|Supplier#000008224|TWxt9f,LVER|3|13-845-388-7056|-989.86|otes use furiously carefully even theodolites. even, regular pint +8225|Supplier#000008225|i7mYUAWhVs EOVc|0|10-617-946-4318|4364.23|idle ideas are furiously? packages above the exp +8226|Supplier#000008226|cIoqqdt1e7scqwO9LQZ33dMlGt oHx|5|15-173-793-7725|-453.06|y even instructions wake? deposits cajole blithely carefully special reque +8227|Supplier#000008227|UzuOMkIwu58C|15|25-468-486-9857|6274.62|ic accounts are quickly among the furiously ironic instructions. slyly fina +8228|Supplier#000008228|WyCUGJa30D|12|22-915-176-4354|9522.75|deas. regular deposits alongside of the slyly thin packages cajole +8229|Supplier#000008229|HsplGrXWd0CjRT|9|19-131-649-1409|7970.34|ely regular dolphins around the bravely regular platelets dazzle in place of the slyly regula +8230|Supplier#000008230|NbuT6,ZzZni0|9|19-446-942-6869|8531.92|ly. blithely special platelets wake furiously careful +8231|Supplier#000008231|IK7eGw Yj90sTdpsP,vcqWxLB|3|13-323-448-7114|9123.73|ounts before the slyly final requests grow fluffily unusual requests? +8232|Supplier#000008232|TcdVP5HIwxdjfZlqxTzbY39gXv0IWN6Np|17|27-214-561-8532|542.90|ironic packages along the dependencies wake against the regular theodolites. blithely unusu +8233|Supplier#000008233|pf5SN,fa1vryDFEENBWVivV7WZ|12|22-197-762-2605|3538.43|accounts cajole blithely after the carefully ironic deposits. realms nag car +8234|Supplier#000008234|IbPDbICEvbqGQF |20|30-881-743-9719|2460.91|express pinto beans. carefully pending packages use alongside +8235|Supplier#000008235|TjVWq6bTdGJB|21|31-700-234-5513|9402.25|s packages wake ironic requests. blithely final depen +8236|Supplier#000008236|3P4fq3KXTHDlBl|2|12-615-406-7253|6640.60|e according to the carefully expres +8237|Supplier#000008237|J5ZD6nYd9Mf1FLGDwEFRWhP|10|20-513-534-4393|541.97| cajole carefully slyly bold requests. fluffily fi +8238|Supplier#000008238|LcAJrR6RFABlXBmXl7Kcpm|5|15-575-214-7975|9906.80|s. furiously final theodolites cajole blithely pending pinto beans. care +8239|Supplier#000008239|aKtYP9h8cUx4dfwYPrMT3IH2ZxCCERDPSyGrasMi|5|15-106-898-3843|5642.03| accounts nag fluffily after the slyly even instructions! silent, even dinos boost +8240|Supplier#000008240|kFOt9kVFENwW|24|34-938-160-9730|3625.42|nstructions poach carefully final platelets. spec +8241|Supplier#000008241|e1wpbDnuCl8KaoMAXqG1HooMMDIVVuxvyg2tk|2|12-366-481-6221|1356.16|e of the even packages haggle quickly about the silent asymptotes. furiously special packa +8242|Supplier#000008242|nWK3gGNbv815 hQkLupnBYtr j|9|19-964-284-8680|9940.38|braids. requests detect slyly about the slyly unusual theodo +8243|Supplier#000008243|2AyePMkDqmzVzjGTizXthFLo8h EiudCMxOmIIG|3|13-707-547-1386|9067.07|s must have to cajole blithely requests. ironi +8244|Supplier#000008244|JxgZVJH6BOBa9ErhOUAham4JTS|12|22-130-530-4865|5288.93|furiously bold excuses serve furiously. +8245|Supplier#000008245|vIiJoknAO qdcpB1KhlxpIe q|14|24-757-792-1443|9089.82|y express asymptotes haggle deposits. final deposits boost. quickl +8246|Supplier#000008246|9kUVMjClxx3L1OU7nLg7TV2c1Z3Ki2U|0|10-841-520-6240|2606.89|ns sleep furiously along the silent excuses. deposits are carefull +8247|Supplier#000008247|BzpxpWYeZheKz|2|12-903-951-6466|-984.75|cally. slyly express requests integrate blithely +8248|Supplier#000008248|RV Kle n81znB XjxMnKuyu|10|20-799-147-5925|2217.20|ular excuses boost slyly ironic instructions-- fluffily ir +8249|Supplier#000008249|PwUjvlMk y72zaMRtZQ8trbCmu4j|21|31-104-853-7722|5410.75|en excuses boost final, final pearls. +8250|Supplier#000008250|6A2tZv7rgXema1Y7Ie4Xd1DMVV|19|29-565-237-5252|-488.86|ly pending accounts. ironic, enticing excuses grow. furiously regular deposit +8251|Supplier#000008251|2cXnsQkTZiX|8|18-536-106-2778|9626.81|deposits boost blithely even r +8252|Supplier#000008252|fuC55,TL068J0CN7oe|8|18-206-889-3808|7931.18|entiments haggle carefully about the quickly unusual theodolites. s +8253|Supplier#000008253|6XM4Dhnpm,|11|21-424-396-8930|-568.05|ges are fluffily. accounts sleep careful, ironic requests +8254|Supplier#000008254|0lLFVM oUMDv|12|22-405-547-6938|9732.48|carefully ironic packages. carefully unusual platelets are quickly. +8255|Supplier#000008255|qU4odHKWII0zA UP7a0Sfm9yat|12|22-188-396-2763|1900.72|gular deposits are slyly. quickly regular packages thrash carefully fluffily unusual t +8256|Supplier#000008256|Pq,9XP5qIzmtEuEdeopKSlUCM21r3 |12|22-611-120-5704|-602.68|de of the ironic, final theodolites. carefully regular pac +8257|Supplier#000008257|gA20SjV3BGrB,cowC,hXTXtppX|5|15-583-943-4692|-270.33| slyly regular requests boost fluffi +8258|Supplier#000008258|JaTsYwI4yFu283q83|8|18-239-643-9980|8442.70|ions. thinly special accounts +8259|Supplier#000008259|f8Absnei3As,QB1LeyH,Eufh8mGO|5|15-505-907-7716|5533.77|otes wake among the quickly final instructions. blithely ruthle +8260|Supplier#000008260| kGCbbbxxw9QWe2ucX1wplGB|9|19-225-631-5870|-914.98|s are furiously special instructions. slyly express theodolites sleep furiously +8261|Supplier#000008261|OE8VCFunRj1dNf|13|23-540-992-1108|8795.09|ent accounts boost fluffily pinto beans. requests detect carefully. even, ironic accoun +8262|Supplier#000008262|L2I4wL9HgO82PVzZ5nRE3HtF2H6mq|18|28-100-281-9027|-939.25|ironic accounts lose carefully ironic packages. final, regular ideas cajole. furiously even t +8263|Supplier#000008263|QDeCVYC1QjgBvY9U4PPOtTInNJG3|4|14-811-879-2139|8014.44|eep about the carefully ironic pinto bean +8264|Supplier#000008264|12qflBU3Y0xA5bzGZWCP O|6|16-668-511-6480|1781.57|uriously blithely express +8265|Supplier#000008265|Ig x7 AsU,sda27UWa|24|34-831-958-1126|9779.85|ally special ideas are according to the furiously +8266|Supplier#000008266|xiw0dcHqv,i2aqjcBp1VL9UD5x ITQo1|12|22-815-711-3857|6031.65|unusual accounts. quickly even requests above the blithely unusual requests cajole always a +8267|Supplier#000008267|uuF5OOYBlkw2IGRp|22|32-867-184-6539|6982.89|. furiously express excuses nag carefully. furiously bold accounts sle +8268|Supplier#000008268|R49VIOrmA,W|19|29-294-542-5908|488.75|egular deposits hinder quickly +8269|Supplier#000008269|8RHv tRTU0fTx0ykFO0U oFZ2tl|23|33-196-886-3193|6350.31| even instructions along the blithely final requests are blithely according to the furiously +8270|Supplier#000008270|2dbla7q9MZLxCts3TcSAVyfm3WLlc64s|4|14-617-863-1649|5628.25|equests. fluffily express deposits among the ironic theodolites are silent requests. quickly ironic +8271|Supplier#000008271|NBfhLVLep0z99Pxq8Ee4eEsjx sb7V ySe|13|23-347-681-3234|3016.03|s. carefully regular foxes nag s +8272|Supplier#000008272|rjuHAwxY92lo W5nTHeuKq3H|24|34-985-865-8643|426.57|ts will have to nod enticingly platelets. s +8273|Supplier#000008273|noqA2JZJNJsWlfdDa|17|27-942-829-9849|6284.19|efully quickly express req +8274|Supplier#000008274|Cp,iOqOr70NImmwz fCeIV|18|28-311-312-6708|5440.75| breach. slyly final accounts haggle. express packages cajol +8275|Supplier#000008275|BlbNDfWg,gpXKQlLN|3|13-427-631-2658|8695.43|ss, regular foxes cajole blithely carefully special ideas. slyly r +8276|Supplier#000008276|CE5qZdVK1Q7ircuWBKghM|19|29-330-707-1036|7072.79|ctions wake carefully carefully regular ideas. pending, final packages can n +8277|Supplier#000008277|m2M7W6isuP1ejlBYO|18|28-112-434-6207|4403.51|gular packages. slyly ironic courts integrate slyly along the slyly final pinto beans. furiously +8278|Supplier#000008278|NfeMw6gfku iaMkXfd5Vr9|13|23-369-577-7124|-223.13|s use carefully across the excuses. excuses among the t +8279|Supplier#000008279|s8Cpz6OlOq3tpxwYV|8|18-408-246-8407|9806.44| furiously after the foxes. bold pinto beans sleep ab +8280|Supplier#000008280|bxZrcDyQZ66Z|10|20-916-186-7954|1908.98| the carefully express depths. special, unusual pinto beans haggle slyly blithe accounts +8281|Supplier#000008281|W93DK40wkZHj|16|26-947-112-1111|6203.51|ding instructions sleep slyly express packages. slyly unusual ideas along the quickly regular inst +8282|Supplier#000008282|eBrmBWMgEcjgyQCITyRrhCn3h317JbA|1|11-231-892-1749|2399.60|ajole blithely bold deposits. furiously bold instruc +8283|Supplier#000008283|PZyV6hhpTTMT4PxDakKN|20|30-982-995-2862|2044.80|olve quickly bold deposits +8284|Supplier#000008284|agYEWUArk7a Sp6|9|19-421-597-7278|889.08|c, ironic asymptotes. stealthy requests boost. regular ideas affix +8285|Supplier#000008285|QUOVvvbmkdWhw|4|14-192-667-1976|2706.96| quickly. blithely even foxes across the bold grouche +8286|Supplier#000008286|mUejcmmOHeh6Kwnj7lLlWSIT6vJPmohVnLhc 2n|1|11-957-208-2220|8768.76|ymptotes. slyly quick deposits could have to haggle closely according to the regular, ironi +8287|Supplier#000008287|q6Vpk7U Dv neLoruc|17|27-570-613-7931|7589.85|ages affix quickly carefully +8288|Supplier#000008288| rsTQTN7NUo31tpz|9|19-122-725-7808|4725.46|ously furious theodolites haggle fluffily quickly final attain +8289|Supplier#000008289|fFsGY51lOsR,ckGbgfx6N2uH2IqF|5|15-203-606-1989|9801.63|fily bold foxes. final dependencies about the special b +8290|Supplier#000008290|Sv0nK5,G331Y,jY7cL|21|31-144-288-4077|9117.24|s the blithely even realms. slyly u +8291|Supplier#000008291|LWwetCKCFe 56Gq4xrbGrO4C6J1qu,FiS4|5|15-297-514-2676|1511.91| bold theodolites sleep carefully; blithely final deposits print slyly along the quickly even +8292|Supplier#000008292|paQEp sV7TNxy5BkskJTZQkV|5|15-660-195-3804|9568.89|quests nod. quickly enticing sentiments boost silently. slyly even accounts +8293|Supplier#000008293|8oP8yCvFNU2JiQCEtQ6snLU17RK6B4KgVQNm8Zm|6|16-455-849-5265|2394.55|ross the ironic pinto beans haggle packages. packages u +8294|Supplier#000008294|mcspZQuGVa60ELStjl|5|15-390-728-6058|5172.99|ackages. fluffily special deposits +8295|Supplier#000008295|cjdMDF4Eri7k4,ofFW5q5TcRbLV|4|14-875-835-5027|7523.14|ffily ironic theodolites. carefully regular packages cajole carefully accounts. furiously reg +8296|Supplier#000008296|R5ks4rF6wCUyGQvM mZ9JddqlFjc4|0|10-810-470-9957|5442.87|y after the ideas. slyly p +8297|Supplier#000008297|0l2KCe1ocMaACSDEQdWAwH0K|14|24-151-249-5909|255.79|kly across the bold ideas. carefu +8298|Supplier#000008298|0XcmXxswvFHJt1TahdB88 |10|20-329-384-1312|7059.26|ckly bold packages. unusual pinto beans sleep. final, regular r +8299|Supplier#000008299|ktZLjsk2yyC84focDK6sGtx5Kx Rmje|1|11-789-363-9257|8081.75|furiously fluffily ironic foxes. ironic packages after the slyly +8300|Supplier#000008300|YKoriorl0muA6WMqcpZnKI7a2MiUKKywJJa8gtS|1|11-925-285-3502|35.29|lay regularly after the quickly final instructions. regular accoun +8301|Supplier#000008301|8fT54G,7Oz3TuT6FBePq1g6C9J3kqVAps9kiB|20|30-544-874-4257|7992.38|lyly regular excuses haggle furiously. ironic pa +8302|Supplier#000008302|6zp1mzBds96Z1TfXRnd3z6oPHGBsAJv2K|16|26-172-616-7062|5005.53|quests. ideas lose carefully according to the regular accounts. final a +8303|Supplier#000008303|nulXQplnGSS3LXsCo4Ml8jpR8KDGI41lku|11|21-980-501-3504|8944.42|ronic deposits across the furiously s +8304|Supplier#000008304|l3GK4X2Oi3Jq4rYtt2Af241E7BQ7m rla IE,|11|21-689-436-5771|859.05|ons. requests haggle quickly about the furiously special deposits. express theodolites cajole alon +8305|Supplier#000008305|fo5QkANtn00gbgiailVlt|12|22-135-901-4717|3952.14|inal instructions. furiously regular re +8306|Supplier#000008306|FKeicwk3u7yGVU QNVwAZ5UKCKCbVhN5H16ymw|21|31-980-589-8843|3612.65|es. packages sleep furiously according t +8307|Supplier#000008307|n8Co2Ch2cdyLsEjri9,Ej34w5|5|15-906-908-2980|6756.36| deposits alongside of the slyly even escapades unwind against the quickly even deposits +8308|Supplier#000008308|hC2HedXjyfcjz|7|17-396-297-8631|-838.12|es. dependencies sleep slyly above the bold packag +8309|Supplier#000008309|6P,FQbW6sJouqunvttVO6vEeY|21|31-805-993-4638|2594.38|e special pinto beans. blithely unusual epitaphs haggle. express, final requests wake c +8310|Supplier#000008310|zBVFtBCt3Mwd|20|30-527-792-4296|8641.18|e across the carefully bold theodolites. closely even orbits hagg +8311|Supplier#000008311|TwY,cwE2OL1LtXGbFEDjpgZjox0|1|11-549-774-3596|8294.32|ously regular dependencies play ideas. doggedly regular accounts sleep. asymptotes +8312|Supplier#000008312|eWIaaOCmy4|7|17-608-504-1498|8697.69|ding to the quickly final instructions cajole carefully among the asymptotes. final, pendi +8313|Supplier#000008313|Y5MCEhDhda0qzg 6Y,1SYs|19|29-851-635-4723|1122.36|s. slyly ironic packages +8314|Supplier#000008314|K8JixEKRhWkJ|22|32-990-782-4027|7509.65|even accounts lose. furiously bold re +8315|Supplier#000008315|oLcTtlALth2bn1udMeHSUWqq0,Kx|16|26-812-558-9274|9750.25|he furiously enticing instruct +8316|Supplier#000008316|M,BrvLmLtbbDy5O|19|29-143-826-7135|3573.92|counts must have to nag across the carefully unusua +8317|Supplier#000008317|9p CRgS56heCHIm|10|20-487-380-7004|4845.85|ously ironic packages cajole quickly +8318|Supplier#000008318|n1Qfh7vRHj59woCzdKwj47deVisSkDqaz|9|19-713-818-2608|9145.11|le above the ironic, bold pinto +8319|Supplier#000008319|Kqb90EEnlFfuQQhtmlIgm|22|32-582-415-4055|7868.87|carefully express ideas cajole among the silent, silent ideas; dogge +8320|Supplier#000008320|FB TdvQvk36|11|21-388-687-9284|8142.90|accounts wake about the slyly bold platelets. furiously ironic foxes doubt ab +8321|Supplier#000008321|MxbBDSqft2YviX|11|21-888-240-9800|4735.48|endencies. regular dependencies according to the special tithes wake carefully about the +8322|Supplier#000008322|9CyczpMjlNd0fWUa6XwNsgP5DJP5ufbQjbCvAI|11|21-155-467-2532|2244.85|y even courts. slyly ironi +8323|Supplier#000008323|75I18sZmASwm POeheRMdj9tmpyeQ,BfCXN5BIAb|3|13-143-631-3790|8187.92| dependencies integrate quickly regular sheaves. r +8324|Supplier#000008324|SI0AP6KavzSiiXdQ2lDiWbmJsiCh4fA3xI|10|20-340-218-6944|2437.77|p carefully above the boldly express +8325|Supplier#000008325|LR9350W4h,wWxztR9fztXUuwgQmYj2LUOl7I|7|17-379-143-5836|5582.17|luffily. dogged deposits among the permanently ironic platelets boo +8326|Supplier#000008326|guSNtiNA5pR1P5b4scyzK|2|12-631-356-6254|9515.39|ly ironic requests. furiously pending deposits use even deposits. express, bold +8327|Supplier#000008327|7Htb2szDxgGHLwL8IwkGtAE|15|25-974-585-5154|5296.80|furiously. slyly regular warhorses haggle slyly according to the final excu +8328|Supplier#000008328|TsfSkDtBVRLwIanb|5|15-432-476-4924|9357.64|fully special pinto beans detect stealthily even courts. blithely ironic pinto bean +8329|Supplier#000008329|iQC4 odmTUM9ggXcrP1gB|0|10-705-883-4433|4925.55|press dependencies cajole furiously bold requests. carefully silent dependencies nag. blithely ir +8330|Supplier#000008330|i,9fo6W58P09vbUIdNu|7|17-521-991-2604|6774.22| are silently. slyly ironic tithes promise daringly across +8331|Supplier#000008331|uDK3OK3Hhsny79E|20|30-537-147-1978|2410.09|ending deposits. carefully bold accounts wak +8332|Supplier#000008332|e6CxifEgyTQi|11|21-836-667-3398|2365.08|ar, idle packages are carefully across +8333|Supplier#000008333|CzGWB3whG2JNN0N2JyqJ9GAj|20|30-253-808-6437|4849.69| after the express requests +8334|Supplier#000008334|0DfcA0xgrNaG9ekiQyegZCk|22|32-322-805-8991|7766.29| was. express deposits nag +8335|Supplier#000008335|dsmwQ616A2Fg7frTRzWtXys54mXmb|0|10-930-286-5692|49.52|lar theodolites detect carefully unusual, bold dolphins. dogged, pending accounts cajole bravely c +8336|Supplier#000008336|cjsx,2LLeC|23|33-807-277-8743|5214.28|uriously regular accounts snooze blithely. slyly regular pinto beans cajole ag +8337|Supplier#000008337|2nQZobZMJsnNX41DM8SVyMtxAqxUhSFt|15|25-620-941-5707|8453.20|luffily even ideas sleep bold packages. +8338|Supplier#000008338|xLHaCbSSKFPVRR1|8|18-663-384-6371|6007.38|ts. express packages haggle ironically for the unusual theodolites. slyly special a +8339|Supplier#000008339|uWw8 P6u,S|21|31-668-583-9968|6002.43|furiously pending instructions sleep slyly dependencies. blithely even platelets sleep amo +8340|Supplier#000008340|kWKYCAcZP8nnnUeiPQBVCkNZIwA,g|12|22-465-488-3487|-81.70|gular instructions use carefully bold deposits. quickly exp +8341|Supplier#000008341|pX4SIJwcOqJ3zqVimzHhGJHR|13|23-288-860-2159|-243.76|al theodolites haggle. bold, regular instructions main +8342|Supplier#000008342|q6k7RhZrKVhkVq,bWv8PnIGSzkUjS|16|26-898-246-1774|4081.82|ly final packages boost slyly regular pinto +8343|Supplier#000008343| BbHngAVqj0J8|21|31-104-719-4677|5138.73|ar, even packages are quickly across the +8344|Supplier#000008344|5AYCoZKgsB3byN7xd1HCXv1N|14|24-788-298-8071|3636.67|r requests cajole. furiously regular theodolites haggle qui +8345|Supplier#000008345|ktT,Jkcp3axEJc1homJQi|12|22-742-294-5567|3555.51|riously slyly regular pinto beans. blithely special requests sle +8346|Supplier#000008346|Yy3VnDGcUelIgMeO0Jq|17|27-802-811-8509|3485.17|nto beans. blithely bold requests sleep sometime +8347|Supplier#000008347|2XHTRyu8x 2S6UTEvtT|8|18-177-871-2624|9244.61|ic, express requests. carefully final requests haggle. +8348|Supplier#000008348|jSE4BpESdgd2lgFkSP3BZKDKR|23|33-360-581-5299|8226.78|ounts haggle bold platelets. slyl +8349|Supplier#000008349|8Hkx1IDd0mZCTX|21|31-539-655-7949|6480.13| dependencies eat furiously special requests +8350|Supplier#000008350|Xl,OsoSo,oBKPqJ60Zg6|1|11-100-393-5411|9571.65|sits haggle blithely bold theodolites. furiously pending foxes believe slyly quic +8351|Supplier#000008351|N8Zid9nLWLAtcpswmFnsq UN9ET5Hlc0sYI0C|6|16-636-652-3301|4841.43| ironic packages. dolphin +8352|Supplier#000008352|53CWsOambFkFHF9JVpRGIsagsNwPC|0|10-184-453-4820|880.47|sly final asymptotes-- ideas cajole. fluffily express theod +8353|Supplier#000008353|EEFOe4Er5y5sWt0qv|23|33-218-570-1543|982.59|except the regular deposits cajole quickly during the +8354|Supplier#000008354|Ck4DSBX,6WQh,2G17wStiefhq bJP cztbhh42|20|30-908-905-2177|1328.04|fily according to the blithely pending ideas. furiously final foxes thrash. slyly specia +8355|Supplier#000008355|nvgZOmTqsyNs1nlHOEYgk73J|24|34-653-876-4431|-667.19|as among the final, bold instructions haggle into the quickly ironic asymptotes. regular reque +8356|Supplier#000008356|uSKxQwedxMwkoo73DGqj4duR33qRQl2LJyLxY6z|6|16-307-991-1055|4478.80|s integrate quickly against the evenl +8357|Supplier#000008357|NxR4B,oub4GdMpx8lVaR|16|26-975-347-3117|119.04|sts haggle. blithely regular packages against the regular theodolites +8358|Supplier#000008358|CmnoVXcLske7k|19|29-983-473-3523|7710.21| carefully final platelets use. even foxes wake. furiously pending accounts can are near the +8359|Supplier#000008359|iDql4 k64kPv0o0SauFwWINoM,2UJAdQW6YHveaD|0|10-640-841-6879|3596.73|old instructions cajole. permanent accounts cajole. fluffily express foxes sleep slyly hockey +8360|Supplier#000008360|jeLcK7YUbZzQuwD9a,1F|23|33-764-209-1683|3220.82|regular grouches engage slyly unusual foxes. slyly pending packages kindle carefully acr +8361|Supplier#000008361|5uk8Oml Sd tvBWCcagUX7QA2isOxsU|2|12-123-133-7850|-698.87|ickly final excuses. blithely brave dependencies nag carefully regular, u +8362|Supplier#000008362|DeCSqduLu2JE9brcwCj|22|32-663-509-7912|22.79|theodolites. slyly bold sauternes after the even theodoli +8363|Supplier#000008363|7tLHZjjFDr,nLXiG1C3hBv |12|22-986-422-9102|6907.92|thrash blithely requests. carefully final frays haggl +8364|Supplier#000008364|X yetf1hqluKnG WOKCUTYGXPaz3Z|17|27-190-288-3784|5506.97|ests. furiously regular dolphins after the quickly close excu +8365|Supplier#000008365|X7Y78XFI4x|22|32-249-537-7838|-349.29|symptotes among the furiously pending ideas sleep blithe +8366|Supplier#000008366|h778cEj14BuW9OEKlvPTWq4iwASR6EBBXN7zeS8|3|13-926-224-4770|1359.75|s haggle carefully above the final, express pinto beans. unusual, regular foxes boost +8367|Supplier#000008367|W,X3iqYN7crcqnjA|0|10-990-192-1806|4353.83|ending packages doubt carefully alongside of the furiously bold packages. special, final cou +8368|Supplier#000008368|rjkUjGUP2MeEiitkk5uJfHo6uje5ZuxoFd1j1|1|11-357-976-6093|797.45|d warthogs. ironic package +8369|Supplier#000008369|F7J5M18silHzUntuI7cKOZPXnWLlYB|24|34-983-960-8783|2737.31|dolites. even packages thrash. carefully ironic pint +8370|Supplier#000008370|S72tfWosohkej7FjUi9z5J11ultFFv|10|20-182-281-2705|1985.22|luffily toward the furiously ironic d +8371|Supplier#000008371|3w07P9otks,mGJ0BTuXxd2tiiR2F6HO|9|19-542-303-5317|522.97|st quickly furiously bold courts: f +8372|Supplier#000008372|tM 6NSmPysNXtzW3y2evAZUVvK vfBO|0|10-564-193-7747|9488.65|ide of the pains. carefully unusual dependenc +8373|Supplier#000008373|fKWgRbTLSos|13|23-453-151-3284|6818.44|ntain slyly alongside of the slyly even foxes. fluffily ironic accounts nag slyly slyl +8374|Supplier#000008374|rIi5HjxjPxn3aNJpBQIQ9qPmoa|2|12-389-750-2367|8473.90| final foxes. final packages mold slyly blithel +8375|Supplier#000008375|BwYSHWOUUrV0MX262zGz WUX0BE6hV6Gi|23|33-250-729-7072|1875.11|ts. blithely silent excuses cajole furiously about the slyly express somas. carefully +8376|Supplier#000008376|gH9mVai3bfWr|20|30-761-608-5360|8918.94|e furiously accounts. fluffily reg +8377|Supplier#000008377|,Yk0mflw2LqQCTxMYR sU2juj5DorUAG4w6i|21|31-354-319-5782|1687.28|sits sleep furiously pending asymptotes. carefully furious ideas cajole. blithely regular d +8378|Supplier#000008378|yuzY2wQ KN08rNGzehM6Cg,yPiqyigN7waPc3r|17|27-270-336-4501|9121.29|tterns detect quickly instead of the ironic accounts. deposi +8379|Supplier#000008379|GeHYbtj,EQacbKAWsyz7SilH5BJ7cY6Pq|0|10-247-201-5353|3558.61|ts! even instructions sleep furiously. instructions sublate blithely. carefully even instructions b +8380|Supplier#000008380|UAouU3b4cqhOKrmNs|1|11-258-914-7061|524.03|y regular foxes. regular requests impress fluffily across the fluffily express deposits. +8381|Supplier#000008381|UNI,edNdjZ9IT8DfyjzQqT0va4|12|22-630-723-6868|5359.40|g to the furiously final requests. even, final deposits do haggle fur +8382|Supplier#000008382|RQIy,ZRmPQVZXu89XCwT8g|16|26-297-975-3183|5466.96|refully even platelets haggle slyly +8383|Supplier#000008383|zTLHVaxfp1F8aVbKyAWl2b10jVtROE|15|25-539-565-5305|9177.81|ncies. furiously unusual ideas nag thinly blithely regular accounts. quickl +8384|Supplier#000008384|,fHBV2rnJ6qMS8VQmfJKCjM2o|15|25-508-924-3702|7503.64|ly alongside of the sheaves. ironic +8385|Supplier#000008385|t2 p6wnZ6zfpc XO|5|15-881-456-6267|7694.38|ctions. carefully silent multiplie +8386|Supplier#000008386|r1wxuuhiwKtWDu7q5he,|21|31-893-272-5894|1531.26|r theodolites. permanent accounts haggle. ironic theodolit +8387|Supplier#000008387|QdajYtgOvNZjAdY4Uu1VcYqnrSbSCz0r8|22|32-535-328-6371|4892.05|e furiously final packages. furio +8388|Supplier#000008388|ftgPAroHdZOm12ZLz7|5|15-628-465-7589|4610.83|! silent, express accounts about th +8389|Supplier#000008389|,N1gJ,pO7OGhFc6DX|17|27-474-840-9525|8295.71|even accounts? slyly sly dependencies haggle blithely carefully final packages. +8390|Supplier#000008390|S5fj7UR7wc6|20|30-478-369-1888|9987.37|as boost slyly during the final, ironic ideas. slyl +8391|Supplier#000008391|yORrAeKllRIMy,tFqE5qQYl,0J rxVOw7g|2|12-815-751-3929|6877.70|accounts. slyly final theodolites thrash against the ironic pa +8392|Supplier#000008392|88mDQjkIaRqQH2RrFXb,v|16|26-971-722-7151|3560.63|haggle alongside of the furiously final packages. requests sleep. fluffily regular packages nag sl +8393|Supplier#000008393|wbPjQhCQGBsIpc2ZhCO69TrgsaVN8wKm5MAx|5|15-830-175-5274|1436.89|l accounts nag. carefully speci +8394|Supplier#000008394|PEfV8nyE SKPYnULhxH4BqVm7FUp6i8zLWfhC|20|30-796-525-5124|2720.71|riously along the bold ideas. sil +8395|Supplier#000008395|PWMaZXgi,0jWrlGUcJoasrScyXo|7|17-745-676-3177|8935.64|l requests haggle carefully above the blithely regular pinto beans. slyly bold foxes snooze fin +8396|Supplier#000008396|ofml8rQt3uP,RfwzKJO|20|30-594-869-8475|7372.90|yly. ironically unusual platelets are slyly ironic pinto beans. furiously pending pinto beans +8397|Supplier#000008397|GomUbkpCA1|1|11-398-138-4822|-543.26| blithely final requests above the evenly expre +8398|Supplier#000008398|EkXd4mwesEQ,cAOxqbTJ,HvYyQotg3o6vlnd|20|30-713-464-1834|8349.50|nic multipliers. pending accounts among the carefully regular packages wake quickly fluffily spe +8399|Supplier#000008399|0kw6rk0zf9zgzMhwTcyGeJ6AyTug8Z,Xp|12|22-815-494-2565|1051.15|es wake. sometimes even packages cajole. foxes sleep carefully regular, silent platelet +8400|Supplier#000008400|y0rF4,BBqwBE6hFAdkEMOeSVoVydZ|18|28-215-743-8600|9576.53|ecial instructions. regular, pending deposits boost stealthily after the requests. bold +8401|Supplier#000008401|UQEIjAI37Vb0Pvj|8|18-283-493-8785|9869.67|d, final packages. even packages around the regular, final deposits wa +8402|Supplier#000008402|Hg606yo4H3H92OEZgWUoiRxVjejN|19|29-548-285-9275|8621.83|he bold sauternes. pinto beans cajole slyly. bold, sly deposits haggle carefully +8403|Supplier#000008403|4qHoK3GIQYqs3I zPlzOwRnS5XiAogAAcFCkyt|4|14-566-285-7809|2164.96|ular requests cajole fluffily unusual deposits. furio +8404|Supplier#000008404|JmmQpsT 19ZctXXxCToDGqgXupeb|14|24-906-890-4936|7488.58|he patterns hang quickly ironic foxes. excuses sleep. quickly +8405|Supplier#000008405|1hYhzpvXK0uslRJ6CBUZqm a|14|24-465-424-6781|7543.03| regular requests grow quickly alongside of the thin accounts. blit +8406|Supplier#000008406|,qUuXcftUl|22|32-139-873-8571|9681.33|haggle slyly regular excuses. quic +8407|Supplier#000008407|qZIW3kvrBSFDUXcT4y|16|26-165-112-7048|164.99|ts to the furiously regular deposits nag carefully ironic instructions. spe +8408|Supplier#000008408|Sc9u9mcc2bLTWfHspMScTrbdaut|19|29-475-811-8890|9917.49| ruthless deposits boost even account +8409|Supplier#000008409|N9Ch4EGQ0I|0|10-445-108-7715|9339.97|xpress dependencies? slyly ironic accounts cajol +8410|Supplier#000008410|9riCCazinZsgc8mbjKa1wEfbGv oVlgKhanGW|3|13-188-800-4871|4875.41|he special, regular instructions nag carefully unusual dolphins. +8411|Supplier#000008411|vtb0jArF1kmSJI0hI3JEF|5|15-213-446-6812|26.82|ses. regular deposits wake after the quickly ironic packages +8412|Supplier#000008412|gOBtQQlqTMmPpdlkyYRNfnYp,Dvu|14|24-192-482-7068|8693.17|e carefully blithely unusual accounts. regular deposits cajole. blithel +8413|Supplier#000008413|pWEMNP3EUdCy ds3zURKq4w2VDW|1|11-775-936-1233|2179.22|s. blithely bold theodolites run +8414|Supplier#000008414|jkNFVmWI3VWZThBU,gb nlSsaSp67yU|2|12-900-269-8590|5108.56| blithely close packages at the final packages cajole furious +8415|Supplier#000008415|MknyFywbrBY89GGbubIeAQuvVYT5rx9pHCh|17|27-384-823-7032|2528.76|accounts. quickly regular requests cajole slyly across the carefully silent dependencies. caref +8416|Supplier#000008416|RrFXnKKPGWOmzDMkaQsgmAzc3nPvkmxJIK|0|10-422-505-4431|5853.40|sly regular deposits cajole furiously blithely final foxes. blithely quiet ideas +8417|Supplier#000008417|KMstl4j5LDLfg,FRnl39cQBxajMls|13|23-833-184-9498|3088.69|tes. excuses along the blithely regular deposits cajole bold deposits. final asymptotes boost. +8418|Supplier#000008418|OREmdbzO9TCdp0nGD3tDB|8|18-740-790-4790|556.42|ounts. carefully final deposits w +8419|Supplier#000008419|yLcbiqfp3WpCqpTE6usqXGwAFEIMFwqB5A1NQBxN|15|25-719-776-7477|-376.38|ts sleep regular, regular accounts. blithely regular +8420|Supplier#000008420|bNMSwRBINnrsNB6I049|17|27-599-541-3605|8078.22| about the brave packages. dependencies along the quickly i +8421|Supplier#000008421|x2sr5EHkwDOimr0n9uWd,cDEXyIEXngBLI|6|16-554-443-4756|6778.63| according to the pinto beans use above the carefully ironic foxes. pinto beans na +8422|Supplier#000008422|EuIEI3T6B kwAnjl55Vmc|21|31-278-642-4974|-532.14| final packages sleep. enticing asymptotes cajole acros +8423|Supplier#000008423|RQhKnkAhR0DAr3Ix4Q1weMMn00hNe Kq|3|13-889-168-9050|8313.01|into beans sleep blithely. furiously +8424|Supplier#000008424|m7,zJlWujHpvIuLqZdXg0VWUQ0ZUy1 J4AQ|9|19-886-503-6563|3749.07|lyly? theodolites are around the packages. car +8425|Supplier#000008425|MaETn S83sc5RdXg7aQnmDPN|4|14-606-442-4596|670.03|l packages. furiously unusual deposits use across the quickly final dependencies. +8426|Supplier#000008426|Qsu,Mh2ylkvxYsyZjxwzl42cMr|24|34-685-517-9185|6179.18|counts; packages haggle bli +8427|Supplier#000008427|J1Vd3lqn1UvN2|4|14-632-452-6847|535.00| about the quickly special pinto beans. dependenc +8428|Supplier#000008428|YeM8xSQcU5k7i|8|18-481-625-2961|2650.42|s around the carefully ironic ideas are quickly express deposits. +8429|Supplier#000008429|IKsf2rgK5C3HZXjuIvNNT2rAFdkfx|10|20-553-284-9567|4561.76| carefully final packages haggle bold deposits. carefully silent war +8430|Supplier#000008430|HCC99izA3YX5kVTJYwNpEHJJwR2cEfqcWp|13|23-147-789-4736|8455.11|luffily final deposits. finally bold pinto beans cajole along the accounts. slyly special acc +8431|Supplier#000008431|V5udjaztpR60UuoBm6boa0kMaBhMWrS4|22|32-515-509-9935|5176.65| theodolites. blithely pending r +8432|Supplier#000008432|bNJp NFRcxiMrDRLqFk|11|21-243-719-7560|2980.69|ironic packages. furiously regular dependencies wake during th +8433|Supplier#000008433|xSo5D3aq5vkm3o5CyknKEdc|6|16-736-465-1902|366.49|ly regular foxes. carefully bold foxes haggle furiously. carefully bold acc +8434|Supplier#000008434|dM71AAl9v9qP|13|23-119-336-9281|8498.99| blithely final requests. slyly bold foxes are final, bold foxes. blithely dogged courts from the +8435|Supplier#000008435|UHmLbgZyp7EC7K5a5|22|32-593-113-8050|5551.46|l theodolites; regular accounts wake da +8436|Supplier#000008436|W4FsPY6LIheuLP5cSdAxCJkrPJD|7|17-622-453-3539|-590.56| unusual, special courts about the slyly regular frets nag daringly according to the even excuses. +8437|Supplier#000008437|Y58iKUOib1zUPTX|22|32-127-249-8582|5415.82|yly silent accounts. blithely bold gifts boost. bl +8438|Supplier#000008438|MHX2cideiqjxZgCyenirqSChO|7|17-510-783-5625|7721.78|es nod slyly furiously final ideas. blithely daring packages sleep bravely f +8439|Supplier#000008439|di6C32okzgTjM|4|14-656-728-1229|52.83|efully slyly ironic ideas. express courts wake slyly along the regular foxes. sly, unusual pa +8440|Supplier#000008440|hLodfZoCv4q4axaQm7r925TDKn|8|18-495-197-2800|5883.30|ly. quickly special requests dazzle quickly. pending patterns wake accor +8441|Supplier#000008441|huSfIBoFLxPxtaNk2qgpq,hAzg7vn|18|28-532-202-4077|-338.96|symptotes. blithely daring accounts about the quickly regular foxes wake blithely across the ca +8442|Supplier#000008442|So5,Vf93wcvoHpP7thm7KxK1F g3DKmXMKZR|2|12-926-257-2326|9953.38|ains cajole furiously special deposits. quietly ironic foxes slee +8443|Supplier#000008443|4NVFgmnKNRe|13|23-767-598-9866|7601.35|re quickly above the express, +8444|Supplier#000008444|AWXKqQxZEB0Jyzp9OsN4I3RDsTl5cFpTN|11|21-985-891-2086|7.38|mong the excuses. slyly fluffy instructions wake furiou +8445|Supplier#000008445|MaGM3kem,EKQ5AVeWvwy2tSv4aZBrc079nB7pap|15|25-782-373-2941|981.31|carefully ironic pinto bea +8446|Supplier#000008446|txlVyI71F5r,9ct34|19|29-361-236-8438|1766.27|le slyly quickly unusual dinos. blithely ironic grouche +8447|Supplier#000008447|kYxlpT,F8AJgEj8uF|23|33-992-618-4096|5621.87|tect blithely against the silent packages; final platelets accordi +8448|Supplier#000008448|2tLLbL9k4MkV98 1xKypqW|8|18-940-998-8999|5726.46|y according to the blithely r +8449|Supplier#000008449|Wp34zim9qYFbVctdW|10|20-469-856-8873|3667.19|onic instructions. carefully express packages +8450|Supplier#000008450|2EySLcI4F9ySNU7lbhtqNFP9XiAcBS|24|34-523-714-2279|-553.13|ymptotes. bold theodolites wake along the slyly regular packages. s +8451|Supplier#000008451|p7rhi8lJCt3uFSFbSQM,gEGSUc|17|27-450-658-6737|7535.05|ronic instructions sleep quickl +8452|Supplier#000008452|lTrl0TcxcYo0Xyydy5X4Lan|2|12-686-389-2091|7550.43|ular courts. regular deposits sleep am +8453|Supplier#000008453|oCazUq2FT17SpC5vnlTJNF9Ob52ZKAUQgb|1|11-293-826-4540|2144.32|al requests affix blithely unusual requests. fluffily express requests sleep quickly +8454|Supplier#000008454|eMlEBxvKjN2SXsu6BkN4bTtZVq6aoFcBl4JeMs3|22|32-658-462-8813|1555.96|blithely pending foxes cajole furiously abo +8455|Supplier#000008455|,zB2VbEOs1o7,YOYmE|18|28-228-864-3061|2024.52|uriously regular frets cajole after the furiously unusu +8456|Supplier#000008456|pvf0FPIUzA9UF9Smo3vcrcFLJNvxoHfrw4s8Ocv|19|29-247-738-3245|1147.46|ts. quickly silent requests sleep across the +8457|Supplier#000008457|CAvkqakaOWv5OpT0YyRZF81lBDl5tpjHL |16|26-398-482-7559|235.00| beans boost furiously quickly ironic accounts. pack +8458|Supplier#000008458|rihX1QITvV1JaC|4|14-822-547-8952|9535.76| foxes boost furiously across the slyly express excuses. quickly ex +8459|Supplier#000008459|bi5ogIxzj kP mbT0yulUqiR4pQhvcgbo4FM|4|14-703-366-2735|6636.17|nag. slyly pending theodolites sleep closely a +8460|Supplier#000008460|oKkRuk0y0cypZxvPEv9joSpnyy,p9LU|8|18-262-610-3962|3137.52|ial deposits. unusual, express asymptotes haggle quickly carefully bold ideas +8461|Supplier#000008461| PeQ3wub7gh3Qmda|16|26-286-487-8621|7229.47|. dinos cajole ruthlessly. slyly silent requests cajole. enticing platelets promis +8462|Supplier#000008462|3K6 wmXG E64Mfx 4Et|8|18-878-705-6103|8693.69| bold, regular foxes beyond t +8463|Supplier#000008463|20Qk0i7nFty1TrvC1nlh29Bh0vEb|23|33-579-124-9718|-103.21|ake furiously. even requests affix slyly alongside of the special pinto beans. quickly bold req +8464|Supplier#000008464|dtrFDLdfguJKwbVO9h4AyQwP3YP5rjJ|7|17-710-692-6846|1236.18|le. final pinto beans was ideas. ironic deposits according to the ironic accounts use above the +8465|Supplier#000008465|1ljQImkOB8MGxJ5|14|24-469-535-2573|6384.60|eposits are express accounts. warhorses nag carefully regular sentiments. regular packages again +8466|Supplier#000008466|pGUWUwyKCxT,9Qb6|5|15-653-391-4079|9934.49|ual escapades sleep around the ironic, expr +8467|Supplier#000008467|mOzxpu7,0X7f2,BY3d OnG6yDTPz7k74|3|13-460-470-1371|9064.19|ages haggle furiously. special warho +8468|Supplier#000008468|5R4jsweitleustYlE3w,u5otW|21|31-775-722-6460|844.25|its about the regular foxes detect pending theodo +8469|Supplier#000008469|KO8JEXd9bjCGExjeLS8FH,mE,OSHl6vTHVHsFyw|6|16-529-676-8077|8257.32| quickly above the furiously ironic packages. express theodolites cajole carefully fluffil +8470|Supplier#000008470|wURUeuqN,W5P4gHfx5wM69o9xn7Wy|23|33-630-311-3942|2799.27|ng to the quickly final ideas. quickly final ideas sleep carefully even platelets. +8471|Supplier#000008471|yby8sR0,75PDNPOJmWtHFu8eCMPEkLpctO|12|22-536-259-4439|9345.49|regular packages. blithely pending requests above the carefully final cour +8472|Supplier#000008472|4at1BaGXf8r|6|16-516-582-7374|9778.55| ideas boost ironic hockey players. requests maintain. blithely special requests detect +8473|Supplier#000008473|OdJPu18DkC,bgQp6,UIKYrAm0BId1G8G,SbEjiN|11|21-510-110-3730|6155.56|ges. furiously ironic foxes above the blithely final depen +8474|Supplier#000008474|6PP Vy8sEsdg3r6wcgOEFR,xikND7uHsOCCwMKS5|19|29-167-952-8672|-417.15|ites. instructions detect blit +8475|Supplier#000008475|RKZHXR34i8BCbD,qGa3hel9EmHVx|15|25-646-316-4850|4871.07|pendencies under the carefully unusual pains haggl +8476|Supplier#000008476| a,cswFIvmn NgyIjCfCErBrQ0jy9zPEFdt|0|10-981-618-7449|4050.51|. regular, special theodolites detect busily after the enticingly final requests. furiously un +8477|Supplier#000008477|p5TvX4xf1n4aLh75Hw|15|25-323-997-2362|6899.88| nag according to the regular theodolites. furiously ironic pearls nag. furiously ironic excu +8478|Supplier#000008478|Tj3eE6Al7EqoYO5usiXfHCLfxJk5EMAu3Q|21|31-885-222-9592|2334.47|its. accounts unwind slyly +8479|Supplier#000008479|D 4USLOIvHXmJRXPs787|6|16-507-677-5585|7397.35|y unusual dependencies. excuses alongside of the idly unusual accounts boost carefully express de +8480|Supplier#000008480|4sSDA4ACReklNjEm5T6b|3|13-839-573-3826|3241.49|heodolites nag. even requests cajole slyly final foxes. furiously final accounts nag according to +8481|Supplier#000008481|Uz03L1Ig8BfEqq3RA0beHxg8IO,WGjb|11|21-163-290-6334|5750.97|ily slow packages. evenly special platelets wake. fluffily ironic foxes sl +8482|Supplier#000008482|DsdkDvolnjlpfLkNY0r89wsqJq|12|22-612-911-3916|8610.63|l theodolites are quickly. furiously silent ac +8483|Supplier#000008483|v464kLe2m,VtKzOQgm6KlDq0HvRhhxz4TB0i1Lex|8|18-710-845-2845|886.34|tes haggle. blithely regul +8484|Supplier#000008484|YgFHZx0af26o9dq82lpNGzuyp0vB6IX|3|13-220-902-7962|706.88|e fluffily pending orbits +8485|Supplier#000008485|UKbWnxfcaS,gZklz|5|15-711-844-1174|9553.61|lar theodolites haggle slyly foxes. express instructions cajole pending asymptotes. carefully iro +8486|Supplier#000008486|ngtBMvFNcFT2Lc5c4,sn8ozQX|23|33-489-227-9233|9638.25|riously special requests haggle quickly? even deposits are furiously +8487|Supplier#000008487|XDWfWQqWRkvTZI WsysCAnorSZznsSWp KVbaj1|13|23-120-587-8370|2176.32|g to the regular deposits. quic +8488|Supplier#000008488|fL8BwJNM4,Q8XK,OlNp6CaRN|5|15-140-796-3382|406.21|ounts affix carefully quickly permanent asymptotes. blithely bold accounts are +8489|Supplier#000008489|9POyLv4,k5eIJ|8|18-475-311-8785|5077.66|le blithely except the furiously final accounts. even foxes +8490|Supplier#000008490|txOd4J8il5G|20|30-841-305-1854|251.76|, final asymptotes. furiously regular requests among the furiously final dugouts +8491|Supplier#000008491|m1s,CS3yn4Ty|15|25-956-836-8036|5280.65| deposits. final, pending ideas haggle busily-- slyly even +8492|Supplier#000008492|TzB 7njlatzQdw4h0ChDH|20|30-484-180-4829|1822.02|uriously regular platelets. stealthy foxes according to the furiously spec +8493|Supplier#000008493|cFyFEw7POykAqWkQeW1,1WLEiiz|15|25-618-559-5016|7575.13|ooze regularly unusual packages. unu +8494|Supplier#000008494|8Lb41M39FM9oULM,2LG|24|34-303-859-1524|-699.96| express deposits. unusual accounts after the theodolit +8495|Supplier#000008495|RGAKqbyph7l2r8FEDY|21|31-229-537-8994|3759.81|ickly alongside of the platelets. regular, ironic packages are slyly. fi +8496|Supplier#000008496|YQCfEcP6D5 3|16|26-117-326-7579|5111.61|nusual, final packages use. accounts are slyly. slyly ironic instructions above the carefully pendin +8497|Supplier#000008497|xef7AF1FKfjml5HrGfEHtkhk3Tbeg0ms|2|12-748-918-9360|9886.73|. foxes nag slyly express, regular multipliers. requests after the carefully ironic accounts grow re +8498|Supplier#000008498|W 1pxS,3INXAPPn9tlvLG6nWPcHmLgpyH|22|32-560-835-6989|9312.15|ilent, bold packages near the quickly regular accounts sublate fluffily according +8499|Supplier#000008499|Z5f y6tASps4VckVSvA9JLg|12|22-379-481-8071|-578.55| deposits lose carefully about the slyly iro +8500|Supplier#000008500|AjOpSW7qLo|12|22-800-223-4622|9151.26|nos. regular requests x-r +8501|Supplier#000008501|Wkegg0LmykRRl4aYHM7V8|1|11-167-579-7313|1338.71| carefully. slyly final dugouts cajole +8502|Supplier#000008502|7sWDOClgh15PopqCXg31lxon0XbkDXF|17|27-997-943-6884|2700.94|usly ironic ideas! busy requests integrate furiously besides the caref +8503|Supplier#000008503|EIhePQhDdhklwciepW|0|10-661-665-3603|6594.86| slyly. quickly final accounts use even asymptotes. +8504|Supplier#000008504|S7nEEnbeSHtq6Hp5Cb8cDC1xT2FcLDf0Kb9|4|14-318-631-5865|1825.82|; carefully even asymptotes +8505|Supplier#000008505|Glk1OiM44v6CB0CpAzrf|7|17-461-783-2686|5487.79|into beans. furiously specia +8506|Supplier#000008506|VDxpPz,XgrZPi11A4iSVJ2JFbnf8x,2w|10|20-119-781-6781|7364.72|fully regular foxes. furiously silent packages mold caref +8507|Supplier#000008507|r9xIe2EIrj|17|27-810-472-6422|4881.37|ong the express ideas wake furiously special pinto beans. unusual, sile +8508|Supplier#000008508|TGpsTDKgtFc5aoFFyl,ImrYMjHjmV|2|12-615-316-8574|1232.90|ffily regular, final deposits. express depths are blithely unusual braids? carefu +8509|Supplier#000008509|E,gt5UKRp8oHmTY|22|32-274-103-2779|6231.26|ts sleep blithely above the final platelets. ironic gifts detect. unusual excuses wake blithe +8510|Supplier#000008510|aUlYzASj024j xvxU|10|20-622-650-6857|6751.53|ake alongside of the blithely pending packages. regular, daring attainments haggle carefull +8511|Supplier#000008511|JwdJZrdbup7rJh|22|32-948-885-3347|5952.86|gside of the furiously express accounts? express requests boost slyly regular grouches. +8512|Supplier#000008512|WBcS0re,fvhhz vQ4aFScEUWvycJuGDasQgDI|16|26-368-922-7281|2702.52| wake after the blithely bo +8513|Supplier#000008513|ihTrBem,lzLWyJzYXQnkoqmp1Ug|8|18-527-469-4228|201.51|s haggle ironic, final ideas. blithely special accounts are. regular, pending depths w +8514|Supplier#000008514|0cnuMdRfrsX6yuS5AbfOTEQEn3|10|20-995-676-9196|7794.99|ole slyly special theodolites. carefully unusual accounts again +8515|Supplier#000008515|TA12u9cZKAlmzPorkWJgvtJrjus,5|19|29-357-508-1973|9430.81|ily packages. carefully regular requests +8516|Supplier#000008516|UaL9jrvT8BqR2GB|11|21-185-933-8590|2691.61|osits. furiously bold asymptotes about the ironically unusual frets cajole car +8517|Supplier#000008517|us1ou02ozRlSNW3hIrFb,vaiPn1l3e|14|24-774-997-2846|3905.96|totes. regular, silent deposits sleep about the ironic, final requests. +8518|Supplier#000008518|2jqzqqAVe9crMVGP,n9nTsQXulNLTUYoJjEDcqWV|6|16-618-780-7481|8386.08|blithely bold pains are carefully platelets. finally regular pinto beans sleep carefully special +8519|Supplier#000008519|caX1G0CktaS,9fqLOta260Y|10|20-177-279-9686|-138.74|ns. foxes nag carefully regular platelets. blithely silent packages nag blithely against the request +8520|Supplier#000008520|RqRVDgD0ER J9 b41vR2,3|7|17-728-804-1793|9104.83|ly about the blithely ironic depths. slyly final theodolites among the fluffily bold ideas print +8521|Supplier#000008521|ydw4lF30Ch8DX,o U9a11sEIhFGVsF|13|23-251-142-5448|1882.89|he quickly slow asymptotes. blithely unusual forges above the quickly final forges sleep alongs +8522|Supplier#000008522|5IBcn2A48KCdoFL0q|22|32-950-249-7921|7626.64|nal packages nag blithely. +8523|Supplier#000008523|C4ocdfNu5I2nnnVG2xSd3016J6KNLIg|21|31-758-497-2263|7000.18|elets. carefully express deposits haggle furiously about the furiously bold package +8524|Supplier#000008524|iFMHJvUlALZoDpuzUMYsteZVz5D4GjP|11|21-465-917-6611|5004.14|o beans haggle carefully even pinto beans. regular, pending packages serve +8525|Supplier#000008525|NBzAj3RpEG5 ,168|12|22-570-937-7745|7516.90|ar accounts use silent asymptotes. special deposits +8526|Supplier#000008526|3RreW34eNbq15bM8MMjk|9|19-986-831-6193|-408.28|ges. furiously final theodolite +8527|Supplier#000008527|TYA0XEMpN,g2UedCfXz1o|20|30-402-236-3658|-64.16|s nag blithely quickly final packages. fluffily i +8528|Supplier#000008528|O,BS rNVx7CsdlsjUgZ5fNN6yx0z2Bz|11|21-265-460-1770|8638.54|he furiously regular ideas. carefully ironic dug +8529|Supplier#000008529|l4Y,f11TYz7Ph1erL7IM9AXf0poUIrO2OS|20|30-791-579-2088|8045.75|xpress deposits are furiously. blithely regular dolphins affix blithely enticingly regular t +8530|Supplier#000008530|5EacwOuDZqQjHTQgG8XV5Eja65B4HXh|11|21-267-583-5534|-18.15|. slyly bold foxes cajole. special requests haggle fur +8531|Supplier#000008531|qTCTjmpcMVoihJcsOJYaYd2epCZsDPtsYhGSl|5|15-901-206-1728|863.84|p regular, final requests! blith +8532|Supplier#000008532|Uc29q4,5xVdDOF87UZrxhr4xWS0ihEUXuh|3|13-865-396-2710|7566.78|azzle furiously regular dependencies. quickly ruthless accounts across the slyly final deposits +8533|Supplier#000008533|R 0kQH79zpDxZKr15KcktHav|10|20-634-681-8231|1998.72|escapades above the busily even accounts haggle furiously express instructions? even deposits ha +8534|Supplier#000008534|Lejalo7504fMf,L3|21|31-549-453-6110|7505.44|pitaphs. slyly regular deposits wake furiously fluffily final requests. fin +8535|Supplier#000008535|M82fIolbV6dpmRngfMN18OpwWFL0n|10|20-988-914-2469|4935.59|nding theodolites wake fluffily along the quickly even requests. final requests cajole quickly alo +8536|Supplier#000008536|78WMZMzeuD|1|11-262-452-2654|-613.82|affix. final, even packages detect blithely! slyly ironic excuses cajole ca +8537|Supplier#000008537|rHWLILYbcdIs1qW|7|17-396-624-4223|-564.51|pinto beans are furiously. special asymptotes sleep carefully final excuses. slyly special deposits +8538|Supplier#000008538|,73mumH eL4AT53bY8dZ2gTFHhiSpL4br5ozQPyT|8|18-104-429-6314|3141.69|ully ironic asymptotes: ironically regular excuses boost ruthlessly. furiously even +8539|Supplier#000008539|mtOsFdgudNlBJ3L6Pi2zfoIqK8ER2fsfDrRsJI h|19|29-492-791-9837|359.82|ackages whithout the furiously regular dugouts sleep above the express packages. deposits run sly +8540|Supplier#000008540|RQGjvRP7uFsx3pccm0H7NQ0fQp7GAUNAvtkakUid|16|26-239-939-9900|6348.78|special instructions. furious +8541|Supplier#000008541|nFgcbHhyG0CPeRDZVm|16|26-955-266-1279|1933.38|pending requests wake quickly according to the carefully final pinto beans. f +8542|Supplier#000008542|YPhfijxCiFKkm8,S|24|34-764-457-8674|9330.71|as sleep blithely pending accoun +8543|Supplier#000008543|XyvbucQe9stlJlXSC1WHpQ9C|21|31-433-501-4783|3259.19| ironic requests haggle blithely across the i +8544|Supplier#000008544|sXjb2fuyyDJ1Ka65tD3IcAGPkKpMwmyLr|22|32-244-321-6139|6156.32|uffily bold foxes. requests may boost slyly about the furious requests. carefull +8545|Supplier#000008545|dCE06YfPKks38|7|17-669-333-8409|1332.10|lly regular, pending ideas. bold, special requests doubt blithe +8546|Supplier#000008546|4qn718cUjQEKsfWfJy5|2|12-827-860-1192|1506.41|eans. pending requests play quickly. +8547|Supplier#000008547|2IK PvdoGtbZUyFHeQAPqIzYVxXDHW|19|29-778-579-5144|5187.63|ix according to the regular instructions. furiously regular deposits ac +8548|Supplier#000008548|6hA0SVDEVUTaQcpeASuT7LemNhyWWGVMwf7BZ|0|10-688-859-2844|468.59|iously express ideas cajole slyly after the unusual requests. slyly final accounts lose fu +8549|Supplier#000008549|FMsXzKfMDnRHpsjdVb|3|13-339-338-5688|8557.42| quickly regular packages. permanently special ideas affix furiously. blithely +8550|Supplier#000008550|yooUNyFfNV2XEKwOI|1|11-816-674-5599|8539.38| regular deposits after the slyly bold accounts +8551|Supplier#000008551|991B,Fp7RohkxhGRhg4645Xe|20|30-978-275-9653|2485.84|y bold requests nag along the theodolites. carefull +8552|Supplier#000008552|Mr9bM9,nqB0w5hocbjWJ,|23|33-232-964-3881|-478.87|rts sleep quickly special accounts. blithely ironic theodolites among th +8553|Supplier#000008553|9cfBlcQUuHiVY C nhZs5C13ALqXrY0J|6|16-919-476-6611|1607.30|regular ideas was slyly silent epitaphs. fluffily fina +8554|Supplier#000008554|UVtVFbwh7wjCm|0|10-158-340-9799|6830.14| quickly regular instructions; blithely special excuses after the express +8555|Supplier#000008555|yJWU0ZtPOIuv|1|11-878-821-5627|7193.13|c foxes sleep about the quickly final excuses. carefully even theodolites cajole. slyly final pi +8556|Supplier#000008556|ShhyTdODCKl|22|32-149-401-5846|3056.71|nts detect alongside of the regular deposits. exp +8557|Supplier#000008557|xYhxoPgJObCs,UWoYM4VbqFrY9RXtG|17|27-106-129-4662|2158.66|xpress deposits haggle furiously a +8558|Supplier#000008558|XOtoPKUMX4wHJkQj|4|14-109-690-3294|559.24|eas are above the slyly ironic dolphins. carefully even requests around th +8559|Supplier#000008559|IvHvgZ0TtOMX3xFOW9kgYDNXsZv|3|13-857-535-4259|3051.23|ly thin theodolites about the slyly silent pinto beans doze carefully fluffily ironic in +8560|Supplier#000008560|0ecAr,jIG3F9p8bQ4O49D9S0boSXifq|14|24-544-801-9413|9049.00|lets. unusual sauternes about the quickly ironic +8561|Supplier#000008561|Mkd7mWU6XZNNkUqnsy34Kt1aYxo|19|29-309-888-3946|9652.21|s accounts in place of the express accounts na +8562|Supplier#000008562|mlEKbVlwfdV6lRyzI7xEVg7LTbtB3zlTQC66tpR8|8|18-247-297-6917|5718.36|kages according to the unusual deposits wake furiously fluffily bold dependencies. fluffily regu +8563|Supplier#000008563|VNgYAWCC5WAEyD0zMizmj9gneLRSgXiH8XNcg e|18|28-411-389-8774|-481.91|quickly against the regular, final requests. pending, idle accounts detect requests. de +8564|Supplier#000008564|kjqlGwW2J3yhj3MLluVp8ywQawNtNEX3TH3S|1|11-562-252-2472|5382.16|carefully ironic accounts cajole. slyly regular requests grow blithely final p +8565|Supplier#000008565|fZrp4 nL0P3jU|7|17-671-732-8056|2962.26|thely furiously ironic foxes. final dolphins are car +8566|Supplier#000008566|CJJ vg1hGBwps2HgREKQqbzmQ038LPXtbha26vqA|6|16-158-515-4201|9749.51|elets haggle after the slyly even ide +8567|Supplier#000008567|X3ndLTt8z4Svp4B1HV3d7i|19|29-731-239-3533|2130.55| carefully carefully unusual sentiments. quickly bold pinto beans +8568|Supplier#000008568|snNJuxNAHgMjMSIo0WQgAVrq068|6|16-247-780-8680|1101.89|y unusual pinto beans. special dependencies haggle among the slyly +8569|Supplier#000008569|zQOGXyONUkkdUcl5 j3IbRB6,T0s3wiIBMB|11|21-328-882-6096|4330.80|ideas. carefully regular pinto beans use carefully fluffily quick requests. quiet accounts aro +8570|Supplier#000008570|tVFRi4Uq,iOEWJcVBrOTiS3naWEDSIMvolIhjW|16|26-995-212-9357|9723.27|gular theodolites nag finally. regular foxes are furiously. regular accounts sleep accordin +8571|Supplier#000008571|ASFEOWTBT3lfFH6Xqr8VeOXoW5o|19|29-534-738-7942|1061.89|final requests. instructions h +8572|Supplier#000008572|2PC3EaagcRGLoCbK|4|14-371-272-3628|3175.32|foxes. final pinto beans hinder carefully alongside of the stealthily special packages. silent +8573|Supplier#000008573|cFz3Iter91KwJcioQiaci7YnzX90N|1|11-327-567-3040|4645.24| furiously special deposi +8574|Supplier#000008574|,vEvQz0cbaBQvZ,7eOqQ VY0hzzs3le U,Wlg|13|23-444-959-5572|3487.50| carefully even accounts. instructions detect slyly around the blithely regular pinto b +8575|Supplier#000008575|JBxuXPNZaP8q|17|27-440-814-7131|4382.53|ly bold accounts are. silent, ironic deposits are fluffily. final foxes ne +8576|Supplier#000008576|b9gsg96njGJSamE yI0YxidS5yt|2|12-220-669-8884|-759.18|heodolites haggle even reque +8577|Supplier#000008577|C4vTbzKKCuz8HFn|14|24-320-703-9621|9440.83|. even instructions cajole c +8578|Supplier#000008578|bx7za3ymbcxn5aUN9w77uM c3|13|23-767-790-5293|6143.78|ost blithely bold platelets. carefully express instructions across the slyly reg +8579|Supplier#000008579| hMJ9jiSdp1H M7h7xpGOjT6VHBkI QPVYUDV|3|13-698-342-1983|-544.13|blithely ironic packages haggle finally furiously silent sentiments. +8580|Supplier#000008580|t5ri71bM6Sox3riP4JUZsMMNC|21|31-557-162-9191|4724.43|ly slyly final deposits. even packages b +8581|Supplier#000008581|mGa8SV8VG1wSRLMj|17|27-339-593-3170|3570.28|y special packages sleep quickly after the ironic pinto beans +8582|Supplier#000008582|B,Sy8pyruZPcyhEMUf|21|31-274-355-1342|9711.40|ultipliers. carefully final requests lose fluffily? regular asymptotes sleep according to +8583|Supplier#000008583|X51EuQ7WtA Q1hl|6|16-973-502-2217|-503.99|s. final, bold deposits cajole against the deposits. +8584|Supplier#000008584|VFUIyZAK34P1q7OYwf4u Il6HPsI|23|33-835-785-9186|8280.18|lyly even dependencies nag. slyly even dependencies above the blithely bold deposits +8585|Supplier#000008585|1rsEBCKUBd5026zLAxS4XBZlhRRxHyOy8KroMB|24|34-116-951-6320|2104.10|odolites-- slyly fluffy requests after the slyly regular ideas prin +8586|Supplier#000008586|DEFRVxUmVEOrpjsR hXEFOxGjG|12|22-832-217-9564|7637.66|usly even instructions sleep blithely across the carefully bold dependencies. fluffily careful fr +8587|Supplier#000008587|eQOgec6prwzTBrU u901pWPT|5|15-803-611-2450|5156.40|hely bold accounts. ironic, express grouches sleep +8588|Supplier#000008588|2hnDG8SRQP,Z5qGaDuphcS,UU|0|10-509-417-5501|3601.41|l requests x-ray on the quickly regular requests. fluffily even deposits after the furiously iron +8589|Supplier#000008589|ZFWWsRSfOZt3id4|0|10-936-646-8917|4962.58| the packages are fluffily final requests. regular, pendi +8590|Supplier#000008590|slOzkYi1Z5Pmk6|24|34-787-907-6102|2036.81|use blithely regular foxes. unusual, regular depo +8591|Supplier#000008591|3G86tepvfRxn|0|10-546-978-5154|9640.23|instructions. furiously bold +8592|Supplier#000008592|UGki0J,dRb3umlbvRD,jkET0|2|12-347-684-6545|2411.82|unts. slyly special packages are c +8593|Supplier#000008593|72,y1RTZma WYsp oxb61sQGWt|12|22-356-402-7135|9435.02|e final ideas: quickly express packages ha +8594|Supplier#000008594|EbT1I5lT889uz0kfOZytux,pdcyFT p2|17|27-516-843-8455|7416.92|s the silent packages. ironic, even dependencies haggle quickly after the final foxes +8595|Supplier#000008595|MH0iB73GQ3z UW3O DbCbqmc|3|13-841-273-9262|2195.24|ily silent deposits into the slyly silent request +8596|Supplier#000008596|vuSX2muycZqscMe|18|28-931-478-2730|-705.12|warhorses grow furiously along the carefull +8597|Supplier#000008597|50oFp6fU47TWDim p2XTaG4quT6u1hLWt9pz|6|16-524-354-2340|4997.47|deas. blithely bold requests sleep quickly according to the final depo +8598|Supplier#000008598|i1Z531x0yksPsQJzZbz|14|24-482-107-8932|5477.57| against the ideas. blithely pending +8599|Supplier#000008599|4shU12vuiCeg|2|12-804-637-9812|601.02|ully about the final deposits. quickly +8600|Supplier#000008600|ZYFNk9cRM66qHdcrOR8wNNRKQsjh|7|17-389-804-4258|1273.32| silent requests. blithely final foxes haggle slyly at the carefully regula +8601|Supplier#000008601|Tq2xNb9t22iA|18|28-110-804-3016|3636.98| cajole bravely permanent asymptotes. regular packages among the carefully ironic +8602|Supplier#000008602|MUSInewxOfFpizzfFeQ7VlOsGc8YJoMRKRiXtjhC|15|25-705-385-5827|8726.51|ons. quickly ironic platelets boost. special, even packages cajole. daringly final p +8603|Supplier#000008603|2sE85alXUx3Pk|22|32-624-274-9130|1474.87|hely final packages doubt-- regular account +8604|Supplier#000008604|8dGdxJL2deTp6g3w4O1UVtwa7s|11|21-475-947-9256|3015.07|are. unusual pinto beans cajole. unusual accounts h +8605|Supplier#000008605|7rmF4yhCBdlgGjnkCb|4|14-543-917-7299|6248.69|ans. blithely ironic accounts alongside of the furiously even instructions haggle slyly sometimes +8606|Supplier#000008606|jfDNiQE7AIxSTXJQkTUSIpAR0 O|16|26-761-422-4132|2785.40|o the slyly unusual theodolites. furiously regular requests snooze furious +8607|Supplier#000008607|gBRMYUGPTJEyM|22|32-142-780-1684|7190.45|e. quickly final instructions sleep among +8608|Supplier#000008608|XJzhJVBTZ4ZcI3|11|21-566-927-2651|1925.03|ns cajole. fluffily express deposits cajole blithely carefully fi +8609|Supplier#000008609|,5hiAiES6wzPTZGZBHWfTYE,5|14|24-240-591-8887|981.87|packages sleep slyly furiously final requests. pending, ironic requests haggle blithely fu +8610|Supplier#000008610|SgVgP90vP452sUNTgzL9zKwXHXAzV6tV|3|13-107-134-7845|307.37|lar theodolites sleep carefully +8611|Supplier#000008611|I8aC,ltMm4sWamT|15|25-311-340-8226|4944.78|c realms. blithely express deposits within the furiously final deposits nag blithely after t +8612|Supplier#000008612|jQEKalzEiDSGUtUPCFnrzYWUzpQsM3GX|24|34-465-293-8301|-92.06|its. fluffily pending ideas x-ray across the foxes. fluffily regu +8613|Supplier#000008613|yRkoyFqUby8b4jyX 4IX|23|33-912-140-2009|1667.07|inal platelets. fluffily express accounts haggle. furiously unusual pinto beans according to t +8614|Supplier#000008614|S6pUUI5mLe64LTUPO0mB4DAJuTFTqs9|8|18-644-661-8573|-612.30|unts are behind the stealthy requests. stealthily i +8615|Supplier#000008615|XBXVCbAk2EjUFFw0EEBAz5b19yUr6jC|16|26-882-209-7376|3022.22|hy platelets. furiously even requests along the furiously bol +8616|Supplier#000008616|XM,Qh xf0WHFR|11|21-407-501-2425|5500.93|quests. carefully silent foxes cajole ironically quickly +8617|Supplier#000008617|PQqLpfsDoVloc qIy5Kmu,ooCuE0bR8M9n5AO16|24|34-223-417-5965|8084.05|hely express excuses against the slyly ironic instructions solve slyly special pinto +8618|Supplier#000008618|SAyFf3D0yHU3kine|10|20-200-765-1405|5766.43|ular, unusual foxes. blithely express deposits detect slyly packages. caref +8619|Supplier#000008619|hFUNwmm94nsn9O1aSAwu6r|21|31-979-362-1298|4329.89| blithely regular deposits integrate fluffily carefully regular requests +8620|Supplier#000008620|fnE6fyXgoqfP4aqx1LRbQiqYm7854p5jhuZ|7|17-623-256-9547|5379.61|lyly busy instructions. carefu +8621|Supplier#000008621|tmEg6ez5rYYDxek4e6 NTwmtAGri3,|1|11-797-806-5347|4175.93| eat above the accounts. i +8622|Supplier#000008622|OiLHSLVlZ9r|8|18-380-541-6284|2896.32|nts. quickly final accounts use blithely furiously bold multipliers. theodolites +8623|Supplier#000008623|F8LwoBXK01DqdKGPkaHVwltbz58R|19|29-265-595-4732|1392.40|inal, final deposits. foxes +8624|Supplier#000008624|kq J6H4TrKRsmy1tAH|17|27-682-620-6179|3056.42|sly ironic asymptotes detect furiously. regular escapades sleep slyly carefull +8625|Supplier#000008625| Nvi5vjlqXyFKg|10|20-223-114-1661|8146.06|final platelets. busy theodolites nag bli +8626|Supplier#000008626|NlmyS7HMothrQwlUN7JxDo4|17|27-285-765-6320|2894.37|even theodolites promise about the silent waters. quickly final pinto beans haggle +8627|Supplier#000008627|RqW2ksY22ddo|20|30-470-519-9322|1732.50|deposits. regular foxes against the daringly ironic deposits believe ca +8628|Supplier#000008628|53g2KXKl4TjAvGWrkUNqQJuFGiOoHwsT, D|21|31-422-517-4839|-894.23|old asymptotes sleep boldly. carefully +8629|Supplier#000008629|6xEvD9WJffahEGHuMnyUJHYgMP5FVtuh|17|27-254-940-1023|5555.83|s haggle final, even accounts. bold +8630|Supplier#000008630|CaT5FxnCwIcQf2gCXkbZKuREgd YxuhdI|1|11-530-317-6499|6833.50|y ironic dolphins. ironic accounts nod furiously. blithely +8631|Supplier#000008631|CYmjUKWaOad39X3qc HB|17|27-858-492-6923|8617.51|requests sleep along the doggedly regular dependencies! packages haggle after the +8632|Supplier#000008632|QqHtvzhTWJlr7SJm1n,bqauRGd5XFIlO|19|29-543-253-5866|6502.62| deposits cajole fluffily ironic packages. furio +8633|Supplier#000008633|rAzhw4hQnyFPiAw911|10|20-387-254-9384|2596.74|usly above the furiously express requests. carefully ironic instructions wake blithely bold +8634|Supplier#000008634|jxLgpOMPQ,M71SfOJq7UF1QXaMf49HQw5|7|17-950-885-6667|4316.99|ecial ideas. blithely regular deposits eat. slyly unusual theodolites unwind slowly at the +8635|Supplier#000008635|zek4Jz SrK3|24|34-857-521-3294|8637.46|nts. carefully final deposits doze along the deposits. bold, even accounts print +8636|Supplier#000008636|2xyuzIpfVD49Bym5xienpkGwE73Sf4oAd1x52hm|11|21-992-222-2036|900.99| the express theodolites. fluffily express instructions use about the ironic somas. b +8637|Supplier#000008637|OBDD5mzy,Mo6IaAiey9ueghVppZT8lk,6F|7|17-830-250-4442|4800.75|y final packages. regular +8638|Supplier#000008638|yxj50B 8aMql|21|31-289-423-9527|-849.21| special ideas cajole blithely bold inst +8639|Supplier#000008639|S8EwLL3ljcU48,CbwUxNM9I3 zEu69qNHq4jcAC|17|27-736-446-4369|3221.19|uriously regular platelets. express tithes detect. blithely ironic pl +8640|Supplier#000008640|cCVH 156kfryb|23|33-316-349-3061|5444.00|r accounts affix blithely along the slyly special deposits. ironic theodolites are bl +8641|Supplier#000008641|oniPdoQP2uFDE0T|24|34-864-285-5244|8243.47|leep according to the slyly ruthless pinto beans. fluffily unusual Tiresias are car +8642|Supplier#000008642|qnN9N9du9Dg2arf6kjD xW0DjMT9cM|21|31-264-558-7831|991.69|inst the special, regular tithes. express accounts among the blithely regular the +8643|Supplier#000008643|gkAf5dYBg,qviByyDP1QVuY MMOZdxq69EurhLk|7|17-172-931-9649|243.62|regular instructions cajole quickly. +8644|Supplier#000008644|Wb7d79W7q4WVSmDtGhSeTCD3mN9X|7|17-426-413-8639|458.75|fluffily even foxes sleep blithely. final, regular ideas use? +8645|Supplier#000008645|FGe2D8JCtxfT0FE0VHc|2|12-714-202-7354|8111.34|nusual deposits. furiously special deposits integrate slyly along the regula +8646|Supplier#000008646|6ZPPU0du2w8R 9E001GhYwAjSmxLwTL52ByD0|11|21-600-942-5110|2390.83|ckly ironic ideas. slyly regular asymptotes after the +8647|Supplier#000008647|yxVUE6STSCSj9|8|18-450-355-8251|8640.53|ckly regular deposits. carefully ironic ideas about the slyly even deposits wake blithely abou +8648|Supplier#000008648|8bi56LCzClknqW6Ocs8swrzAHo58WUShU7mCH,|5|15-223-150-7533|8410.47|as cajole furiously fluffily pend +8649|Supplier#000008649|ijmXpUdw4SrxvD,qQ,Iyu,umt|14|24-308-452-6226|2831.08|ld packages. quickly final accounts affix c +8650|Supplier#000008650|HbYa1Cb49 DQ3E8wBO|9|19-467-523-3923|5002.57|-ray unusual depths. quickly +8651|Supplier#000008651|pfw32RGA7BPXrUiavYqE|21|31-838-626-4206|3282.59|ar excuses use silent frays. blithely final packages must have +8652|Supplier#000008652|WIfOji6z Rla4YS0YMf5af67OFLQRcGXVvl1Eh|15|25-821-352-3769|3532.81| bold asymptotes. deposits wake slyly furiously ir +8653|Supplier#000008653|Wn2zVNWn47ofgEd1YMCcdDcKyUMgtoCusxHPGEY2|4|14-106-598-8327|6916.02|p regularly after the unusual requests. blithely regular +8654|Supplier#000008654|3dYcEMsFs9Ek f6w Bby2f|14|24-564-233-1813|9591.68|rts. regular, unusual theodolites dazzle along the blithely f +8655|Supplier#000008655|h78,rbPDGunL|14|24-881-877-8158|422.50|lent ideas haggle regular dolphins. carefully special warthogs nag. quickly regul +8656|Supplier#000008656|sMvnqSyJogRS2Cdfrjz85bh2KgG|18|28-315-800-1023|3811.95|s sleep slyly carefully final instructio +8657|Supplier#000008657|FopH1MYnZM6Ns7OWC nEy8KiI3KBnjZt9YrySp1n|12|22-324-245-3548|1381.46|ccounts haggle furiously regular requests. qui +8658|Supplier#000008658|4gKA7Jmxr7GihkcbLX8vjldaCkVg|12|22-475-871-8177|6202.39|al packages. blithely final fo +8659|Supplier#000008659|ZydN4jeo3uXphe9cGLX24|15|25-293-120-7470|5595.20|y: unusual requests along the deposits cajole furiously +8660|Supplier#000008660|8O0v8,7La8RZZz,1mE|24|34-821-250-4127|8595.83|terns haggle blithely final dependencies. deposits haggl +8661|Supplier#000008661|K,uXFt NKM8r09Y1cje8cDRgz|22|32-974-820-9483|5248.25| ironic instructions are furiously blithely e +8662|Supplier#000008662|9AzWzV IXHY|9|19-382-865-3056|7410.93|posits. final theodolites into the furiously special requests affix bo +8663|Supplier#000008663|XYM5S,1Z4f2zUYxkxrj6xh5wo5w|12|22-572-622-3785|525.74|lly express foxes x-ray blithely regular theodolites. fluffily e +8664|Supplier#000008664|mmw03ZF7KUXN 75MMVf8v8xkcTqU8wuzYu|5|15-529-301-8528|1009.75|along the blithely unusual foxes boost ironic, unusual asymptotes. quickly +8665|Supplier#000008665|OeZ15ORrY8aqhuYWNGIo4g685newxhnnnl|15|25-913-283-7897|6767.21|e pending, regular packages. carefully ironic exc +8666|Supplier#000008666|D5NfQ1NwF3QDH6h6mkPQuEOa|12|22-993-304-7456|5583.01|ts nag carefully idle dependencies. final foxes nag according to the even, ir +8667|Supplier#000008667|EG3BUReUgU6UDA3JqClhk6oy3RJtBT|18|28-379-442-4120|1680.83|ts cajole furiously quickly regular packages. fluf +8668|Supplier#000008668|NTkuNJRa,UItC|16|26-319-194-2678|-305.09|ts. quickly special instructions run furiously. regular pinto beans +8669|Supplier#000008669|Hb7oSQVb6tpE2 3gD1OI7pJLNPu1CpDn,QmXP|20|30-353-587-9193|7277.81|rs. furiously unusual deposits after the evenly ironic instructions +8670|Supplier#000008670|4Iz1wROd7UvoCWlirbzq97Ddcgy|10|20-434-653-3719|7235.63|ld requests thrash carefully reg +8671|Supplier#000008671| VBlaRlruL4T4u6rHsi4FDhI|20|30-397-160-6507|2228.79|iously express accounts. carefully pending +8672|Supplier#000008672|8ZN9cxXmDwtVFx6MvqcPmx92DXz|4|14-597-717-8380|2216.90|ccounts thrash furiously after the even platelets. excuses doubt carefully. reg +8673|Supplier#000008673|7KORafFmc,w7Ii5LD86ONV46gcZBQGYnkdbTCsxm|2|12-566-515-4631|332.73| carefully even excuses after the carefully thin pinto beans use blithely ca +8674|Supplier#000008674|jbo9baZXGbu5zo3WcKAzGvEebXDVNO|2|12-736-989-3394|2190.37|fter the accounts? theodolites boost near the final, ironic courts. ironic, unusual d +8675|Supplier#000008675|hfXdVwIc6zdO,OLMWuSBjdwsPTw3eEp|2|12-782-314-1801|5447.59| pinto beans boost slyly around the carefully silent theodolites. fluffi +8676|Supplier#000008676|,Ch0bG pkQ0,F70Ei2Euz8HoF,NSIUwMInY|23|33-230-218-4163|2672.84| the daring ideas. bold theodolites along the sometimes bold pinto bea +8677|Supplier#000008677|WNa780JZzivxuGBEsDszqoT1Pj|10|20-899-256-5288|7401.46|onic instructions along the furiously ironic accounts haggle fluffily silently un +8678|Supplier#000008678|Fw0ipbBZ38EsmGcIEUjc5WN8ZWmRPScfYJnI|6|16-725-806-7361|6348.82|ully special theodolites. regular, regular accounts across the furiously stealthy dugouts sle +8679|Supplier#000008679|JWFVoSsCwn9p8o|21|31-579-345-1014|2707.16|quests haggle carefully along the ironic deposits +8680|Supplier#000008680|HrJU0nZBgS6T1GFC0KU9vf|17|27-694-363-1149|1208.75|n packages: slyly even theodolites haggle blithely besides t +8681|Supplier#000008681|UnLZKAlfh9ZOCiewQXkXGr3PxZSkM|17|27-206-598-2594|7446.18| after the pending packages. quickly final packages nod after the s +8682|Supplier#000008682|LwJN,ypokyyKljmTLMNLDvnFBpd|9|19-346-776-3515|9977.56|o the furiously final depos +8683|Supplier#000008683|gLuGcugfpJSeGQARnaHNCaWnGaqsNnjyl20|3|13-477-822-5129|6963.32|ly special deposits haggle blithely fina +8684|Supplier#000008684|68Mbg Y2i2lVN7hlYKamGXeTwn0utfsPJN|2|12-456-342-5923|1563.24|bt. silently regular theodolites use sl +8685|Supplier#000008685|zAUQv45LA3duYPJp14WlWQ7t9LA|15|25-831-849-5585|3394.75|foxes. express ideas boost quickly attainmen +8686|Supplier#000008686|4Onf4yxuNwHCRIC0y|4|14-454-946-4151|7465.41|ly final ideas. bravely unusual deposits doze carefully. expr +8687|Supplier#000008687|D8tZnhJ2li|0|10-969-326-2698|3234.47| furiously idle requests x-ray special pinto beans. bold, un +8688|Supplier#000008688|EsNKPBTXDMIAMcUb|15|25-768-203-1702|8756.62|eposits. ironic, silent accounts wake always. furi +8689|Supplier#000008689|BL31Awgb2UVMnqF4aNqOl3JEZKUEXY9|23|33-289-331-8236|1569.24|le against the carefully pending dugouts. +8690|Supplier#000008690|ejGqlTB0xgqntBcq3HE7XEJKkQN5|19|29-211-818-8088|5179.00|ual accounts wake slyly dogged packages? fluffily regular Tiresias sleep slyly. deposits +8691|Supplier#000008691|tqAgYLH3heebryx3buenTEzlpPx86Hz|6|16-671-409-5178|9489.13|l platelets. bold, pending theodoli +8692|Supplier#000008692|ke4YY0yawKS8 ICpJfIJt0tXGymxAgt|6|16-592-792-3475|3905.57| express theodolites snooze blithely final requests. fluffily final deposits alo +8693|Supplier#000008693|roxrGYHhr8alN5OCqh PkwypxqA|11|21-109-417-4331|1838.07|requests. regular, regular accounts detect quickly bold accounts. enticingly final packages +8694|Supplier#000008694|Kyg3DxOgIKpV5onB ATSdzFiIz42 0c1RrR|17|27-721-319-4631|2455.43|gle quickly-- carefully ironic instructio +8695|Supplier#000008695|ayqRnrqXMiHbYKUH,5HwAWmqrvBRW|20|30-139-982-3270|5355.96|equests. fluffily regular ideas boost carefully among the dolphi +8696|Supplier#000008696|s0I94iZbagQ6fezY97xKng6uwl7MX,|24|34-715-947-4920|2663.51|ke carefully alongside of the fluffily regular requests. thinly bold accounts sleep furiously aft +8697|Supplier#000008697|YFvi0uR6WTuJ74F,|7|17-823-489-3077|4589.79|ly express instructions! blithe +8698|Supplier#000008698|qKfxXqaDHZTG2CUJJdNW5BAO,FFNl4d1uHRyBb,n|10|20-339-957-6390|6607.34|le final accounts. requests wake slyly final foxe +8699|Supplier#000008699|gjx8vlb5xR3xvmE4EsJhwVxzXsy9vmVhM1ex|4|14-955-698-5817|2655.54|s. ironic requests detect slowly. carefully regular warthogs nag furiously. speci +8700|Supplier#000008700|YrxRNVhM6c|24|34-316-575-3775|2562.89|ng the regular packages. fluffily regular platelets haggle along the fluffily special pinto bea +8701|Supplier#000008701|PLR2QehcW08|11|21-747-984-4244|7460.80|ideas use carefully pending, final deposits. ironic, pe +8702|Supplier#000008702|OCrMw12KclPXTi3RWAEl2b5E|20|30-779-407-4353|2777.52|kages. furiously regular deposits detect final, express requests. +8703|Supplier#000008703|wMslK1A8SEUTIIdApQ|10|20-782-266-2552|6790.39|eep blithely regular, pending w +8704|Supplier#000008704|a6DjHp0B6mifKBtqUk,C|21|31-216-250-6386|-856.10|al warthogs boost slyly id +8705|Supplier#000008705|aE,trRNdPx,4yinTD9O3DebDIp|3|13-886-987-7629|8515.36|e carefully unusual dolph +8706|Supplier#000008706|JgrjzbA2s27F|7|17-588-630-7533|5761.85|ans haggle across the ruthlessly final asymptotes. foxes +8707|Supplier#000008707|5D AEM7mQ9KAf4oJ7Yp2G|16|26-333-620-6634|4355.21|ly ironic deposits. slyly even packages ought to sleep alon +8708|Supplier#000008708|Ot1ZKUnQN8v8jutLVlN,PiSpYogI|17|27-142-158-2848|2034.97| deposits would are ruthlessly. u +8709|Supplier#000008709|adZ18haKRYZMLqM Kf0GTGiBdrk|7|17-890-361-5346|661.76|ent, even deposits try to run. carefully bold reque +8710|Supplier#000008710|zA012bb21nlu0yg7ToFigpQF|19|29-453-928-9502|-828.89|arefully unusual deposits. carefully final deposits cajole by the qui +8711|Supplier#000008711|,jpE5oabRUIkfI1DQ3jfuSQkmveRdWNx|16|26-768-672-4810|3847.81| carefully final ideas sleep carefully +8712|Supplier#000008712|4H9DA2lf8jb qMoWLdSoG5E,bdMc,QpTcNCgTE|11|21-799-498-6669|-790.96|l theodolites ought to cajole furiously. deposits haggle slyly final, dogged packages. +8713|Supplier#000008713|WeDUkfczPEm6v5C05TEhDwA5c|20|30-310-765-5713|3326.30|fter the furiously ironic deposits nag blit +8714|Supplier#000008714|5m3huRQp5gCl95WoC7n4ilOgjUa|22|32-500-947-8893|1102.29|y ironic pinto beans. regular packages detect furiously +8715|Supplier#000008715|eCy0m5PA1DSreMUK|19|29-320-456-1546|150.77|g to the quickly final requests. blithely special packages are among the blithely bold requests; ide +8716|Supplier#000008716|UvYM29P,cel1kaSnsBfQY5xTjG|12|22-338-187-5741|1935.70|arefully. furiously regular dependencies nag fluffily. care +8717|Supplier#000008717|gcjJDIaOMmzn|22|32-771-832-4290|4340.10|sits wake. carefully pending packages could +8718|Supplier#000008718|sGJK5zefbUlWPbRGO7TrOjfaKf|20|30-768-427-6008|1028.13|iously unusual foxes cajole against the regularly pending instructions. accounts nag care +8719|Supplier#000008719|fhSlYHPeATB,InU 6nK3ksVra|12|22-132-922-4037|7810.82|nusual asymptotes wake evenly for the blit +8720|Supplier#000008720|urarbrSMfabAfLQHmFbgJqFvBqJbX4HP5GoQVb6s|22|32-374-657-9997|493.10| even packages wake package +8721|Supplier#000008721|jHo55qQkn9MejwcVT6|15|25-576-839-1734|2484.51|he quick realms. fluffily ironic foxes haggle blithely regular gifts. slyly express e +8722|Supplier#000008722|IntIqWtVWnMu6SRP,R60CIIk|9|19-162-892-4701|-935.11|st. regular forges haggle blithely according to the quickly final pa +8723|Supplier#000008723|NcYvPA3A0PFdc34jiiXxWY1|12|22-556-101-3402|442.55|ss the slyly regular accounts. express ideas haggle fluffily along the final, stealthy p +8724|Supplier#000008724|6jJp6wmmdd02GDHZynXHHwvbxB300hscHe|16|26-383-868-6894|-54.46| furiously final packages. excuses above the final, ruthless accounts sleep thinly according to +8725|Supplier#000008725|pRZKl 8XxGE|10|20-224-405-3175|6617.71|ourts! unusual ideas engage according to th +8726|Supplier#000008726|Mhvo,BpNYSHUuZ2khW B2CZ8GsjWM2b|9|19-670-767-3204|825.81|haggle pending asymptotes. blithely ironic deposits wake alongside of the carefully exp +8727|Supplier#000008727|bNQmfV296S5Wp|22|32-336-714-4000|6251.57| requests. silent instructions should have to nag slyly across the thinly ironic accounts. dep +8728|Supplier#000008728|akbOO5Qpw4HRYupOdz5jXK|14|24-638-576-4473|7159.66|ly. slyly ironic packages are carefully. regular excuses are carefully abov +8729|Supplier#000008729|wd91gMDPpsyfVd1WD2|11|21-824-778-8668|1445.94|ckages. fluffily ironic requests detect. blithely even d +8730|Supplier#000008730|WhTjwYrEw4QNCxp4Fhpc|22|32-267-482-1156|795.96|ffily unusual accounts. ruthlessly regular packages are. bold excuses +8731|Supplier#000008731|RA7nbWrxa zBDi7,Xw JEwJ9SypWkrv HV|18|28-559-108-8251|-282.26|y final theodolites wake ironic, final theodolites. quickly sp +8732|Supplier#000008732|se9LZxoJ6lPB Zole9,nJqgf7iJkwS|24|34-112-335-1831|5906.41|inst the express deposits. fluffily reg +8733|Supplier#000008733| DPJbqn40Q,|10|20-474-370-5903|3005.21|slyly carefully regular packages. regular deposits along the fluffily fi +8734|Supplier#000008734|tfvPtg,fT2n5EJ1hrXB,|10|20-479-741-7495|5528.77|ackages wake. carefully regular excuses integrate. blithely +8735|Supplier#000008735|4TuOf ,iMotRLNUYHGV1w4BFpSwBO0T9Y6K|4|14-456-452-9738|2998.09|accounts. quickly regular instructions nag f +8736|Supplier#000008736|t8IEmRy5byJl0tdlW54XK,uLmKSKrm4,hTgMi|6|16-266-163-6099|6959.31|quietly final requests. slyly unusual theodolites in place of the even deposits cajole perman +8737|Supplier#000008737|MsdGxF9Xoq9 8s|21|31-595-224-7696|187.61|lly regular deposits. packages integrate idly agains +8738|Supplier#000008738|mhYELX8lpq7PqZMwYE|0|10-725-307-1137|5147.25|fluffily final pinto beans. carefully ironic packages +8739|Supplier#000008739|vwv7Oaf9zPCO|20|30-741-515-8775|3878.91|ts integrate final, express accounts. +8740|Supplier#000008740|k9bBsgnSTHz3Vi|13|23-614-341-7438|4896.07|es haggle blithely furiously final deposits. careful +8741|Supplier#000008741|WjtVZrAoCEyCaEAFriztDDpYtfn|16|26-150-693-5535|6228.72| the carefully special dependencies sleep slyly about the regular ideas. accounts +8742|Supplier#000008742|HmPlQEzKCPEcTUL14,kKq|3|13-513-714-1723|8163.89|sts. ideas sleep furiously. bold theodolites alongside of the even, ironic ideas will use desp +8743|Supplier#000008743|MGEbOhOxKTbzrl9 |18|28-613-975-9179|5581.02|kages. bravely even frets lose carefully. ruthlessly even multipliers ar +8744|Supplier#000008744|aEie8GeP1xnWQVzqrRCMnGm|22|32-146-789-5208|1272.29|, regular requests wake carefully. fluffily pending requests among the accounts sleep +8745|Supplier#000008745|Ru150MO88CRlh1aSAWLjMOhMmiPK|5|15-816-554-6784|8365.59|ackages? ironic, ironic packages cajole slyly. i +8746|Supplier#000008746|rPFXKyYGgA M|19|29-179-898-1885|7504.88|lly bold theodolites. platelets cajole slyly about the regular, express +8747|Supplier#000008747|C0d1NNxo,pw|0|10-950-774-5576|6537.73| are regular asymptotes. unusual, special accounts boost-- furiously iron +8748|Supplier#000008748|CtrxObjXNfPadUaK|2|12-204-143-5759|9730.93|fore the blithely bold pinto beans promise bli +8749|Supplier#000008749|IIwYLAA1qEUAL|12|22-560-833-6117|8853.82| pinto beans affix quickly carefully pending platelets. slyly regular request +8750|Supplier#000008750|1SrKDBP9Y0PAguWqMUm7EJrtfqIOsVaTmfz6fP0|3|13-215-930-1101|7273.82|ing deposits wake alongside o +8751|Supplier#000008751|,KXqTzRq7IiSC|17|27-257-160-9803|6190.50| against the blithely final theodolites cajole bravely slyly special theodolites. furiously +8752|Supplier#000008752|4gV5Ggsd4tNyNAXbJ9XCmpWsAYE|5|15-243-897-4103|3690.55|ckages. ironic decoys wake carefully packages. quickly ironic theo +8753|Supplier#000008753|6GOqHvcUSO18Yc|3|13-438-421-9299|7087.54|quickly pending requests. unusual braids must have to use fur +8754|Supplier#000008754|BeUvgIWAICBPw4ROi|20|30-206-877-6439|-669.25|ely blithely silent packages. blit +8755|Supplier#000008755|ep9O7fTWC8yUFtKOIEHotrTZQQ4Kkkptpmhy|11|21-902-419-3429|2890.13|sits are carefully above the requests. fluffily express packages engage fluffil +8756|Supplier#000008756|cs50kLQEky4gv|19|29-880-355-6540|7469.73|r requests nag against the sly +8757|Supplier#000008757|Atg6GnM4dT2|23|33-821-407-2995|9721.95|eep furiously sauternes; quickl +8758|Supplier#000008758|iRK6pTtk,Vzf|24|34-437-869-6103|962.75| sleep slyly regular somas. ent +8759|Supplier#000008759|j3IjZ7KXrDXB|22|32-855-661-3280|8816.77| regular packages are slyly closely silent r +8760|Supplier#000008760|86y6w9qOzj1jihvaAXrPod12|19|29-415-515-1353|520.27|es. furiously bold theodolites are blithely along the +8761|Supplier#000008761|iIxXQmYUSE1KrlofumB XXsnjHh3|22|32-127-792-4482|8200.29|e furiously final instructions. special foxes wake carefully according to the fina +8762|Supplier#000008762|WiHo5Ibron|8|18-931-496-6922|1255.13|s throughout the carefully regular pinto beans haggle requests! furiously final +8763|Supplier#000008763|X7YENUZHNCMQ|1|11-596-985-3447|6003.48|ntain fluffily according to the idle, regular foxes. blithely pending deposits affix. fur +8764|Supplier#000008764|jwL8nL9LUXBie|18|28-365-512-5073|6593.01|ross the blithely final foxes are slyly along the carefully +8765|Supplier#000008765|h3oWHXVZDsJm zeB 8wusN0G8 AWr|9|19-284-572-4702|3777.78|ng to the ideas. foxes doze slyly i +8766|Supplier#000008766|RhYLwyTlHoNkIB ZqE8M,IBQlU PaLh|0|10-952-108-7676|-860.77| quickly regular pinto beans. fluffily +8767|Supplier#000008767|YwJ mUqcO65RA2JdRrwF|8|18-720-149-3929|6957.41|he ironic, unusual deposi +8768|Supplier#000008768|TbRn9LG mlMNrWqwiI,I HkeQRb2t4skLD Xc6E|11|21-662-883-4523|1191.12|t the platelets detect upon the carefully even requests. furiously bold foxes h +8769|Supplier#000008769|KiiYCl3QefsoUd,uv0oqOJ4EmWVpp|9|19-222-817-1767|8128.75|grate furiously blithely bold requests. even, final requests above the slyly ironic de +8770|Supplier#000008770|R7cG26TtXrHAP9 HckhfRi|6|16-242-746-9248|8929.42|cajole furiously unusual requests. quickly stealthy requests are. +8771|Supplier#000008771|7P7G1qq3hzBz8njAPU7wavtxWCJu|19|29-429-186-1033|1772.28|riously special instructions. blithely pending theodolites detect fluffily slyly regular waters. qu +8772|Supplier#000008772|8MSpT0p34itImmB3BUATEeRSD|18|28-918-973-5620|7661.38|ven, regular requests are alongside of the enticingly specia +8773|Supplier#000008773|v14CVyAvhD3jTY9,9gbpsHF,|5|15-786-171-2752|7855.14| warhorses. instructions use. furiously busy grouches cajole blithely carefully ironic requests +8774|Supplier#000008774|w07Lr99j0SK4|5|15-897-579-3890|8405.86|ffix fluffily after the furiously final pinto beans. +8775|Supplier#000008775|DmVHIRXagI9vIGM8ncRdHb9Di|17|27-469-399-7927|2402.37|oxes. deposits eat quickly about the furiously bold pearls. slyly bold packages across the fur +8776|Supplier#000008776|ga2ujpkg9PWoqsFsZ3mbKWYIez|4|14-716-695-4701|4624.68|y fluffily final ideas: furiously special packages wake blithely above the fluffily unusual excuse +8777|Supplier#000008777|9ypeSvnkjBbpy9r7c,BAQ82sxjk|12|22-865-963-9453|-918.74|eep pending deposits. carefully bo +8778|Supplier#000008778|NAhah5alhxI 6bed,eG5hSQH|18|28-704-955-1461|678.85|press theodolites sleep carefully along the unusual, final requests. iro +8779|Supplier#000008779|8X9mMxS9cMrpfOJbusgiSXQ sR|21|31-213-253-1116|412.93|gular pinto beans cajole c +8780|Supplier#000008780|AczzuE0UK9osj ,Lx0Jmh|6|16-473-215-6395|8046.07|onic platelets cajole after the regular instructions. permanently bold excuses +8781|Supplier#000008781|6o9yipNkc5i7UBCXlBwm9C7Qaq|16|26-285-518-8700|5356.36| sleep quickly final instructions; sometimes regular pinto b +8782|Supplier#000008782|x3s6V13vML5pegjYgoO9Hg8|14|24-221-880-2155|9362.17|old requests solve furiously regular escapades. pending dependencies sleep furio +8783|Supplier#000008783|M8ZImbCJo2jWJTGkhlG,yqQuc2I8tQi|14|24-460-391-2928|4098.04|ions. blithely ironic deposits haggle carefully thin, pending excuses. car +8784|Supplier#000008784|BZP73YHtEXj08SHlm5n0XuBYIhR35|0|10-969-638-1519|2758.67|thely even requests use bravely with the final, express accounts. blithely bold +8785|Supplier#000008785|bIMzqBjaiEgJsgwDF|6|16-451-960-6232|8889.94|ar platelets integrate across the accounts. packages according to the even hockey players sle +8786|Supplier#000008786|MoQKex3SHOANUI9czfzQAbv8Vei|22|32-460-461-4845|9369.06|ously regular packages sleep quickly regular dependencies. carefully thi +8787|Supplier#000008787|D4KuRSgCr1RnTzVLqYD 8WVo0oKVpg,AI|7|17-929-294-2305|9817.97|theodolites. deposits are furiously theodolites. slyly pending accounts haggle quickly. fina +8788|Supplier#000008788|xA,XBK9kurq75ZQZ pM18mUOjWLtoxRjnLvt4pUl|16|26-848-849-4077|8734.98|pending requests are furiously furiously special requests. slyly ironic +8789|Supplier#000008789|szBHT6,z09ycaFNnpwrt|12|22-232-450-7787|890.21|cingly blithely final accoun +8790|Supplier#000008790|9gYO31Lnq1ViZiP2ykAyTZONRXd|23|33-719-467-9957|-269.38|ests. bold pinto beans haggle along the slyly special hockey players. thin requests caj +8791|Supplier#000008791|ATBZ8n8glMXM2,3954OwbWuP1963AYet,|15|25-847-206-8366|2511.93|efully ironic requests sleep foxes. even dependenc +8792|Supplier#000008792|F0c2Z h0wL9A3D lBd4|8|18-678-946-7940|9024.99|en accounts. slow theodolites wake thinly blithely even deposits. furiously regular +8793|Supplier#000008793|Zny6qK91IFFBw9kl24|7|17-385-224-3603|2022.23| slyly blithely special packages; blithely final packages detect furiously according to +8794|Supplier#000008794|5PTH0cbNWbwhmQ0VDb|12|22-187-231-7328|6584.35|rding to the furiously even requests. pinto beans along the furiously express dependenc +8795|Supplier#000008795|dejD48ZPfOr9 v1Wdyd8dbNkqlNo48|4|14-179-846-5329|1276.88|hely ironic notornis wake slyly against t +8796|Supplier#000008796|BeuaOy,783yUTFakya45iURF8tPVQJv8 Eh|13|23-400-432-2971|8519.16|ts. final platelets haggle furiously. pending ideas +8797|Supplier#000008797|dyz8WERDG3fdz|22|32-474-915-2675|3540.29|. theodolites boost. regular requests play? blithely special dolphins cajole unusual asympto +8798|Supplier#000008798|Lttpv4yH8nCPzBHG2a92jG1HNiKgH79|15|25-985-744-7360|678.37|uriously. furiously unusual pinto beans hinder blithely instructions. +8799|Supplier#000008799|WqtU7S87t2gAAYFBxoojD|24|34-419-306-9132|3280.86|tions? even instructions across the even, special dependencies cajole carefully about the fo +8800|Supplier#000008800|mS5Z2UM4R,rK9xh6X25pPWM6Xq9gzo|14|24-713-468-6101|7391.92|ickly along the regular deposits. carefully express accounts haggle furiously according to the qui +8801|Supplier#000008801|BQyr1va pDvyXChto0bhj4ihxzqAv1|1|11-419-695-2043|90.29|y regular pains. pinto beans cajole along the regularl +8802|Supplier#000008802|xFN0wcWSTKZClB0q5KTj|15|25-772-695-8771|8356.10|bout the decoys. pending theodolites sleep abou +8803|Supplier#000008803|pECh1zz 2V7LNdU3IQal|18|28-484-998-4572|1972.34|ave pinto beans. regular d +8804|Supplier#000008804|yHTMzQqQv15km5f|13|23-354-178-6496|4026.52|lar, regular requests. express ideas haggle pending, even de +8805|Supplier#000008805|8KMSEbIbBRrg3cA9DaG,Q|10|20-803-123-4206|-288.97|accounts cajole about the requests +8806|Supplier#000008806|474Q,OCoa8ecS7v6IDJ91gm3RVu5|18|28-244-191-7061|3196.35|y according to the regular instructions. always unusual dependencies use qu +8807|Supplier#000008807|,ee56Ab,s58BQj,uYJvNiN|5|15-401-653-1126|6094.53|frays. furiously special accounts haggle slyly. stealthy foxes according to the notornis sleep s +8808|Supplier#000008808|7Iafu04b4xzs6Qea3KKqnCyHIuzdRbjLrx9xq ,F|7|17-977-811-4180|7734.53|ep furiously at the furiously ironic requests. instructions may haggle. +8809|Supplier#000008809|vE7N LgsRACSMSW0n09QrClYfU0ySPVxn5|7|17-200-955-1239|5221.57|ess ideas wake according to the blithe +8810|Supplier#000008810|,Ot93zDXOFSjWSKsKrT7XJ4YPCP,A|23|33-527-478-5988|9955.05|ily according to the carefully express pinto beans. unusual requests use quickly carefully s +8811|Supplier#000008811|yJCcdwVFkIU,AhfuFEhlEkekeLMaYw P|20|30-702-202-4404|2659.40| deposits impress carefully against +8812|Supplier#000008812|cZ7bt RoqkxevPt0RrP2cUYWn|15|25-628-821-7609|1527.83| requests maintain furiously ironic, even requests. blithely final deposits according to the blithe +8813|Supplier#000008813|jAzP xAkHA977iq7,|19|29-782-498-8163|2175.72|ccording to the quiet, unusual deposits. slyly fluffy instructio +8814|Supplier#000008814|G64p9bmN4lEar1sbb65|0|10-274-747-4283|1301.47|nstructions outside the even de +8815|Supplier#000008815|b43x,qwVG,dbkhA6c|6|16-138-616-8806|3797.86|ages use carefully about the f +8816|Supplier#000008816|VzifTr4jjSCXIPHsZ6e8lQN9M3Xf43xStQgnq|9|19-973-919-9040|6641.79|ress, bold platelets sleep fluffily according to the +8817|Supplier#000008817|Qw,zeAl2XdzYH9hWejZaSSKHJ2EQfEfqCfZz|16|26-613-991-8861|9674.21| final pinto beans. fluffily ironic pinto beans use s +8818|Supplier#000008818|hsZvpoAK6s,SZUJluvJXNijblDLCSa5DeFkQsW|10|20-625-642-4308|1583.05|r theodolites. regular theodolites are accordin +8819|Supplier#000008819|c9mTo4u5,PzinQycB4SG0L7ob3 crGkyb0|0|10-616-218-8718|6304.24|ggle carefully inside the instruct +8820|Supplier#000008820|uAsBvPBNsEsO|21|31-423-955-5613|1337.42|s integrate fluffily. bold foxes use above the express inst +8821|Supplier#000008821|zGWrzKUClXQqDxi0S|4|14-405-781-2565|3019.76|y even accounts. ironic asymptotes grow quickly blit +8822|Supplier#000008822|beRCtTw0gbVAS|4|14-368-107-1962|8011.88|otes. deposits haggle deposits. slyly regular asymptotes engage after the regular +8823|Supplier#000008823|66H6L3MOAH1|7|17-797-763-9282|2803.08|unusual deposits affix final, regular requests. ironic dependencies af +8824|Supplier#000008824|5swWGTWH3octudPQu0UZJFtIH|7|17-828-566-7930|7410.88|sts. pending, unusual courts believe blithely slyly ironic accounts. c +8825|Supplier#000008825|,MDHXkds9J qGeME5c|4|14-919-925-1562|-328.36|s lose. quickly even packages haggle fluffily inside the regular packages. final requests +8826|Supplier#000008826|0WNm3iYvmnAJkylsviMD7,VJCcig4j|15|25-737-561-2009|1159.39|ckages are furiously according to th +8827|Supplier#000008827|Z2K1FqSMEIVzaBRLYovwC7,kj8SAbHWEH3oIKM3|10|20-256-286-5898|2802.80|p blithely against the furiously silent orbits. quickly final asymptotes sleep slyl +8828|Supplier#000008828|DrOYzSGqdWIJOq0H|24|34-850-976-3222|2635.47|r requests are slyly silent requests. slyly special in +8829|Supplier#000008829|lNcY7xNLDonCw TuRYL|21|31-687-809-7474|6259.44| beans nag fluffily. final deposits grow blithely; special waters haggle id +8830|Supplier#000008830|sS4xPx2V4q0nCAZ7Z838vIHDL5|7|17-915-301-9134|7459.94|out the packages. final theodolites haggle bravely silent packages. ironic depende +8831|Supplier#000008831|jvIXgLiNdSXAni5POCMq1dk|7|17-307-245-8592|-748.19|y quick pinto beans snooze. regular accounts h +8832|Supplier#000008832|,oR1U6VqNk1AcT41I8K1Txzm1CTica5ocOyR94SM|0|10-814-704-9460|7050.21|sts wake quickly carefully busy accounts. pending excuses haggle carefu +8833|Supplier#000008833|ERwMW1YFF73uW|2|12-292-851-6826|9932.72| silent packages cajole pending pinto beans. unusual, regular platelets cajole blithely care +8834|Supplier#000008834|RLwqg,fShy8E341tKwog,6cZk8NXtnrXb5V|19|29-519-155-7628|1246.47|refully about the closely even instructions. quickly even pinto beans gro +8835|Supplier#000008835|RM5tOTZxosLn34JtphhF6yN|5|15-979-985-3172|3367.51|ding to the carefully unusual requests. carefully final accounts main +8836|Supplier#000008836|BR9s8T,rvpQTvMe0Z0IDe2p|1|11-808-231-8424|762.27| enticingly pending asymptotes +8837|Supplier#000008837|30kMU47oW4YJyT4BokEsln|22|32-165-179-1361|5368.40|w dependencies x-ray quickly pending decoys. furiously bold deposit +8838|Supplier#000008838|RWv,lKbrrm3bz,WCTEIIg giLx5vXahLv7ifyVzJ|13|23-384-338-6850|2783.85|special requests are packages; regular deposits are fluffil +8839|Supplier#000008839|vMrgJcstItZckedC5or|12|22-232-176-4074|3079.77|bove the blithely bold accounts are slyly thin theodolites. ruthlessly even instructions haggle f +8840|Supplier#000008840|aHUrIIcTaVuMfHZ|10|20-913-184-3064|4637.03|above the unusual packages. slyly regular asymptotes doz +8841|Supplier#000008841|I 85Lu1sekbg2xrSIzm0|3|13-996-181-3415|8503.72|le. bold foxes are quickly fluffily regular requests. carefully ironic sheave +8842|Supplier#000008842|GY1,SMmFs9YIZR5BOhgEce1PMekPHjmaBuuEiA|7|17-915-259-5382|8097.99|e. special, regular courts affix final, pending pinto beans. slyly u +8843|Supplier#000008843|bz0TKDXjotnAk|10|20-898-404-4723|8912.80|y regular dolphins. careful +8844|Supplier#000008844|Bl7,DYZRvmKHaw8v|11|21-640-498-1362|8417.34|ic requests cajole slyly among the +8845|Supplier#000008845|qA5PS4ZxBfRAIyfw GN okWf|7|17-626-359-3682|8301.44|latelets serve quickly slyly even requests. slyly express accounts hinder blithely inside the +8846|Supplier#000008846|rslOygSn4nNRHmYLWzt|10|20-491-907-4984|7370.37|ost closely bold excuses. daringly expres +8847|Supplier#000008847|1sq3ggDMjL|0|10-645-735-1948|784.74|fluffily. fluffily special instructions use. theodolites cajole abou +8848|Supplier#000008848|KyfCLN1DyhSeDYri0jglivIJF21KGgajR7drWs48|2|12-385-418-1713|9000.31|. unusual theodolites against the fluffily pending foxes detect b +8849|Supplier#000008849|jFdWH9Ur0NNdVsgWzSWnM8U|23|33-182-694-8043|5195.54|deposits haggle carefully. even instructions above the carefully express requests sleep furiously sl +8850|Supplier#000008850|eY7expDsb4x2Mbqfn8W7OI9hUZqne7krEDd|4|14-977-291-9923|1492.20|ual foxes. slyly bold foxes above the quick, ironic deposits haggle blith +8851|Supplier#000008851|nWxi7GwEbjhw1|6|16-796-240-2472|8348.74| boldly final deposits. regular, even instructions detect slyly. fluffily unusual pinto bea +8852|Supplier#000008852|4CECtC1ewnpi53etUsxb13NGY OOzPboCUCZWUjb|9|19-448-819-3230|-651.27|oxes. furiously special foxes integrate. slyly +8853|Supplier#000008853|w1eYDfbGI SkgAmzykyh1RV,bDiWQZI0mA3|8|18-477-737-7965|8063.32|egular foxes wake about the slyly even requests. slyly pending ideas boost. multipliers haggle +8854|Supplier#000008854|1xhLoOUM7I3mZ1mKnerw OSqdbb4QbGa|22|32-524-148-5221|9274.80|y. courts do wake slyly. carefully ironic platelets haggle above the slyly regular the +8855|Supplier#000008855|4Otz38w8BmX,F2TXhVE5TGV71QVj46tYWa|1|11-503-966-2355|9615.51|nal dependencies. express, special excuses are courts. foxes unw +8856|Supplier#000008856|ZjVab6fenuFGsr KB99 c4oUuqCNuIzO7ZLwdMO9|16|26-219-810-5660|5021.16|lly even deposits wake furiously slyly even reque +8857|Supplier#000008857|dP2th8vneyOLIUFwNBwqixkFD6|11|21-691-170-4769|7598.31|s. quickly ironic frays detect carefully +8858|Supplier#000008858|Dl3WWJKxEgBPRs9XOE0Uf36|16|26-491-699-6237|3813.66|ithely ironic deposits use furiously even ideas. quickly quick excuses wake carefully regular foxes. +8859|Supplier#000008859|AYm60aU0T5zRtNTfftYcs3a|19|29-123-203-7913|8237.64|ag alongside of the bold, regular depths: quickly final requests haggle slyly. fluff +8860|Supplier#000008860|nEc66WsYy52iPSvk9F2WJlM9SA|19|29-370-286-3141|2078.02|sual deposits cajole across the bold, ironic deposits. braids sl +8861|Supplier#000008861|5KfqtN,gqndgsAN7Cwu6|12|22-647-984-3631|9175.98|the slyly ironic requests. fluffily pending accounts about +8862|Supplier#000008862|uBkJv4e1 YsSjXYRl35EUDsLI1DOLdgPd5cXCgKI|24|34-790-873-6844|9730.14|mong the furiously regular accounts. permanently regular packages along the carefully re +8863|Supplier#000008863|C6S3M8dDWHt5cWuQZFy9|11|21-609-280-5505|6559.88| ideas. regular packages cajole quickly after the deposits. carefully final d +8864|Supplier#000008864|vnRhuJ8gsx3Bu|16|26-440-604-6656|8003.04|es. slyly permanent accounts hinder according to the +8865|Supplier#000008865|C78aexH4q5Vl472oOi5eeYKcgu5P54oe6MiQzCpr|1|11-869-386-7434|8189.77| around the thinly unusual requests. slyly ironic deposits nod b +8866|Supplier#000008866|XTeSFVA7CeJUKV2JxsUhtopN4dJyCekK|2|12-573-263-6187|448.62|lly blithely bold deposits. bold foxes boost carefully special asymptotes. quickly final t +8867|Supplier#000008867|1SwGoYme2bdYE8QEa72P|9|19-873-178-8581|2863.17|ckly about the slyly bold pinto beans. stealthily regular instructions +8868|Supplier#000008868|Fmk O6m9t6Cb71g29R4kPeVA4wX3v,tR|24|34-506-324-8506|5286.22| boldly ironic packages wake furiously unusual instruction +8869|Supplier#000008869|0RRJutNr4z1vrtxY0IsSE1lMiUlu8o|22|32-529-182-5023|7481.63|uctions hang slyly. slyly even ideas wake quickly. slyly f +8870|Supplier#000008870|VCkAyHLTaSIwU,bKuUnU3WQ|16|26-996-861-4331|751.88|carefully final packages. slyly ironic accounts nag quickly; t +8871|Supplier#000008871|ek1wFrXzUZd|0|10-512-544-3389|6733.34|d packages sleep blithely up the furiously even accou +8872|Supplier#000008872|8D 45GgxJO2OwwYP9S4AaXJKvDwPfLM|3|13-483-360-2131|8197.46| ironic pinto beans. deposits are even, ironic dolp +8873|Supplier#000008873|FkUth92uT1L60nkraEs DXHY|16|26-721-463-3377|8736.43| packages. silently express instructions must have to are carefully across the carefu +8874|Supplier#000008874|d0bXpR7xI8v4|10|20-549-739-3457|3195.93|ic ideas integrate slyly bold excuses. pending, final foxes p +8875|Supplier#000008875|hRdOqKqyU,sHq|19|29-132-904-4395|9984.69|ong the bold pinto beans are furiously blithely slow +8876|Supplier#000008876|vM,byK6roXWl3R|20|30-331-943-1386|5757.04|l requests use. even deposits must wake ruthlessly careful +8877|Supplier#000008877|4uhPAKRHn2VJl4usTdC5Z5sJY8oz|17|27-132-543-8195|589.05|xcept the carefully ironic instructions. special foxes play. bold requests cajole +8878|Supplier#000008878|x2izxmBfmTq8sqpHkyhJ97 c,Rafv|19|29-352-765-3956|9077.31|anent ideas. quickly regular packages are fluffily! quickly +8879|Supplier#000008879|rDSA,D9oPM,65NMWEFrmGKAu|3|13-477-693-9090|1393.64|ing pinto beans. blithely pending dependencies about the ironic deposits print according to t +8880|Supplier#000008880|7BuqIReCo4j9CBrVhUMTy7dUR3v|16|26-117-800-7468|1930.85|ironic requests haggle furiously regular asymptotes. regular, ironic deposits amo +8881|Supplier#000008881|fzNDxzR,NuetcuyF7apCP|6|16-933-140-9979|-317.70|lyly ironic, ironic asymptotes. requests among the foxes sleep careful +8882|Supplier#000008882|0ugA8kwP7Lk0fIl|20|30-179-819-9745|4364.89|counts. final, express accounts are. slyly even ideas wake quickly about the carefull +8883|Supplier#000008883|0T4fkeJc93HxWG vJ8tcR|16|26-392-523-2133|-304.83|ent packages sleep slyly about the regular foxe +8884|Supplier#000008884|XyXnag9SsmdLqlQ3gD|13|23-966-299-1843|619.25|p against the furiously even packages. carefully regular requests +8885|Supplier#000008885|WUeuKLfasHQqG6n6YePqoBc3b0XKxPb9OOai|14|24-694-243-9008|-254.06|quiet accounts thrash at the requests. slyly express pinto beans sublate. quickly regular +8886|Supplier#000008886|554vqNjeDjskG5D1k3OFPzV|9|19-409-597-5727|6857.51|regular notornis. even grouches across the deposits are across the accounts. slyly bold theodo +8887|Supplier#000008887|iUA2sUoyIHbUQ|0|10-479-795-5578|5159.42| ironic pinto beans. fluffily regular foxes print +8888|Supplier#000008888|sJHYd7hjH5EdLSPaOdm|14|24-834-151-1500|-558.10|ly final dependencies. accounts impress slyly; unusual ideas +8889|Supplier#000008889|csxAmnZKLnbPt63uPG,mSqsc|24|34-461-662-9392|1011.25|ngside of the slyly silent requests. deposits after the carefully express platelets affix alongside +8890|Supplier#000008890|6lmM3OrUukwhKXY0zqypO2qEsgj|19|29-208-398-4306|9990.05|ts. unusual deposits haggle furiously along the even +8891|Supplier#000008891|Yo,jbNbJGJEg|22|32-103-391-3580|7829.71|lly ironic instructions acro +8892|Supplier#000008892|wPqMedndVj|12|22-511-495-4839|-975.59|s cajole fluffily carefully ironic depend +8893|Supplier#000008893|E4JZLbc4QB4Env|19|29-957-742-1522|2257.37|s breach blithely about the carefully pending dependencies. regular +8894|Supplier#000008894|,FCImpJ5dB1KmqSHqi8FYGtsBsDZSXFaIwgbV|13|23-598-737-5141|9596.08|ously regular pinto beans. furiousl +8895|Supplier#000008895|2cH4okfaLSZTTg8sKRbbJQxkmeFu2Esj|3|13-845-389-5459|8339.24|osits engage fluffily according to the furiously pending asymptotes. blithely unusual theodol +8896|Supplier#000008896|HtivFouKEZTIhN3jeya8KEkx6rXnRoQ2U9KL9|18|28-292-664-6667|4712.01|theodolites. carefully special deposits are furiously about the +8897|Supplier#000008897|7nXqitelxx9z5dTw0|1|11-168-337-8356|3784.15|ructions wake among the slyly final dolphins. carefully regular pinto beans cajole across the f +8898|Supplier#000008898|qkoZBx4m 72,Svpfn8C1a b5bee0wYqUlqv,nl|0|10-111-184-7959|7148.05| affix at the furiously pending accounts. regular ideas sleep quick +8899|Supplier#000008899|MmWq5HAlpyN5pPGMwVDbsTLtADaPfG|16|26-739-911-2992|4324.79|es haggle sometimes at the carefully +8900|Supplier#000008900|0Y1MRMPSS5|20|30-249-817-6693|7709.93|unts doubt quickly regular theodolites! +8901|Supplier#000008901|80p4fwgJ1a|24|34-363-626-6781|9795.57|s sleep blithely about the slyly even deposits. slyly final accounts cajole furiously furiou +8902|Supplier#000008902| 0gpMGc1WtjF|23|33-399-835-7462|6180.84| the ironic deposits. even asy +8903|Supplier#000008903|XD2fYUucZTlX30lijkb3cvh0hVCjLuRDhMvv|10|20-687-493-3802|6415.78| foxes are carefully. slyly thin accounts haggle about the e +8904|Supplier#000008904|8QO ffx0CbnWjULKN6wqq4|11|21-555-308-2906|5971.50|egular braids are slyly quickly final foxes. slyly even foxes haggle quickly blithe +8905|Supplier#000008905|8sELF0amidZtqTcCoLJN|16|26-916-810-6563|8051.23|al instructions use furiously doggedly regular +8906|Supplier#000008906|3FcmY6MOlgAUtdft,UBp|11|21-622-688-8025|4023.49|he even, unusual ideas. quickly spe +8907|Supplier#000008907|ddhvFO xvUs|1|11-665-786-1699|3707.49|ietly bold excuses. pending ideas among the final, even instruct +8908|Supplier#000008908|wFlLRzBb9gRbP5rc6hbkqvJRqEoTFpnD|20|30-730-480-7056|5689.41|regular accounts about the blithely regular theodolites dazzle about the foxes. furiously pending ac +8909|Supplier#000008909|H1CQjeKFrzD7jPR|6|16-413-884-8531|7860.00|eas. even platelets solve agains +8910|Supplier#000008910|oNRhL5p axvIN|21|31-156-686-8260|2028.20|s. final, regular foxes after the p +8911|Supplier#000008911|CGtLI3B 2ifX,nx|2|12-373-160-1950|7936.35|nusual frays. even theodolites along the carefully bold accounts sleep quickly carefully fin +8912|Supplier#000008912|L8WkXrFPQqW8IoH|17|27-595-152-4380|9337.02|s wake carefully across the slyly special packages. blithely pending accounts are quickl +8913|Supplier#000008913|LLxdYf47tGHhBFE0cJEMhOew3gURe5cw|20|30-427-241-8847|-647.29|ickly regular packages cajole. ironic, bold foxes serve. always bold platele +8914|Supplier#000008914|huVxDVev2yTz2oWEYq8BW |8|18-278-359-4920|8916.30|s along the unusual, final requests are slyly express theodolites? packages boost agai +8915|Supplier#000008915|5,6vc4drzY8d1YxXJq7|19|29-754-306-3305|1297.27|c deposits sleep slyly slyly idle somas. thinly pending accounts nag carefully alongside of the fi +8916|Supplier#000008916|Y0pCkSR5qPKd9TKAgQW0SvgvwTztALd Q|18|28-518-591-9103|-877.00|c hockey players. carefully regular request +8917|Supplier#000008917|pIcGMSyzm70yjgPCi3UAV69fEm|5|15-688-918-6080|9245.10|nts mold fluffily quickly even deposits. ironically unusual deposits use. ironic fo +8918|Supplier#000008918|G2CjrWuIMVgdjvkc8UHCaiieZtXtzpOkI|19|29-850-457-3217|5397.33|ts boost carefully fluffily express escapades. furiously regular pinto beans cajole around the +8919|Supplier#000008919|iWIBnOtCBTC|23|33-104-509-4730|4905.58| the enticingly special ideas. final ideas nod. furiously unusual theodolites +8920|Supplier#000008920|yo9SyYc810Cw|10|20-362-919-2936|1149.11|cording to the regular, final packages snooze accounts. +8921|Supplier#000008921|87BMUPQmff|8|18-524-643-3312|6030.20|en deposits cajole slyly ironic foxes. carefully express platelets are? carefully even pinto bean +8922|Supplier#000008922|VRN0sxKKGkgFA6HDzSbRg9HF|14|24-251-988-7722|4622.68|bravely unusual platelets haggle blithely pending deposits. furiously bold dolphins wake slyly. +8923|Supplier#000008923|B4ulQi18cpme7e,PM7bjKNDCd,Ev|19|29-771-622-8950|9337.05| slyly special theodolites after the furiously even accounts are above +8924|Supplier#000008924|htQDIHQnJZdOjR3xvE9GmDiKNVau5xD2,Qzk|14|24-398-268-1019|-174.66|ly ironic requests lose. regular, unusual asy +8925|Supplier#000008925|JkpcVQHdedQBOKf02mpjwN|1|11-817-315-3677|-516.67|ly at the carefully regular dependencies +8926|Supplier#000008926|wR2CPQNOtUVJDc3j1NsGrVMjjMHz|24|34-685-727-6892|4302.67|zzle furiously. bold accounts haggle furiously ironic excuses. fur +8927|Supplier#000008927|LgwVct4dJnK59FBF50U4lvb6lNGeO9iD9lHkz|0|10-485-509-9864|-995.53| are quickly after the blithely regular +8928|Supplier#000008928|8j1TeOM1tpIQTYlUyrpKo1A|15|25-632-171-3756|8058.56| courts. special deposits are blithely above the quickly even +8929|Supplier#000008929|MCstA7Wb7ospec7xjTor0PbFyPK34PV Z|4|14-468-661-6827|6342.59| furiously ironic instructions affix slyly. slyly quick excuses cajole about the asymp +8930|Supplier#000008930|LzjamrXfmHRF1Q6kim9fZtxWvH,1i8|4|14-230-813-3108|7929.95|al excuses sleep quickly according to the express theodolites. furious +8931|Supplier#000008931|xKyDI8qEWM1ebSB3Q|23|33-790-149-3116|789.36|s cajole along the pending, special accounts; acc +8932|Supplier#000008932|kfP7XX1vCDj0O5ptFfl17Aj4NpMOM8ECGi|22|32-414-595-9654|9740.15|ns nag across the quickly express dino +8933|Supplier#000008933|pxGLAqRcQic9|7|17-767-807-2723|9726.66|t the closely ironic packages. fluff +8934|Supplier#000008934|x0EtN3W3X5qm3Fvw3Lls3XzZjeqFfinM7afPFM|17|27-501-548-3135|3027.51|nstructions. doggedly special deposits sleep. furiously even requests sle +8935|Supplier#000008935|J3i9FUPbAGRVI4NmuSRPg9,xPWIjd9zLZP5,,X|22|32-655-240-8264|6380.05|thely according to the deposits. pending accounts cajole doggedly. even packages are furiously regu +8936|Supplier#000008936|OkNmkz3NUGOKZ7j1uTJkR,|0|10-121-799-8759|-519.42|iously quiet accounts among th +8937|Supplier#000008937|3IBszYLXSHEToRon3,0OpNaJ7ucO|13|23-700-179-3836|2112.43|gular deposits. fluffily bold requests haggle dinos. slyly silent ideas use fur +8938|Supplier#000008938|AymnssphwTHTJTy|7|17-363-294-6578|4552.61|its hinder. final accounts integrate. furiou +8939|Supplier#000008939|FNG4YBI38pq6U6jLU,|21|31-442-299-2419|8653.24|ual pinto beans. carefully regular dependen +8940|Supplier#000008940|ZRUVrM9hL9AOpDy0VPgwBhDQy2jxzMiha|9|19-716-184-4972|2053.90|s cajole slyly according to the f +8941|Supplier#000008941|MNZt7UI3liKN43oH2Bvf|0|10-858-545-3580|-176.57|ages use. blithely final foxes above the ironic, regular dependencies haggle quickly slyly fin +8942|Supplier#000008942|BPUsPbcwkiHWoh6pEwF00vhzrWFw4c|18|28-435-943-3392|260.51|al instructions. final requests will sleep unusual, final courts. blithely pending deposits thro +8943|Supplier#000008943|v04SKvDoY 7FHfPaXAo6UKmj |4|14-496-196-5336|9342.87|nding deposits cajole carefully final requests. regular, special fo +8944|Supplier#000008944|YaOp7MxYc86lFM3Or5zBTU5|8|18-493-131-2030|9704.66|lar requests. fluffily unusual pinto beans sleep fluffily. bold packages nag furi +8945|Supplier#000008945|M7zOSL0ac2tZ8p6cT44PyWXsqpdN4EVIntCI l|6|16-623-156-4343|7231.61|sits cajole. fluffily regular requests haggle enticing, special dependencies. quickly regular +8946|Supplier#000008946|H5GRyJBvI3mZJ6w,1Qp82ioQR7Q WWWTR4qa0|22|32-556-539-1611|679.51| about the silently regular packages integrate furiously about +8947|Supplier#000008947|1Ij3T0egGHnVbLich98HzY,UeCdVbxzYa ZpKDVc|21|31-694-176-5568|540.50|t the unusual, express excuses. slyly ironic foxes kindle blithely by the final excuses-- final req +8948|Supplier#000008948|wdFaYH8,iV8f22gboep6eSEeae|4|14-719-754-7660|7130.94| sleep quickly unusual, express dep +8949|Supplier#000008949|bnFmDdpFen,dYubmHvuJuxQkaSr OANifA9Q|23|33-510-996-1338|2781.49| cajole blithely alongside of the regularly final packages; slyly +8950|Supplier#000008950|xfFNQbegOa YSDpfF60Y03YkB2YrG8|23|33-952-437-4497|7412.99|kages haggle furiously across the quickly ruthless dependencies. boldly final theodoli +8951|Supplier#000008951|IfKLAjU17yYGF2 PBWnT3idBkNTXEYG|20|30-624-675-4219|2220.03|ccounts are slyly fluffily regular pinto beans. furiously regular forges after the q +8952|Supplier#000008952|Pieung,FW1704mtb|5|15-856-379-5065|9024.58|y even ideas. ironic theodolites use furiously fluffily express pinto beans. +8953|Supplier#000008953|KBTuuAT0z0oV7YFa6LDp6|22|32-992-603-2445|6160.50|omas boost quickly regular, ev +8954|Supplier#000008954|kjJuxAFghboJrKxgvHpKqY,|6|16-182-421-2847|5617.83|dependencies. carefully close packages about the instructions a +8955|Supplier#000008955|4lmGGLzPP4Sommbd m|8|18-290-400-5130|3844.28|ic, regular accounts. careful requests nag slyly quickly final excuses. theodolites +8956|Supplier#000008956|32JIIlXqj1TFk8jMT3HJm|22|32-954-853-1419|5767.90|nto beans. stealthily final +8957|Supplier#000008957|f2v0skC2m3tQt8heKXi6aficqfV69gFjzQ7o|5|15-119-290-7994|7061.49|ogged requests nod. accounts b +8958|Supplier#000008958|N9yYTKnPjC|21|31-489-620-6997|1525.51|sts above the slyly ironic packages affix slyly +8959|Supplier#000008959|5DAHB95,c7Z9t9zjVx78DtgqMtljpBRVz Z|24|34-401-708-8154|7195.86| affix fluffily along the excuses. quickly ironic accounts wak +8960|Supplier#000008960|aLOLJsGDBjHCz47Jr7BYO4|5|15-187-740-7440|4824.60|ions play carefully enticing requests. furiously pending tithes after the exp +8961|Supplier#000008961|q6mrSY0F8JyJvhzv My74ly7Yy79860Wmi|24|34-880-655-7105|4042.24| silent epitaphs after the instructions haggle careful +8962|Supplier#000008962|YRmr64nSfVmc7jsSoS5v0gwkYxF9x|17|27-455-200-1575|-32.25|ckly final theodolites; furiously sp +8963|Supplier#000008963|KZ857 LeUmCEMifRs HbCYgoAS|4|14-809-744-1506|5826.91|ns sleep. even, even packages +8964|Supplier#000008964|U2YJW,Y1xCbUWbjuovtzsLfsl|21|31-984-439-7577|8838.80|lly bold requests. slyly bold pinto beans after t +8965|Supplier#000008965|zNFbwL0YhoMGeIiXO5Gztn939Xf|11|21-653-619-1628|7829.61|e furiously ironic dolphins. bold, even ideas according to the theodolites snooze c +8966|Supplier#000008966|xORytrohBHFNzuRMs99jyDtg7k4YlPZ4QnMNnGOD|3|13-897-292-9766|8305.88|regular, even foxes haggle slyly pending dolphi +8967|Supplier#000008967|2kwEHyMG 7FwozNImAUE6mH0hYtqYculJM|3|13-478-661-6000|4926.77|es print slyly by the furiously final excuses! blithely pe +8968|Supplier#000008968|HjS7aiphdniN4juQfKdzgH NH3|8|18-753-319-6468|5268.22| special theodolites. furiously even packages cajole slyly against the final platelets-- +8969|Supplier#000008969|Bgr7cJ0x7Ngg1JjVZ|18|28-370-761-9561|2025.98|quests boost quickly sly, expres +8970|Supplier#000008970|T88G0V eBY8OWY1kSFy|17|27-209-992-8141|3616.49|packages nag slyly final somas! ironic requests sleep blithely about the dogge +8971|Supplier#000008971|xN5JICikNyR3yJbnGrNDqQm2kmxvV|17|27-156-524-2156|5288.09|egular accounts. blithely regular deposits boost along the furio +8972|Supplier#000008972|w2vF6 D5YZO3visPXsqVfLADTK|3|13-579-975-1539|5997.13|usly final requests are blithely silent, unusual accounts. slyly carefu +8973|Supplier#000008973|t5L67YdBYYH6o,Vz24jpDyQ9|22|32-188-594-7038|9852.52|rns wake final foxes. carefully unusual depende +8974|Supplier#000008974|4JCXOJ3MyPfa51mIf,MQu|21|31-814-423-5055|6446.54|g to the blithely unusual accounts. carefully even requests integ +8975|Supplier#000008975|HhUfGgfjarTz62JZuP|24|34-457-581-5461|6060.09|he regular deposits cajole always across the +8976|Supplier#000008976|7sKjqX YeRyPJcjPLzrPFcAcMW7sXu9j|4|14-494-851-1873|1027.67|kly. accounts wake about the special foxes +8977|Supplier#000008977|c7R3uHqoS3tt6I4JIsWvf9EMP8GSGoWIqE|16|26-981-837-3494|-70.03|hely across the evenly bold pinto beans? silent, regular platelets detect. +8978|Supplier#000008978|5re RpCB7z,qMIxho dWfbu|6|16-798-575-4529|8938.87|across the furiously regular requests. furiously sly platele +8979|Supplier#000008979|kuRpGXGhLhHVCPsELzu,WtV0NZg5VdJ|1|11-447-925-1778|6638.05|rhorses. blithely pending platelets use special deposits. regular packages are quick +8980|Supplier#000008980|80F jrm3i,,aQIt4wmBJsjWup3fHC8eGxhKW3qy|23|33-123-315-3071|1810.74| carefully alongside of the slyly pending accounts. blithely silent dependencies cajole? +8981|Supplier#000008981|LStHoDdqo5P7Hbds0RKTUBFfs5R7ZlQ49|24|34-433-517-7761|5079.63|otes wake slyly regular requests. busily ironic foxes nag carefully bold, iron +8982|Supplier#000008982|mDfqeo0DZdIPRWTD38VJkq|6|16-686-961-5499|365.36|s cajole quickly express requests. spec +8983|Supplier#000008983|zs375dKUp,IYq9GE918SNra|19|29-333-937-5715|476.88|quickly regular requests. daringly regul +8984|Supplier#000008984|Ap5g9fjZUuY0FnaRxsqaZ|19|29-569-265-6236|4852.77|lms according to the ideas wake +8985|Supplier#000008985|DQlo6bAOeylGzburi66Io6dtsqX1rU3,rrEyZ8i |19|29-284-229-3052|4584.16|ajole according to the carefully ironic packages. blithely reg +8986|Supplier#000008986|bNndZKvRwEdZ47NsGFFwv2NI CvCKNAvtHR|23|33-188-323-6178|-68.22|mptotes. furiously final deposits after the blithely pendin +8987|Supplier#000008987|eUg7nFSwAPTrQqCIsF6TWplQtSe98xcesLxZKe3|17|27-733-901-7416|6426.98|ly final tithes after the carefully final theodolites are slyly along the p +8988|Supplier#000008988|nwJvetiNIIbobMiCgQy|8|18-783-456-5180|7247.11|ut the pinto beans. ironic foxes sleep. final excuses boost fluffily. theodolites sleep ironic, +8989|Supplier#000008989|qRN1ZwuGg9dQk9WGrtYdgfaX6pVB wtSHBIXV|8|18-951-647-8754|8190.88|y express packages? idle, even requests affix furiously regul +8990|Supplier#000008990|R4rCANakSjWQphz1uupK RrgbsTh290|10|20-421-478-7010|8104.84|ilent foxes cajole. slyly even packages wake blithely. blithely silent requests c +8991|Supplier#000008991|jS19kC4v8fdR|24|34-558-984-6240|3170.26|sly unusual dependencies. carefully unusual theodolites wake according to the final depo +8992|Supplier#000008992|yJSqVZjqwBvaMt91zRpDVCy3tnAq|17|27-670-969-4243|7562.68|g the bold foxes. blithely ironic theodolites above the fluffily special accounts sleep +8993|Supplier#000008993|jWxltbLn01bkCt8uvu,oYGmw5bDJAOX3QSTY|12|22-701-268-5778|-927.99|counts wake fluffily slyly express requests. quickly express packages are furiously +8994|Supplier#000008994|,vztuBkzlz39NJLxjAMd|14|24-241-249-8119|-821.66|ic dependencies. foxes wak +8995|Supplier#000008995|EUivCq8ssbJPvvp2dcuus3Mma10XGw43ApNzO6|23|33-752-926-4838|4197.09|. blithely final instructions wake even, ironic foxes. blithe +8996|Supplier#000008996|xVQPDA0pVcs8zYUoxVkiW7hkNUVRsD8r|20|30-413-921-9925|6554.07|nto beans sleep slyly excuses. ironic instructions alongside of the carefully ironic instructio +8997|Supplier#000008997|KY MmMEcyQ6FEDCooFj xa uCwF2GbaeA8|21|31-775-577-6003|9367.10|ic requests. ironic deposits above the furiously blithe accounts wake quickly fl +8998|Supplier#000008998|cTCn79ziEcQz66KkdD Gil,VvaDHdXli|17|27-988-387-1160|9872.08| deposits. even, fluffy theodolites believe above the carefully regular instructions. foxes sleep. f +8999|Supplier#000008999|fpMNdR7m9gsgyg6jK6kwAAIQ|18|28-972-543-7523|5130.48|s. ironic ideas cajole at the pe +9000|Supplier#000009000|b0YEeuzRiGPLrPzAFHBuKiur4sV7Laxu,Iz6NAT|2|12-908-707-8215|7970.29|blithely close theodolites boost furiously about the enticing ideas. +9001|Supplier#000009001|5wn2LjwDEYMoUb0XDkbPkcap9aDA3JiQje|6|16-485-420-1200|7676.62|unusual packages are accounts. furiously final requests are carefully furiously ironic fo +9002|Supplier#000009002|8PAqvKqo00|10|20-360-566-1480|16.56|hely express deposits sleep. carefully special gifts according to the acc +9003|Supplier#000009003|svHUXJGOij1uq2L PYP9OFPmPdvixluPXYody|18|28-470-501-7683|1215.27|dle deposits impress quickly above the ironic, regula +9004|Supplier#000009004|nSdPFga2ARrshqSR Zqkd4VBTtv|18|28-588-104-4114|6958.76|es above the instructions cajole along the regular, regular de +9005|Supplier#000009005|zp0CwDNMpzJG y6R3hfxOq79|15|25-988-399-1040|1993.37| furiously bold pinto beans. blithely regular Tiresias must de +9006|Supplier#000009006|K8Cb,GE i2jpSmrq1jIhTOkuexh1GYmHurHYp|9|19-539-268-3770|5447.57|unts eat. theodolites whithout the quickly silent pinto +9007|Supplier#000009007|39UHgveOhYkEwzdIr,hDui1EcCnlhD6iF2wO|5|15-645-562-3510|2369.72|bout the accounts sleep blithely packages. quick accounts alongside +9008|Supplier#000009008|tqXQySU5c4Pvry|15|25-100-733-5634|5811.39|furiously after the daringly unus +9009|Supplier#000009009|D96aIrY6lR|7|17-148-808-2052|999.09|uffily about the furiously even foxes. regular decoys haggle sly +9010|Supplier#000009010|Ft95sVR9Tbr|22|32-690-758-7248|3728.90|usual instructions above the slyly unusual pac +9011|Supplier#000009011|eUVpuk35UDv6BtA9YI0,u4j lW7SbKuP0Ujvkf|1|11-690-330-1559|2600.88| unusual accounts? slyly regular pinto be +9012|Supplier#000009012|iUiTziH,Ek3i4lwSgunXMgrcTzwdb|19|29-250-925-9690|7937.93|to the blithely ironic deposits nag sly +9013|Supplier#000009013|MJ0bHctTH,qRRt|19|29-513-589-4374|2892.54|counts. quickly even forges cajole carefully across the carefully +9014|Supplier#000009014|VESdCwbtiG1mRHWvkdOZsmZJU|0|10-628-325-8932|3853.97| even packages boost slyly carefully +9015|Supplier#000009015| hqZvf3UAZhl0EgVZcm0|11|21-997-953-6904|3734.15|idle packages-- special de +9016|Supplier#000009016|WiPG0LZgNN9ke9JSyaeTh66OfbPj6OXmCYY5v|22|32-826-716-3859|2804.19|eans. carefully pending excuses haggle ironic packages. carefully bold +9017|Supplier#000009017|p7zwkIpBAKLmC7Wgd7LckT|4|14-170-101-4780|4046.45|iet deposits after the daringly enticing hockey players x-ray slyly i +9018|Supplier#000009018|3f2cOg,VutETPkzSo3cNTONdWkIkqzz |18|28-230-488-1227|4718.96|ackages. regular accounts sleep blithely platelets. quickly final somas boo +9019|Supplier#000009019|n8p5X,7yaHI8u72uyYVSqw |14|24-566-213-6246|-362.19|nst the silent sheaves. carefully regular asympt +9020|Supplier#000009020|JaTDiZo5RHvqv1K4wzwLFqTa|10|20-515-199-3757|4556.82|ons sublate slyly after the quickly bold frays. +9021|Supplier#000009021|2cI86dCJiou|12|22-277-653-5354|2556.42|tect blithely regular grouches. even, regular +9022|Supplier#000009022|Ep7Zzcxr9dHF pC|13|23-905-336-4879|8670.94|y according to the blithely even asymptotes. slyly regular deposits detec +9023|Supplier#000009023|,JQVPGb5Q7GiTz4Ys|8|18-579-828-6514|2618.77|he ironic warhorses. even foxes cajole blithely fluffily silent theodolites. slyly fin +9024|Supplier#000009024|rxNd8I3zet|13|23-418-146-3758|679.62|es. deposits across the slyly bold packages +9025|Supplier#000009025|F1HRl9BAzcEo,|6|16-328-186-9849|2383.86|inal accounts. even pinto beans sleep carefully alongside of the slyly ironic deposits. carefu +9026|Supplier#000009026|,yb 5xmekw|16|26-300-653-9615|1489.26|ay blithely. ironic, regular braids serve carefully +9027|Supplier#000009027|99FpBGy3asoY2Dkmd|2|12-436-880-3118|3014.23|alongside of the regular instructions; carefully pending requests wake +9028|Supplier#000009028|CGp5UiY8rUcMid6tcKWcCCGBrO G8b|14|24-488-812-5079|3195.87|y. carefully bold gifts against the carefully final deposits cajole special pinto beans. +9029|Supplier#000009029|tMbcraGBCsx6et j8|4|14-197-738-5698|8952.99|ies. ideas boost ironic instruction +9030|Supplier#000009030|kW2C1c1 MPjvOE3uSrgn|22|32-968-112-9297|1919.66|jole furiously across the ironic instruct +9031|Supplier#000009031|FhUkpgtadZ5h7QNKpj7qCP7TOVk8enoPBH,1a46|5|15-429-301-9156|9673.05|fully regular foxes. final, regular dolphins are fluffily carefully express packages. quickly s +9032|Supplier#000009032|qK,trB6Sdy4Dz1BRUFNy|3|13-441-662-5539|3982.32|regular requests use blithely express packages. theodolites cajole boldly along the ideas. package +9033|Supplier#000009033|JSTh,gcJINwyiY4o8wfW 9QZzu|17|27-835-350-7270|4885.34|to the blithely ironic accounts. idly final theodolites nag +9034|Supplier#000009034|oi,JOa0LITaOToR5L,G1eMDN8ZuhLERVo,3O,2|7|17-351-274-7494|1855.07|l accounts haggle quietly across t +9035|Supplier#000009035|8W kvlvsh5t,DgUFGKXMkaQdEAfF3UNxza|12|22-742-124-7874|2072.81|y ironic instructions above +9036|Supplier#000009036|tATCI,XNX4X7ED|7|17-262-703-4783|9702.45|egular foxes print across the deposits. fluffily even excuses across the even pinto beans are +9037|Supplier#000009037|TN4MfTE8ywhIE|19|29-665-258-1017|3125.76|, special ideas. furiously +9038|Supplier#000009038|Ij8T0D9Fdkga2TeG7e,s5yYkuG6YF|19|29-665-277-1087|7662.93|hely special requests after the furiously final pinto beans cajole slyly even requests: carefu +9039|Supplier#000009039|y6FMC2poCxAqi|1|11-980-238-8041|968.07|rs. special theodolites eat carefully. special, special deposits after +9040|Supplier#000009040| DCZDJmcqkjIuyF |7|17-411-550-2852|-686.32| carefully. even, regular asymptotes are carefully express theod +9041|Supplier#000009041|02NwJ8FFNyYurwCWNshQYd0xU|0|10-637-812-6249|1983.64|st the carefully ironic accounts. final, bold pinto +9042|Supplier#000009042|h7ZUczo BbywBsYcTB4yCmF|14|24-904-680-8272|7008.00|s promise quietly carefully bold warthogs. furiously iro +9043|Supplier#000009043|57OPvKH4qyXIZ7IzYeCaw11a5N1Ki9f1WWmVQ,|3|13-721-447-6674|4484.67|ate carefully. blithely unusual requests +9044|Supplier#000009044|IUQ45yv2CVGeB|8|18-532-417-2644|-208.23| until the slyly final packages. foxes sleep pinto beans. ironic asymptotes about the +9045|Supplier#000009045|XCXQUEIp36OKslCv7bLMnYnSxdAMPk3t|18|28-840-440-7483|7233.31|ul packages. ironic packages cajole quickly. unusual requests wake daringly ab +9046|Supplier#000009046|G9I6PPy6WcBpsOPcO4CMORttcmxvbedNG|11|21-928-727-4078|7218.71|ptotes cajole. busy hockey players cajole stealthily pending, regular acco +9047|Supplier#000009047|DbN8Q6vaNl3aFvDCEOSh9yN|14|24-999-545-7944|3872.62|ndencies impress slyly excuses. silent pains nag fluffily across the ev +9048|Supplier#000009048|ubrsBzWtANP80ndOC8AjYB9WX jiohjOdK0|18|28-818-961-8706|9184.96| the packages are quietly blithely express theodolites. final excuses haggle furio +9049|Supplier#000009049|g4dvfWujwb|23|33-903-164-9041|6668.15|ependencies are carefully around the ironic foxes. regular, pending packages wake car +9050|Supplier#000009050|g4HOYh1gUgrOTtF,75L|21|31-399-753-9871|1585.98| theodolites. ironic, ironic dependencies sleep slyl +9051|Supplier#000009051|rqjg3yDWpJlheWrvkAwt9iADnozWmy|18|28-823-689-4487|527.18|thely packages. finally unusual packages +9052|Supplier#000009052|,,TuA03FajJlRCj080K5aYL9V71cnqL4Pk|0|10-184-936-8351|3287.57|excuses dazzle carefully along the regular theodolites. +9053|Supplier#000009053|6h2P5,3j,zDoT|20|30-485-367-6698|-779.38|ickly regular dependencies. regular pinto beans play closely. quickly silent accounts slee +9054|Supplier#000009054|TN0Arq5KOkfmOhuZSi3lov hkDV|8|18-130-136-9982|-149.24|ckly along the theodolites. quickly final packages wake ironic theodolites. even accounts doze ar +9055|Supplier#000009055|h9R,KyJVzCBLgj9En5mxOd0sJL XWE|4|14-136-356-7916|8615.81|s wake furiously about the express accounts. regular, final deposits promise +9056|Supplier#000009056|J XULM9ci1dymQfPtHL3O35IDjm|22|32-702-268-2426|-32.44|deas wake fluffily above the ironic packages. +9057|Supplier#000009057|9X8yfLTApewvoHy5d|12|22-542-862-8502|2716.56|egular accounts cajole slyly slyly unusual requests. carefully pending theodolites are +9058|Supplier#000009058|JDi9yv70YQ,5GEB6|0|10-169-896-9715|5222.58|nts haggle besides the blithely unusual pinto beans. fluffily ironic packages nag +9059|Supplier#000009059|8hio036bAERfw4xaGtLAYlI0qfTu3Lsa1|7|17-878-236-8956|4918.07|sly across the even instructions. slyly regular foxes are carefully along the furiously bold +9060|Supplier#000009060|4i8KTf6YCwNcKtshi2m52R8NtJYxLpAHlCn|0|10-752-251-9004|2029.10|ckly ironic excuses at the quickly specia +9061|Supplier#000009061|DXyzBcTtJ0diaDFN K oiNw9ph5O0hZiF|6|16-875-881-3134|1104.05| packages haggle furiously above the special, final pa +9062|Supplier#000009062|LZtzjMg6m7D,q6XopVqVcy|14|24-231-209-5210|7727.35|, express packages mold blithely according to the blithely even packages. carefully +9063|Supplier#000009063|Bi2Bk7wvzgXHCAoqH6t,r|9|19-227-328-4842|5601.65|nding accounts cajole furiously. deposits cajole carefully fluffily regula +9064|Supplier#000009064|xbtD2V5S6e,rJIGNjRYPyl21u0e|10|20-368-975-7617|6879.57|es. unusual pinto beans are furiously above the slyly idle requests. carefully final dependencies +9065|Supplier#000009065|ZELuiqWrWbJV9zAuco1OnXKTJClhR|21|31-546-302-5610|1989.79|ages. furiously even instructions according t +9066|Supplier#000009066|e3MOn5,2mrRJ1FVYrPHNBhmZn|1|11-223-726-7237|281.13| accounts nag blithely after +9067|Supplier#000009067|TI,wbau X5HpCdASV9BqNPZRpeDlHQpvPzePTtI|11|21-519-914-2068|3840.40|nic foxes. even, ironic accounts affix alongside of the carefully final requests. slyly even t +9068|Supplier#000009068|ErjvNhg,CVml8krRQpo3zW8FYfsbf5iGv5lXGt1m|0|10-265-482-6893|6511.37|ets. slyly even ideas cajole slyly. even depths sleep slyly. regu +9069|Supplier#000009069|pJUQfs44ak8WCNutFki|17|27-890-324-3523|6643.21|ole among the express requests. blithely silent deposits us +9070|Supplier#000009070|vTNwNfUei2fgk52PgK92P8Bqb|8|18-966-167-9437|414.07|y final asymptotes. slyly careful +9071|Supplier#000009071|Eo44RjvwrB82VL1d32|19|29-403-776-9625|5260.43|. dependencies nag slyly deposits. slyly special packages nag carefully agains +9072|Supplier#000009072|SElQfI7y90sSiv7QyojGyuviKsga8xKExYAWX|14|24-420-730-4891|3017.74| slyly regular accounts nag at the bold, regular requests. packages boost. careful +9073|Supplier#000009073|JLQG8zebOLa|6|16-912-317-6753|7120.98| instructions wake ironic dep +9074|Supplier#000009074|RM4V5rxUXlczRAGtVhoxzja8KJEia8vX7D6USo|4|14-914-673-6699|405.28|sleep around the quickly ironic pinto +9075|Supplier#000009075|JDXUmrLAybQmdEDp8ANtLjWPdl|0|10-626-369-1805|415.13|ording to the furiously regular accounts wake above the packages. regular, pending +9076|Supplier#000009076|tPB3z0Il783Aok,|19|29-952-420-6082|990.16|ual, bold dolphins. express dependencies above the regular, final courts haggle s +9077|Supplier#000009077|TSKc 6qUKfGVMcGf|24|34-178-864-4858|7618.39|lar pinto beans haggle carefully carefully bold forges. ironic excuses wake carefully slyly regular +9078|Supplier#000009078|l4CvArLwcgpwaJ5oaX8uSr5KYA0lXn4v|19|29-495-229-3649|328.20|s was after the quiet, final ideas. finally ironic dependencies haggle slyly. the +9079|Supplier#000009079|WJ0NpgRuusAnQR871,TSE4X|2|12-833-892-9506|3075.73| dazzle furiously about the ac +9080|Supplier#000009080|rGU D6cSBNcw8CW1zVEx4qZd6BmF,,R|14|24-739-807-5601|2790.48|slyly regular accounts haggle blithely according to the express, silent requests. b +9081|Supplier#000009081|d1tzc5gzpNRMgTjp1gZrl|9|19-894-687-5796|4627.52|elets. silent deposits cajole +9082|Supplier#000009082|PBE1ryicleQTkevLvkfG|14|24-782-843-1885|2739.25|s nag slyly above the daring, special accounts. carefully express instructions us +9083|Supplier#000009083|wVIdyqHsk0UpdWUiW9YGz7HU8j|12|22-552-431-1228|1595.96|ly bold instructions. regular requests wake along the even accounts? requests +9084|Supplier#000009084|vFZS4BU4E8BTJ5r8PcrZFtc2pnIXm|0|10-830-177-5803|3284.77|cial accounts. slyly final theodolites detect slyly fluffily regular reques +9085|Supplier#000009085|3mbsprr7CRUTKfqiO3E|17|27-119-519-8096|7121.86| slyly unusual requests lose of the fo +9086|Supplier#000009086|nU6wxt80ORwxECbKGo9pX|24|34-265-660-6775|4097.83|kly express accounts sleep. care +9087|Supplier#000009087|R93KV7UdAptefzXjnqO9pfqgsV ZqpdHcC|23|33-722-513-3852|-722.12|oost evenly special theodolites. boldly even accounts integrate. furiously regular pinto be +9088|Supplier#000009088|w0QXNsQj44PdNP79s5KRcEuY6,leVVY|18|28-799-267-6993|5195.60|cross the quickly silent packages. ev +9089|Supplier#000009089|IZerr6dMVc RxftltX2p|10|20-970-478-3730|2390.81|ts. evenly unusual requests nag carefully. instruc +9090|Supplier#000009090|XC9q8e8jlbcAfi6CTFUwG6Q|19|29-294-757-5959|8254.50|press, regular deposits. express deposits +9091|Supplier#000009091|lbaCDQ7 x3WK,kz9P0Mu0wUcpqg1gaAIUU|4|14-169-838-5652|-559.38|arefully even theodolites. carefully final asymptotes are slyly final +9092|Supplier#000009092|xrTbMrl,fCoK|1|11-390-819-7605|1354.17|ding to the unusual requests. bold, pending requests +9093|Supplier#000009093|zRcf1ht4TZLlvqRBs8sBr7aJAfawyX3yvG8mg Al|1|11-274-769-7251|9543.46|e the ironic dolphins are furiously regular dependencies. slow, fluffy dolphins +9094|Supplier#000009094|OdXhqpW7stmrXA4fMHv8Z MAXv|9|19-209-370-2359|4071.37|uickly ironic accounts are furiously slyly bold requests. careful +9095|Supplier#000009095|oDzJLjn,eiHwYxuaZhwDB0|23|33-694-615-5432|5589.90|y quickly unusual warthogs. always even theodo +9096|Supplier#000009096|SsWOriAm1ybplhhyP|16|26-107-366-6452|7487.37|ven, express foxes. pending, even pinto beans boost furiously furiously final idea +9097|Supplier#000009097|SEYCPcjWKFr5EYfCfbL4WdHo7w9T3A|23|33-158-636-7238|9268.38|ress, pending sheaves. daring, final excuses d +9098|Supplier#000009098|O0dbLkI96ePTovMDZH0FZ9KzuTLv|18|28-339-166-6923|806.57|onic requests after the special dolphins boost bravely even foxes. ironic asymptotes are sly +9099|Supplier#000009099|lSP8yyJHNW6qK,ZjoGVkcdTlYS4frXc05kkHQN|10|20-578-352-5500|2019.55|rding to the regular foxes. regular deposits grow furiously. carefully special requests along the +9100|Supplier#000009100|aonr5DJ4olXNJ1AWK,CE411m6R7|0|10-159-653-1899|5524.99|ages cajole carefully furiously pending instruc +9101|Supplier#000009101|Up6zowOQBEhC WmO90|14|24-643-145-9641|5509.25|ly ironic accounts. regular package +9102|Supplier#000009102|s7Y8ojTFEnwKyW,4UAwI|16|26-180-438-1933|5060.47|s! even accounts sleep carefully after the furio +9103|Supplier#000009103|trVEAlzXJxquuvKGo0XNqEe9H7|14|24-862-623-8036|8361.05|counts nag carefully blithe, bold accounts. close deposits use after the regular, e +9104|Supplier#000009104|vSOV6vQGaZQoDYC,kfUE3,eDRjmcS,jnObSwHFt|10|20-960-874-8029|8306.56|egular asymptotes use blithely alongside o +9105|Supplier#000009105|hnymiJjaqsJ8Y8q4nUJAgykpZahHvG2KcyfO|12|22-718-791-4307|5478.59|l dependencies haggle. special, express waters wake across the silent patterns. requests sleep sly +9106|Supplier#000009106|AMxVmYjvWzBZKrrRurkk0q|10|20-396-280-4525|6753.51|inal ideas wake fluffily final packages. ideas wake busily against the quickly final id +9107|Supplier#000009107|DT9ZkOPMlednufUM5tJeuuhp9hDmXbhq qmLdFf|19|29-928-638-9549|6304.41|eans. ruthless accounts haggle blithely: +9108|Supplier#000009108|NXBnE30zQRKYuVCWczpKVSlGuHo9Y7|16|26-637-289-1092|4173.66| after the regular orbits. slyly ironic +9109|Supplier#000009109|TJWdOCmg1g2RtIU1SlTi3bxP9GhqOZLLmUJb|21|31-876-418-8871|8656.48|cajole slyly blithely final packages. blithely even excuses cajole carefully according +9110|Supplier#000009110|XF7zJMesO8DPCVXLFFK6OI|16|26-849-550-2740|1602.63|nusual ideas boost carefully! theodolites wake furiously. blithely pendi +9111|Supplier#000009111|FStUlew1jBP7OKdjZ4q3ZK,fQrEbJWHDFPsmKvD|18|28-967-390-2289|5317.64|tructions nag furiously above the slyly regular deposits. carefully final theodolites +9112|Supplier#000009112|KTSqjcziaRQckd8R,NOwD|0|10-993-409-7001|7593.88|uests along the quiet, reg +9113|Supplier#000009113|MRiD3IZCKabJ1Hwb71T9nCZGleNg8g1lLKE|11|21-267-707-3081|6385.96|usly. regular pinto beans haggle blithely. slyly express accoun +9114|Supplier#000009114|nkn6bcPvlP5w,lUpO0nZTBSj|21|31-952-889-7524|-106.92|thely express packages above the carefully special i +9115|Supplier#000009115|r,dAzSd7Y2ZA4NZjVekfqHGnaxJmjhkXfQMQ|23|33-473-888-1017|4981.87|en theodolites. regular deposits integrate inside the final asymptotes. unusual accounts cajole +9116|Supplier#000009116|ULjaQwNbcB XUG9dvbZDHvJVwLo08utswt|11|21-241-469-8343|7256.46|ending deposits. slyly ironic dependencies breach. blithely speci +9117|Supplier#000009117|FqQA4qeCJGh SZthzhNDKl3vFxY,A|24|34-479-107-1662|291.47|the blithely express courts haggle silently alongside of the quick packages +9118|Supplier#000009118|RMx38QA rm1H8ByjZtOcg3f4wXprc|0|10-397-743-5060|1695.32|bove the quickly regular foxes. even deposits affix slyly around the special courts. special, e +9119|Supplier#000009119|2qGs90gYaaA1Oq3YrvdNUjAV|22|32-290-856-6148|1147.96| asymptotes haggle fluffily. packages are b +9120|Supplier#000009120|Bynpx5CVPcMADbs5qigOBUWhVruh9XzXbcCso8l|12|22-408-932-7929|8736.81|rns. doggedly final theodolites nod slyly. slyly pending ideas boost: +9121|Supplier#000009121|RTS0DIWKkYEtASj9s7fFRiJFCHIjz4S3|24|34-140-751-5846|9692.71|lar notornis behind the fluffily express theodolites boost quickly instructions. packag +9122|Supplier#000009122|Qyb3Z05AcQyE1tVmk1sstk|9|19-755-451-9067|4475.70|. unusual dependencies should have to sleep furiously above the slyly even packages. foxes are slyly +9123|Supplier#000009123|69Yf8,gr1mZR5vgkN |2|12-391-909-4443|3069.08|otes are quickly final fo +9124|Supplier#000009124|b24qGaEjp95hrQkcsubqJ5WBW2F|9|19-914-792-1918|943.31|te along the regular accounts. furiously +9125|Supplier#000009125|IQbCXbN1mmght|21|31-864-673-9551|1152.60|s? pending pinto beans above the ideas nag across the ironic, unusual accounts. quickly +9126|Supplier#000009126|VUdvyOMwXOnKP3oNGYekoVLDAzZL4P YRpfglv|9|19-584-254-7605|9698.60|structions. unusual requests across the furiously ironic dep +9127|Supplier#000009127|he7i97yMBOz,Mpb8hznvD5zJxidp07KUFrT8mvv|11|21-656-396-6718|4372.43| pending asymptotes wake quickly regular instructions. slyly iron +9128|Supplier#000009128|LrpEZbk4mEfBc7yyH9YfSJ9Y32KH|2|12-394-187-2267|5466.74|c foxes. even foxes sleep slyly across the regular frets. pending requests use after the +9129|Supplier#000009129|QxVt5VE5QNlz,|4|14-541-805-1832|5500.82|nts. unusual theodolites are about the b +9130|Supplier#000009130|UFi48,GW2vNEq8Wv,hfvHy|15|25-761-149-1589|6288.48|after the quickly express requests. furiously regular asymptotes should use fin +9131|Supplier#000009131|gDBXgWtg4rTxu0WUJhhV|21|31-942-710-2719|446.31|ackages. ironic packages maintain even deposits. f +9132|Supplier#000009132|CiEFN7xj1txIqH1,MyqSv2|6|16-903-348-7643|8067.63|ilent, ironic accounts. ironic accounts sleep? ironic grouches sleep. slyly final foxes caj +9133|Supplier#000009133|u,3Wjz1dlWL4hfYBjRcPF4h9qDwqyS1n|17|27-647-753-5253|4655.80|fully ironic pinto beans are across the blithely enticing foxes. furiously unusual instructi +9134|Supplier#000009134|tVtS0RgtgB|11|21-752-802-2969|4986.95| braids doze slyly slyly ironic requests. silent deposits among the blith +9135|Supplier#000009135|HSEBHJosYq3cuEYIvGRlZbhoaulq|17|27-171-571-5733|4747.07| even deposits affix quickly alongside of +9136|Supplier#000009136|sBMP5q8xdBXJASC4t8mSpL56lVpzhNcNCDvWFGC|8|18-788-825-9180|-912.76|fully according to the slyly quiet ideas. platele +9137|Supplier#000009137|KVg2HpwID6gch6IGjAc9Ngfp723XR|20|30-207-738-3003|2161.02|usly regular theodolites. even pi +9138|Supplier#000009138|sz89MiHfAlYjPWCw3xBd7xsNYsI|2|12-582-136-2648|7209.56| requests. pending accounts along th +9139|Supplier#000009139|6,JXHimTm1advCvnSPOV4GO,,sCMqd|16|26-471-399-8038|4107.34|sly even instructions among the even requests should sleep among +9140|Supplier#000009140|PEo82Mna2Dpg Mf|22|32-112-568-9852|5001.37|ily pending foxes along the furiously special foxes are furiously furiously unusual foxes. package +9141|Supplier#000009141|SoThM1N5Kez,f|20|30-945-697-6173|4965.77|lar requests use carefully ironic packages. special instructions are furiously idle +9142|Supplier#000009142|axW5NhEYgOXhtHGCReYC97 71v|16|26-681-593-6020|669.37|usly thinly ironic accounts. carefully pending accounts u +9143|Supplier#000009143|PQbnctYsBsLZa,Qqnysw7HK4B5IC7D4bJbEllo|0|10-114-718-8156|6057.90|ggle fluffily ironic theodolites. car +9144|Supplier#000009144|3T7C1Rrd59Xd7bSaNqk7PGC53JCwTTs|2|12-876-893-4340|8074.96|. slyly final accounts cajole +9145|Supplier#000009145|1iNzrSUnEibVn4KGTFyADF5HOPhV3,|10|20-623-565-2517|6976.37|s are across the furiously pendin +9146|Supplier#000009146|lR2SxF36dZW6FMwBHXDg0exr|24|34-201-396-9834|807.00|osits about the carefully final accounts boost blithely special, +9147|Supplier#000009147|rOAuryHxpZ9eOvx|3|13-447-299-4722|9145.22|the blithely thin packages boost ironic deposits. slyly regular depos +9148|Supplier#000009148|5m5950, qkiXoBfGSXvF6h,EH,mxA6j1nr|1|11-849-244-8989|9734.96|escapades cajole above the blithely ironic ideas. +9149|Supplier#000009149|yKX,bKryD6YtvF,cVLIKC0Z6rN|21|31-872-724-9290|-513.04|cording to the regular, close foxes. unusual deposits kindle carefully along the r +9150|Supplier#000009150|FfA6,tpLupF,FSrb|14|24-827-297-3280|-753.91|ckly fluffily special requests. slyly unusual tithes according to +9151|Supplier#000009151| 25lLOrzuKU8mG0xGP2SLkfo6Kfew7Kb4YuRv |10|20-492-750-3318|9947.70|s. quickly unusual requests thrash furiou +9152|Supplier#000009152|ytxUUd,h2MfgH1ekEncuRYwl26Jc4w9DrD|3|13-586-160-9562|76.95|maintain final, final dolphins. furiously ironic deposits hinder. r +9153|Supplier#000009153|HO1p,hzFE5FdhA0ven|5|15-363-777-8969|4190.32|ironic theodolites. pending, special instructions sleep. slyly ironi +9154|Supplier#000009154|CARD8UKanwytz7 e|1|11-562-658-6307|2712.28|ts are carefully ironic accounts. furiously ironic deposits +9155|Supplier#000009155|JfDOolAPducAOG,oJ|17|27-826-780-5017|-696.97|dolphins. fluffily pending dolphins wake blithely special, special instructions. fina +9156|Supplier#000009156|yKu5huEicRbM7QlUAZaM1VdWURcX0J1kg1U,|7|17-842-117-8653|-985.37|lly express, special requests! regular, pending instructions are slyly +9157|Supplier#000009157|IbIHN9WrdVkE5wuaWwFXhTeH0bOlSvNbLYY0LzT|22|32-630-198-1339|2792.98|l foxes? quickly regular reques +9158|Supplier#000009158|it9tAeDAu9o1xG,G1JH|22|32-604-740-5833|2626.90|s the silent pinto beans haggle unusual, +9159|Supplier#000009159|aw,CBKFilgSVip|16|26-976-858-3615|5336.32|ly regular accounts detect slyly along the special, express instructions. special requests ea +9160|Supplier#000009160|RSRbbgm,RmnKF|0|10-282-787-1261|2463.36| to the carefully regular theodolites. blithely bold +9161|Supplier#000009161|H0uLngg5pUdc|14|24-547-951-1269|6600.82|courts about the quickly final +9162|Supplier#000009162|mQTk6NFXonat5 ocsFkHYFflBe1m6leDm0Q|15|25-111-673-9641|3028.11|unusual accounts. pending pinto beans cajole quickly according t +9163|Supplier#000009163|E0XNTVX9I8VMnga68MkhBX0M jC|3|13-503-944-6248|5361.59|ess packages boost around the slyly regular pinto beans. pe +9164|Supplier#000009164|rYR92AZU,F|20|30-533-868-3983|9224.05|cing theodolites. quietly pending +9165|Supplier#000009165|Fyt8EohQ5IReADmKJp9WHJ,3yqCXk7iAcVdVKe|24|34-681-588-3121|3739.01|after the carefully even deposits. slow, ironic packages wake furiously unusual orb +9166|Supplier#000009166|,TE,42HKhD gj2c2LufLBUkkMK8uX|23|33-601-592-2411|7547.31|counts wake blithely. even theodolites boost slyly. theodolites wake fluffily blithely +9167|Supplier#000009167|5KQWpFTqsMsXhouNkjgVHa|17|27-436-673-9892|4603.12|ing the unusual, regular accounts use above +9168|Supplier#000009168|Lv1a0iORe8g6hpQ0Z,oZqGL6nExjpL,|17|27-747-919-5811|-311.76|lites use. carefully pending accounts cajole slyly along the blithely bold requests. quickly iro +9169|Supplier#000009169|LxflkGFBwmNEca|19|29-475-686-3568|5256.74| pinto beans are quickly above the fluffily bu +9170|Supplier#000009170|HlWv0fgTn0CgIQZ|23|33-765-165-8168|6991.05|en packages sleep at the slyly final deposits. furiously expr +9171|Supplier#000009171|72JDVZIEgr7BzWM7oF0ftJtTRZJ|12|22-210-429-3496|5637.88|inal dependencies affix blithely. furious +9172|Supplier#000009172|ab3TaPZy95Fdp1e|16|26-195-558-4363|-212.56|s. blithely unusual grouches sleep blithely quickly regular theodolites. furiously silent account +9173|Supplier#000009173|vQCb4bAca89UOII,4ympOE7fHyCLyRl|16|26-637-798-3487|426.69|egular requests shall wake blithely carefully special requests. i +9174|Supplier#000009174|5ttJzKeXTYkk1hP8TXLxzWO2Z1toj|13|23-197-965-7184|8296.20| ironic theodolites haggle quickly regular excuses. quickly special excuses integrate furiously f +9175|Supplier#000009175|JYoZYAOQdbh|8|18-483-310-3053|7306.86| silent foxes. furiously special theodolites was. idle, final courts dazzle across the sly r +9176|Supplier#000009176|duvdRRjP3hCE5u1v0Lt|4|14-362-553-8088|467.27|eposits. deposits cajole! instructions wake carefully furiously final deposits. +9177|Supplier#000009177|hoQLFvEPhZn8|8|18-753-874-1164|8307.35|ges use. furiously even excuses use slyly quickly bold platelets. unusual, regular requests haggle +9178|Supplier#000009178|zafaXayok 0PJR kPo,wALge|23|33-556-558-5902|7737.67|ngside of the even ideas. unusual, pending instructions haggle carefully about the fina +9179|Supplier#000009179|vQLUCaGF7NkKDnWHzwbX,KWiN|20|30-747-814-7106|3023.45|ar hockey players boost furiously final, ironic pinto beans. unusual acc +9180|Supplier#000009180|QSsgXlP6e4STjwk,mQ|23|33-823-233-6222|4083.43|ests sleep fluffily about the blithely sl +9181|Supplier#000009181|EhR UPoiEWtO|7|17-134-209-7773|-870.74|final deposits are among the slyly pending packages. carefully even accounts sleep slyly furiously +9182|Supplier#000009182|z56kNgeqaWQ1kHFBp|21|31-640-427-5006|4402.82|according to the quickly regular packages +9183|Supplier#000009183|zGVAmov0l,gmKOA1eWmRQktpZ1C8 fgkXgq|12|22-770-139-6261|-816.02|ts wake blithely regular packages. furiously even asymptotes hinder careful +9184|Supplier#000009184|u7qqtlgu02DWf9aBjKR2qvZ0|11|21-416-185-1079|142.38|eodolites affix against t +9185|Supplier#000009185|v66BUMEgc3P24E8PLV nMkC3w2bWb|9|19-630-483-6872|3969.42|ar pinto beans cajole even, +9186|Supplier#000009186|wgfosrVPexl9pEXWywaqlBMDYYf|6|16-668-570-1402|8109.09|tions haggle slyly about the sil +9187|Supplier#000009187|eIOnssJBbLCqHBSzB,piOjydDZ1VHVWQqW|9|19-245-931-7249|8601.51| above the blithely regular pinto +9188|Supplier#000009188|BbyEfF1ff E3WNsMWSN0c,IZKt7Hu,k|13|23-730-237-5786|1560.46|instructions detect furiously ironic requests. carefully pending accounts after t +9189|Supplier#000009189|fuTTLRi9REaNL2aqM1prskiw9hlOFYirjeOtR|10|20-497-311-1008|580.87|ithely even forges cajole. furiously bold ideas cajole sometimes alongside o +9190|Supplier#000009190|,PPriCG2e9pu8y62ZwvIgp1bKfmdrSgD7ea|20|30-415-586-1303|3556.45| of the blithely even accounts. quickly final requests cajol +9191|Supplier#000009191|WsRGPLN DCZA8mgJnCF FQ29OWam8c|13|23-389-796-8960|8174.65|. slyly special pinto beans boost slyly. carefully ironic deposit +9192|Supplier#000009192|uejb5jHTAqR SajE1E8bf6Fdawr7F8FW25HK|4|14-578-864-7183|-609.18|wly express foxes wake blithely. slyly regular dep +9193|Supplier#000009193|ZAoRBc9qPjGjxYCUfPQcNoSG8,0buSV|17|27-506-209-6288|438.16|jole against the blithely express instructions. regular asymptotes against th +9194|Supplier#000009194|dTnHwpqAN rzP|11|21-151-387-3466|3106.35|the carefully regular packages. regular theodolit +9195|Supplier#000009195|sWocDf5wBdsWVNUmEehGAy8tUQhJIh2j8kZMJ|19|29-492-828-2973|388.66|ironic gifts detect. unusual excuses wake blithely; furiously ironic packages hang quickly re +9196|Supplier#000009196|rzIRgKBKPfPtz1 06C8kIclix|6|16-611-625-6484|7227.46|ng the furiously silent package +9197|Supplier#000009197|kTmXwb9rsaQU3sJiE65|5|15-441-453-4938|7987.36|lar requests haggle slowly about the pinto beans. regular deposits haggle blithely blithe +9198|Supplier#000009198|KxMOBZ9hjBM0eZdWpCheXJ63P3RftrS2UG|15|25-533-724-2058|8686.16|tructions haggle carefully over the slyly ironic pinto be +9199|Supplier#000009199|jU0Xnpq5fhjWjmgQoS|9|19-885-935-6951|5436.34|lithely careful requests! blithely ironic accounts nag slyly quickly final p +9200|Supplier#000009200|9AN1R2aS9g4MEOduFhtStnA|20|30-967-844-5044|507.84|ven deposits sleep atop the pending ideas. quickly final accounts among the even, ironic p +9201|Supplier#000009201|2bhZzJ0CfQZpIdm4jYeFvfm|17|27-762-798-5463|8176.03|across the slyly ironic packages. silent, even deposits haggle +9202|Supplier#000009202|x,pKbV7Qie4CGXNGLuwt|12|22-253-185-3464|523.05|ess requests along the final +9203|Supplier#000009203|0k 3o4VZPgncZO9RXthKBBk|5|15-184-963-9482|-215.26|e. furiously bold deposit +9204|Supplier#000009204|P6chenod24|16|26-155-683-3181|-866.15|ackages. packages boost quickly blithely exp +9205|Supplier#000009205|aOda ,dPIR352HUA6UlqjhI6GqBmqJMh, ahIgAS|5|15-546-132-6261|4493.28|fully silent dependencies cajole furiously. furiously special packages wake quick +9206|Supplier#000009206|2VR3kN5Z71b9s2kxdGHcSGD7Dy7ldIT7F |14|24-288-618-1976|1630.21|final packages. quickly silent packag +9207|Supplier#000009207|RmMptXS LqStqzaTDlWpJTtKIifnh|22|32-647-700-7127|6837.29|. foxes are. carefully unusual decoys are slyly along the courts: carefully bold asymptot +9208|Supplier#000009208|B0ae7o3uBmpW4ugK07waA5eh|1|11-634-874-6154|6423.33|y quickly regular packages. silent, u +9209|Supplier#000009209|OxSf,fja,vu60F3vPw|23|33-610-434-5069|944.96|tect among the blithely speci +9210|Supplier#000009210|sa U,SZL7BsDag12dYnXvYB9Ss2qm|24|34-584-768-4698|6950.04|e blithely idle instructions wake request +9211|Supplier#000009211|Jq9wxuC691|21|31-248-228-7667|9405.31|blithely unusual requests. ironic requests detect above the fluffily regula +9212|Supplier#000009212|zmYL6C8D,yYVE4|24|34-424-428-7412|3176.44|packages are. blithely pending instructions integ +9213|Supplier#000009213|Rk1RNWl8m8N6ODGp7S4C73QAIdV0eEUU|22|32-642-267-3594|1328.50|ckages. stealthily ruthless deposits alongside of the furiously special requests cajole carefu +9214|Supplier#000009214|,qaMSx5PCf Rgvw,zA2NMocm5fLSe7oWn|24|34-435-736-1379|7126.11|ckages. fluffily express accounts boost slyly along the carefully b +9215|Supplier#000009215|g5zNuDDRqHVPz106DoCpVL7PQtVWya1DMzsBi0|8|18-151-724-1447|9519.78|p fluffily unusual packages. furiously even requests among th +9216|Supplier#000009216|Eylm6q3TNja65ulRkiceJauwXTJgiOLm3D0Xh|5|15-417-241-5494|2008.75|nic packages boost furiously. slyly special asymptotes are slyly +9217|Supplier#000009217|0eisHARnPSuqWzqbHw7hysJpbxQ2|20|30-822-179-5158|45.34| final deposits. pending deposits cajole furiously above the +9218|Supplier#000009218|9UwSwDFXuiHdqdY6o0vcZAQry805nXrIzy|16|26-223-206-7687|892.57|into beans wake blithely. thinly bold pa +9219|Supplier#000009219|5,yTuPNvGjunlF3c82hykaO5C6noZzSG|6|16-462-513-1926|6304.58|according to the blithely regular ideas. carefully iro +9220|Supplier#000009220|N4y,vP kdArpcmdypBh,fJVVB|21|31-599-538-2318|7153.83|ckages sleep after the slyly ironic epit +9221|Supplier#000009221|6PHk0Jdw4XGvLC2inbLmnzdT5uNXJHMXDtt|24|34-986-690-3784|2536.22|, even requests. silent ideas wake after the quickly even asymptotes? special, f +9222|Supplier#000009222|v9IhjRciCeigCsJZ2 8sV|18|28-676-230-6030|-713.55| the slyly ironic dinos. slyly bold pearls wake after the pending acco +9223|Supplier#000009223|3mDusY PPK0Jjr|1|11-611-493-4880|6325.42|deposits. slyly special requests wake past the silent dependencies: never final ideas eat a +9224|Supplier#000009224|UdYNnhx9F6rZVEB|20|30-160-981-2569|5431.91|g the quickly regular deposits affix carefully special packages. requests nag fluffily special foxes +9225|Supplier#000009225|s,MIHdC7zF|22|32-542-446-4004|3197.65|he carefully regular packages. fluffily pending deposits integrate. carefully pendin +9226|Supplier#000009226|yzT10vNTFJ|21|31-898-845-5686|5871.35| special accounts. quickly r +9227|Supplier#000009227|dvYajYNbpz|3|13-271-712-5917|7840.81|haggle carefully above the blithely bold courts. fluffily quick accoun +9228|Supplier#000009228|jtReDQS6,jIxGAxKJs9|4|14-690-296-1478|9496.94|egular foxes. carefully ironic ideas cajole through the slyly iro +9229|Supplier#000009229|A 0alml2sSRNNDzJ85OII2P91|1|11-538-888-5944|3195.24|arefully. carefully idle theodolites cajole qu +9230|Supplier#000009230|yxMHRmhq1DWOYpaIpaSDF5xcJML|18|28-265-912-4981|9300.89|nto beans after the final, final requests kindle stealthily against the quickly regular instruct +9231|Supplier#000009231|T37AhB47xDHz48fggP gaQjmPUUEMeL2wO|8|18-767-139-4896|8150.03|le. pending, quick accounts serve after the fur +9232|Supplier#000009232|oKg,qpT55rwxqbmq1gyf7HttG9|22|32-770-768-2763|3538.07|ructions cajole blithely. unusual, regular ac +9233|Supplier#000009233|mNdPed7sgNr7A6WqU,XXBkXKRcP3aD67WKE1b|5|15-374-186-6259|7302.96|tions cajole slyly idle accounts. pinto beans nod slyly furiously +9234|Supplier#000009234|K09FNZnt oU9|22|32-793-931-4699|971.33|slyly blithely silent instructions. fluffily quiet instructions sleep quickly deposits-- sly +9235|Supplier#000009235|wRqFBemrMdvfB4KbOGmwv988A|16|26-909-993-1152|-983.40| dolphins sleep slyly bli +9236|Supplier#000009236|GqhunMWKKexoPktGSe86v6XTrxE7K5i|13|23-889-411-2587|2903.82|refully even requests engage ideas. f +9237|Supplier#000009237|5xxaf 0NCk|12|22-942-504-3189|6521.83|unts haggle regular account +9238|Supplier#000009238|uRuktQQJMIWbU3v5HBuSh0,16sUs8OnaI|18|28-830-306-5949|6394.00|ss waters! thin theodolites above the pending, final dependencies run carefully final sentimen +9239|Supplier#000009239|qcH0lm FlDTsIthnjRa|24|34-383-332-8531|3027.53|ven accounts. foxes try to wake furiously +9240|Supplier#000009240|xNRyBB09ybecSGN|14|24-518-252-9537|6698.49|e slyly bold requests. permanently special dependencies cajole bold, express packages! ironic, unusu +9241|Supplier#000009241|8nrvAcXiaw0NVOKvUthGohy2T0yZQx|22|32-250-945-2444|5621.64|. carefully even requests wake. blithely bold pinto +9242|Supplier#000009242|RNHDzJ8rblcgd3G2hjdnygsZAlrTJL|3|13-421-983-7033|1937.38|s? ironic accounts boost according to th +9243|Supplier#000009243|4KCGoNNRfCpNeZPkEgt2X4fZ|7|17-433-311-8815|-432.17|blithely regular foxes thrash blithely. u +9244|Supplier#000009244|JwuJR1YxeCvrrRoApdtY2WQE1kavfjoJb08s|1|11-205-683-4279|5419.37|atelets. slyly express deposi +9245|Supplier#000009245|cZnE5j68mPoc|15|25-904-400-1018|1284.23|ickly after the even, special decoys. quickl +9246|Supplier#000009246|p9twzv6a6w9NE6MDSG6aPVqFxaH|10|20-147-712-9204|21.82|slyly regular accounts cajole. packages +9247|Supplier#000009247|IfJfNdmtUZGIM9zJTdusaKDg3cGOU5QxeM|21|31-934-273-7276|1586.97|fully regular dependencies. final, final ideas wa +9248|Supplier#000009248|WAkWwdEMsVz9WLqER4YAggB5aaGO04Jg62Li,fV|0|10-392-858-2023|6234.33|oze slyly about the sly, express packages. regular deposits a +9249|Supplier#000009249|gijfGaYsh,C|22|32-910-117-8243|3423.90|olites engage express foxes. bold platelets sleep. q +9250|Supplier#000009250|9Zdg4CKQ4,AACYfH|0|10-185-698-6987|2547.64|ic packages. blithely regula +9251|Supplier#000009251|2e20CXWM6VbVLza4MV0RYXE8Goc51Js|17|27-582-604-7096|4212.27|eposits after the carefully r +9252|Supplier#000009252|F7cZaPUHwh1 ZKyj3xmAVWC1XdP ue1p5m,i|3|13-980-495-4398|8689.46|ependencies nag blithely fluffily final pinto beans. +9253|Supplier#000009253|O hOjqNkcZR2,BmHq0Rns6vCCKIgoNIQpd1CW|16|26-456-424-4340|5835.55|al platelets. blithely even instructions may grow furiousl +9254|Supplier#000009254|kAcH5hHmspYh,aA|19|29-780-238-7086|81.96|r theodolites above the slyly regular accoun +9255|Supplier#000009255|QioTMm,SOc6ZnPagEK2|21|31-359-216-4205|384.17|al asymptotes nag slyly. ironic accounts are bold deposits. regular accounts promi +9256|Supplier#000009256|PxFBMa1lFwPDxQbo|16|26-512-741-5623|8617.31|ar instructions should have to detect carefully across the ironic, bold deposits. eve +9257|Supplier#000009257|l7,JPH4t0v8WTiLFhJyLUcuB|17|27-363-789-5841|3256.08|ully final excuses. final, express requests nag +9258|Supplier#000009258|vj 7yiMQeMGfwaLJ4OpmIRq|7|17-688-248-8351|8959.92|und the deposits grow blithely slyly bold pinto beans. quickly ironic packages about the final +9259|Supplier#000009259|L1,t6WFwiHe0A1MxVpslwlipmaMau|5|15-411-992-6327|8361.63|s mold. slyly pending dolphins use furiously accordi +9260|Supplier#000009260|JRyiZezd,S6xr,TiwfjpA7AcchrCCWo4Eyw6VVe|23|33-206-487-3186|6931.09|deas. carefully regular foxes detect above +9261|Supplier#000009261|bnZkfzyO4dSHOVDSH6vGsQIoQ,NEHIRobkEou|23|33-136-928-7087|9862.79|onic foxes. carefully final excuses use slyly about the carefully regular +9262|Supplier#000009262|MsbjKCAJzlpajvpfscxFMS7aOxa5pCDIbr|16|26-486-646-7733|2932.12|ffily blithe deposits wake blithe +9263|Supplier#000009263|y5cT48CXI,0y48W|8|18-809-566-9898|9056.04|y. carefully silent instructions nag blithely special requests. stealthy, special ideas around the +9264|Supplier#000009264|d7GINn8JkTi0lu9O3UtW|11|21-835-998-8764|6880.47|ong the special requests are regular instructions! fluff +9265|Supplier#000009265|El1z,MYnP5ckp|0|10-318-994-4236|6029.63|ly bravely pending requests? carefull +9266|Supplier#000009266|2LDu6eCrvn8Skc,WgmVYoJOHAo|9|19-303-584-6011|6001.68|he slyly even deposits. ideas about the even, express +9267|Supplier#000009267|00xBspS5 6|15|25-913-150-2428|9276.39|ld ideas sleep ironically. carefully even accounts h +9268|Supplier#000009268|E8CdtkXawxDEzp7hj6x8fwEYGKmduaQyb53lfAq5|20|30-303-354-2256|3424.74|fully bold dependencies doubt. furiously ex +9269|Supplier#000009269|iQk14CbL5TzuTKJEirowP8UBJ1HTBCh5HLkFqp8|4|14-975-151-9600|5582.90|eposits. slyly final foxes detect regular theodolites. furiousl +9270|Supplier#000009270|OVPczHDUsYBQGzLbG4dG3G 6Re9y8TqGyZ8F4|0|10-282-239-9138|-126.11|uriously express patterns. quickly special asymptotes boost always across the quickly +9271|Supplier#000009271|t,UKTw64BHpi4,,VvSZaAbqw3sAkWOj3OqvKKAkR|16|26-899-804-6055|1381.92|ependencies wake carefully accordi +9272|Supplier#000009272|VFcTkTcm7NEEenCtro85Bwd7syhEHEZ9Va|6|16-648-900-1004|2876.63|thely bold excuses integrate always even dependencies. carefully express +9273|Supplier#000009273|TZ4k9XOpDS6YjijWmfRI|4|14-656-188-3805|551.82|lly even asymptotes. furiously e +9274|Supplier#000009274|Zjra556naQxMVROVPu4riskWQ0zcQHHIVM543L|23|33-559-738-4113|-538.22|packages use furiously furiously ironi +9275|Supplier#000009275|9ROhWwDc62wHtvElpgK5hyEVX3oAMd|24|34-345-316-6362|5138.91|nts boost slyly among the fluffily express re +9276|Supplier#000009276|iXojoy9YVbTkggCnbpKtKZPX8IS4|22|32-956-171-4392|6718.78| ironically final packages. carefully regular instructions +9277|Supplier#000009277|H1SGoSQo,AS5AkEHXyorIWI1|9|19-237-233-4477|5583.40|ve the final requests nag fluffily above the carefully bold instructions. idle, +9278|Supplier#000009278|RqYTzgxj93CLX 0mcYfCENOefD|3|13-484-537-8858|8172.71|theodolites cajole furiously quickly ironic instructions. blithely ironic +9279|Supplier#000009279|dw IolA3dCDSXClsiNxJej7tF|18|28-919-517-5641|-288.28|es kindle carefully. carefully ironi +9280|Supplier#000009280|r9xAmwXKzmcvQ369edjp6eWP2acLGos5|4|14-231-827-1063|3825.31|wake. final theodolites sleep blithely. furiously final dependencies sleep furiously. final, pen +9281|Supplier#000009281|3zaUojlR0cW5hXNbJEHL1Dp0u lo9W,m6msg|6|16-427-709-3910|6016.26|iments are quickly according to the regular, silent ideas. fluffi +9282|Supplier#000009282|NlGTGdk YL53Qb|14|24-255-571-4829|1259.02|y close requests maintain carefully. quickly final requests against the quickly unusual deposits det +9283|Supplier#000009283|Rmocl6ClgBzwarceqA MDM5sZcXFL2oNvsW,|8|18-642-906-6749|5234.44|deposits detect? permanent +9284|Supplier#000009284|JF5cmh0k 0WFd10|18|28-679-135-7971|6277.34|y ironic foxes wake. even, furious requests along the even accounts wake about the car +9285|Supplier#000009285|jGb4qMXlm0Zx4uGGROSAjtnizGIdeH|5|15-301-927-8192|326.63|aggle around the furiously even requ +9286|Supplier#000009286|0Rsc3gYBFVPEusbTFPQ|18|28-467-426-2914|9664.67|ly. furiously unusual dolphins nag slyly +9287|Supplier#000009287|GyjYxh9OmFZy5h9cku7BeB6FPZbsFs,V7gd|7|17-951-355-9834|6213.71|ndencies cajole slyly furiously regu +9288|Supplier#000009288| 251AA4ziZ3d7TTWXLGnXjb4BnXv|21|31-284-849-1216|8772.71|ns against the carefully ironic theodolites detect ironic excuses. slyly express +9289|Supplier#000009289|lAIhcH3 p,ij2ci,YjLmNmUbEnNa3OtrJ55PzE|24|34-212-362-3318|7996.46|y special deposits. slyly unusual deposits among the slyly final asymptotes cajole b +9290|Supplier#000009290|H6t,JDB4Pp|16|26-359-813-2953|7490.17|pending pinto beans might nag slyly. pearls detect. furiously regul +9291|Supplier#000009291|mjhaYBLJ5kMInVHISNEkwM0JGBt4d0|2|12-415-763-6304|1359.57|onic accounts haggle silently among the final, express accounts. even theod +9292|Supplier#000009292|Ob,MQ8EN7qSya4P218IMVm6k1slAh5B|4|14-113-387-5773|1230.91|quickly unusual deposits. furiously bold packages about the ironic requests are daringly final f +9293|Supplier#000009293|BZME0dGF3sUlPW6nOsJ6x|16|26-605-455-9355|5931.53|e. quickly special accounts across the unusual idea +9294|Supplier#000009294|o6G5Zq1lU3zny,|2|12-785-941-4731|7323.62|usly even requests wake besides the slyly silent accounts. quietly special dolphins detect slyly u +9295|Supplier#000009295|pzm3xLVBnSgZbIoR75z2yPamaFuCO3O|10|20-600-737-5494|7116.32|he fluffily final requests. slyly ironic dependencies wake. carefully ironic +9296|Supplier#000009296|5USIOzOGLjAhH,l5N|21|31-947-656-9051|-923.94|ct carefully across the regular, exp +9297|Supplier#000009297|l8xTeKqe4sDvbBpiKqkUhiRUw JQu|14|24-260-913-4618|1299.33|ns cajole after the final, special platelets. pending +9298|Supplier#000009298|f89AhVWf0MAtXoTXfWIuihlyJ9FH,Ubl|3|13-504-962-1709|840.46|eans cajole slyly furiously ironic excuses. carefully close accounts after the regular, final instru +9299|Supplier#000009299|UKUpKpXccW5X4r1n1UIIxpjpJH|14|24-957-349-7082|-519.74|press packages are quickly after the furiously silent accounts. unus +9300|Supplier#000009300|Br0nFMGdN,8duCrXx5IqL3g7RGq6aF6y|5|15-400-775-9386|7626.76| warhorses cajole above the regular asymptotes. carefully silent pinto beans sleep agai +9301|Supplier#000009301|cZbZX7WSUJU41dwwjFlVUHBoeY6lrv3S|17|27-159-994-1867|9086.14| ideas are against the theodolites +9302|Supplier#000009302|Plz6TcSNF0pJTiBdKrJEWd6Rw31MquEN6Z|2|12-885-138-1631|6770.81|indle quickly bold dolphins. brave, regular deposits eat +9303|Supplier#000009303|XMkX2032QKygvHfkFF4Hk1dvYKyhk|16|26-106-907-2281|3.43|manently alongside of the requests. furiously special accounts haggle according to th +9304|Supplier#000009304|1, XZttwkmbllozHYC9bp1StcxycsQ9K|2|12-427-913-3581|611.38|press dependencies. even, express theodolites sleep quickly. slyly speci +9305|Supplier#000009305|UJMy,wReN55m8qaO9q5k,l1no52Ox5N|4|14-608-211-3276|-612.37| around the fluffily pending requests. foxes wake enticingly along the furiously t +9306|Supplier#000009306|lAjkRXpKye7R2f5qHbPMs,SFZmf|4|14-650-412-2315|1830.48|slow ideas. carefully express requests use furiously. carefully express ideas against the +9307|Supplier#000009307| AXTjtRFJfmjZRDD37zXjR8iDW6FhOSl,8yYt|7|17-529-917-1541|6847.31|es x-ray carefully around the regular packages. f +9308|Supplier#000009308|yV4zFJaBvx9P5wLH7,|18|28-854-703-2869|9794.83|l packages boost always slyly +9309|Supplier#000009309|ImYmBxbianlOcHdBDUl|6|16-560-442-4703|4734.89| detect according to the furiousl +9310|Supplier#000009310|mgxAqmM70OLIJT ltIOIb|22|32-954-202-4231|3264.34|e the deposits. quickly stealthy instructions hinder furiously whithout the +9311|Supplier#000009311|UinYNHovLgBB82J0tcxN3chdG|7|17-142-643-3942|4807.43|oxes doubt against the regular accounts. fluffily final ideas sleep along th +9312|Supplier#000009312|xYgXzgD7aCvbopimsJs62shDxWinWmpHBy,0lh|1|11-780-379-3162|1551.09| the regular forges. furiously ironic instructions doubt blithely ca +9313|Supplier#000009313|ulZsqYDPQJG4HiT6AXOOnJJEeu|24|34-272-359-1149|8129.28|s. blithely idle dependencies sleep. special foxes detect. slyly pen +9314|Supplier#000009314|XD3HVomMK7W1C8i bB1joW|9|19-869-893-3378|6890.79| furiously fluffy requests doubt furiously above the slyly special excuses. furiously +9315|Supplier#000009315|bCKGRBBn p|6|16-538-336-9026|3565.07|haggle fluffily ironic escapades. carefully ironic ideas would cajole carefully ironic pinto +9316|Supplier#000009316|O681SFDLS8wX25Gi9r8VD6wgVCBGu3jc cJ4IoP|9|19-366-587-7535|-865.78|equests at the quickly final deposits wake blithely ironic, unu +9317|Supplier#000009317|WkcONAO7Fx0TRCXNY|18|28-776-771-6693|-194.85|ly even asymptotes cajole blithely around the requests. ruthle +9318|Supplier#000009318|NLlnTw7WjUNmJ9v5xvHNMcC82El5JzG|2|12-577-853-3354|1146.35|refully even dolphins. fluff +9319|Supplier#000009319|i,ea3Rf8BPxmT,8h3vC,A i|10|20-752-395-3485|3706.78|riously final platelets haggle slyly even deposits. stealthily regular requests thrash +9320|Supplier#000009320|RiFhOw5TUyPqGScO cGMp|8|18-406-489-4500|4792.28|e quickly beyond the carefull +9321|Supplier#000009321|lwznHe,WHtbKlY3yRknx oU PxZwp5Ag|13|23-436-335-2124|-204.04| platelets boost fluffily pending packages. attainmen +9322|Supplier#000009322|F8,4Vt1VvOJ9hZ,,efaU|22|32-380-547-7957|687.35|requests haggle. sometimes unusual packages dazzle carefull +9323|Supplier#000009323|xQzxixcS9G30QYFoAsEiB ph2eQkiCyUwsYc7y|12|22-230-852-1667|276.72| final, regular foxes wake slyly express, bold pinto beans. ironic pinto beans wake slyly +9324|Supplier#000009324|dc17deZC9JCB1wxdd15xce|4|14-135-475-9911|6745.35|ding to the express, iron +9325|Supplier#000009325|kk,8WW2IGQvWn2DKo5q7zFwXiBI1BgVPFb|1|11-412-424-5957|1048.96|s along the final, final dependencies will have to cajole outsi +9326|Supplier#000009326|XmiC,uy36B9,fb0zhcjaagiXQutg|3|13-492-503-8495|5795.68|cial excuses. slyly unusual theodolites haggle carefully among the furiously pen +9327|Supplier#000009327|uoqMdf7e7Gj9dbQ53|3|13-168-561-4489|959.76|dolphins wake carefully blithely regular multipliers. bold packages +9328|Supplier#000009328| DlA43cdkE,QjjUFA50ZdMxW8EoXw8a|10|20-619-531-7692|2560.41|uickly ironic accounts alongside of th +9329|Supplier#000009329|lBFsuZfRRUTipwNtkG|2|12-966-117-5624|9794.42|y even excuses cajole final accounts. furiously regular theodolites haggle. regular requests are ca +9330|Supplier#000009330|VwgNP25KeVp4iIO|0|10-527-259-3339|8850.26|y according to the carefully pending ideas. slyly bold pinto beans sleep carefully unusua +9331|Supplier#000009331|lyv5hzoJsUU1LoBcG,HINRA53w7joMx6m4 |14|24-808-822-6597|2418.25|gle carefully. furiously regular ideas boost carefully among the slyly special pinto beans. slyly +9332|Supplier#000009332|CI0Y ,x66iW53t0i34nc22TDX2hrii93iR vQOY|21|31-956-710-5738|7473.66|n theodolites run slyly alongside of t +9333|Supplier#000009333|e9JTDP8Pt7IrPBBlimjs ,|5|15-289-312-9549|9789.53|e furiously quickly special packages. fluffily final pearls are about the +9334|Supplier#000009334|6HCNzvuEk2nw|11|21-216-421-2698|3773.55|y regular decoys. boldly special pinto beans nag furio +9335|Supplier#000009335|kuNDm2ZNuM9uY|6|16-545-236-3683|5063.36|nusual foxes use carefully above the carefully ironic deposits. slyly +9336|Supplier#000009336|E fTygfu1wRIOGyM3bquYO8eGgF0HdfAjxwh2k|8|18-577-510-3914|1827.53|mptotes integrate slyly accounts. slyly special packages nag carefully. express i +9337|Supplier#000009337|W1ELyZLRBjJePS0ed1Q1,nPfa8O430oO9xC|20|30-789-515-6554|1208.73| deposits boost carefully slyly even f +9338|Supplier#000009338|m3ElPvJHfvIbyFjbTGR6b|7|17-985-253-5364|7135.82|ronic accounts cajole carefully +9339|Supplier#000009339|rdjtJb6CvgXd2ktq|23|33-515-259-7393|7846.57|pendencies haggle furiously against the bold requ +9340|Supplier#000009340|RyK57auA8QwVfDxdGaGgyZj0T1,7ZpBhPy9syRdm|14|24-330-317-8910|7434.27|es. courts detect. furiously silent theodolites engage carefully fluffily even courts. fluffily f +9341|Supplier#000009341|l5tM XlxcSwI78UqwYw3M|21|31-368-204-4869|7792.26|ourts sleep carefully. final +9342|Supplier#000009342|y99pwTZ MFBTVB1S,vMjvBS9HTFoDj|4|14-785-529-3630|7509.94|arefully after the closely bold excuses. quickly ironic deposits for the soma +9343|Supplier#000009343|2dYKVtYDM7ln|11|21-713-185-5048|3638.88|uses boost blithely. slyly regular foxes boost furiously. +9344|Supplier#000009344|N1A48rWNnF63i,ePPBscDmOnHPRSJu|5|15-205-553-9893|7680.21|e along the slyly regular de +9345|Supplier#000009345|v1Cjg3iQHdEFksTcy7Sl zvrYnVMWZRGkL4|4|14-810-955-9497|5376.28|r, unusual accounts use quickly slyly even deposits. specia +9346|Supplier#000009346|F2,WuMvhDqdLGa6ZGWCvn mGK|6|16-674-943-9382|3070.27|ress theodolites are. furiously pending pinto b +9347|Supplier#000009347|ZtUTsLXGWxAzti|15|25-779-427-4737|9932.21|c packages nag around the furiously regular packages. ironic reque +9348|Supplier#000009348|uOf8xE8iJXh0rsu|5|15-764-224-7770|3714.25|ously final, special decoys. +9349|Supplier#000009349|oJIlkL3FqiKkYqmmIeu18HzDE0WnPpoG3R8bS|11|21-197-322-4895|9449.33|al deposits. permanent requests boost according to the special, regular ide +9350|Supplier#000009350|R5zcYayjjFA3J|1|11-864-789-8663|5907.27|ts. theodolites haggle qu +9351|Supplier#000009351|v5wkgMFXQS8p2fTYOj|19|29-982-545-5258|6498.38|d the pending pinto beans. carefully ironic frets sleep along the slyly special ideas. +9352|Supplier#000009352|WwG1,Fwgai nJQ|22|32-629-439-3619|8334.82|ronic dependencies. regular, express packages across the blithely ir +9353|Supplier#000009353|YWMs0OdQY51lTxLTpmfhdxz4mC3|4|14-121-714-6005|7402.34| the blithely silent accounts nag +9354|Supplier#000009354|vHiOY5UtNDswS7tde 1YQMdfB9tlGAvLmfni9l|16|26-252-575-5706|-839.03|ly slyly bold packages. regul +9355|Supplier#000009355|ATw43gXAnm3QIHdlX1S|17|27-371-891-4773|9335.77|gside of the ironic gifts. regular pinto beans after the p +9356|Supplier#000009356|mp4A S oA2eBOo|7|17-494-308-9212|307.91| express theodolites cajole carefully furi +9357|Supplier#000009357|062gQFgOl5Y9n8wvIOTO3fIjbKL8c iaOgKE|19|29-207-441-1355|2933.96|refully express, regular instructions. unusual, regular requests are blith +9358|Supplier#000009358|Ds5FfdEuXPXVjjRGeq|0|10-374-899-2706|7054.49|ntly even decoys. thin, express accounts grow across the courts. ironic deposits nag s +9359|Supplier#000009359|3C6cbvoRrnGxmBi46zAlrIbzS8|3|13-576-481-4137|4805.61|y regular ideas. final, pending accounts +9360|Supplier#000009360|1NVjjX8zMjyBX2UapDTP0Sz|21|31-476-643-5790|3921.62|l requests poach always e +9361|Supplier#000009361|GlnC 2nWxt|17|27-525-357-7776|7684.70| express, ironic packages. quickly final theodolites nag furiously above the special, unusua +9362|Supplier#000009362|ReNJhH k9lB3GU2PNVpSLwOu|14|24-536-333-5374|3767.32|ully. ideas breach furiously. blithely ir +9363|Supplier#000009363|KXKoZ2kPnlM1|6|16-570-225-1500|-598.82|carefully even accounts nag fluffily. re +9364|Supplier#000009364|SlkOn3cSyA MWaBFHhn1O5yD|10|20-416-330-7740|9487.46|pendencies haggle slyly even +9365|Supplier#000009365|Ga98U7khv9saRWNZwHc|2|12-417-761-5258|8878.51|riously even foxes haggle after the ideas. blithely pending excuses are blithely. special as +9366|Supplier#000009366|pJp6kbd dtmX|19|29-565-703-8971|4207.26|boost! even, express frets boost blithely express deposits. carefully even foxes +9367|Supplier#000009367|ZOpGL3Xr7mVxPUVBTMhB6|4|14-282-158-8397|1458.09|ily special instructions. quickly +9368|Supplier#000009368|3EFu0xg,fQLL|13|23-124-665-7492|7075.28|imes regular patterns hag +9369|Supplier#000009369|pxUOpUtJ6chUEk|6|16-533-254-5081|2245.78|usly. regular, bold accounts wake fluffily. ironic, bold theodolites bo +9370|Supplier#000009370|EaNQ6rs49m|2|12-103-823-8017|1752.84|ackages are. furiously idle asymptotes haggle slyly accord +9371|Supplier#000009371|Pg7NzwadhYBgZnMadSuex|16|26-296-281-5239|-103.00|e among the carefully even packages. quie +9372|Supplier#000009372|K0SwMDlveHxfA7hD,95uOry3s2x|16|26-440-852-4319|4823.63|ely final packages sublate never unusual pinto bean +9373|Supplier#000009373|UXNiDODh2wwCKAQaKIWaWVRC7jDE7|6|16-248-857-9945|9807.46|s wake furiously express platelets. carefully final deposits int +9374|Supplier#000009374|,jWSmJIUTT9UiXB5Sq5 N8Q|7|17-647-773-5010|8057.69|s sleep quickly final foxes. flu +9375|Supplier#000009375|UcHq,f8GdVif,TsEP2C|16|26-106-910-1841|2570.31|ng requests detect blithely at the even, bold patterns. pending accounts cajole across the blithel +9376|Supplier#000009376|Gr91X3M1khLCOryjJ8hPe O|24|34-924-348-7285|223.30|sly special requests across the carefully regular theodolites believe across the +9377|Supplier#000009377|A ihYYF3 7p530eF1WEJ1pKb3Roga0X|13|23-627-251-8226|-11.89|-- blithely special deposits wake quickly bo +9378|Supplier#000009378|uuYVQC3cBerh6|0|10-277-616-7697|7157.49|ans. accounts cajole regular requests. carefully +9379|Supplier#000009379|WXbsxhGF6Mwr5Mdt9S8ls3gaaC,|22|32-808-320-2496|9033.28|s requests breach slyly across the packages. carefull +9380|Supplier#000009380|Ka9SQRCx8 KsA|8|18-852-634-2011|9344.35|uffily even dependencies affix carefully against the dogged foxes. fl +9381|Supplier#000009381|rhCTm7QehIznqd8 Np7VT,H5J5zSGr|21|31-405-984-7512|1619.90|tect. furiously ironic ideas wake across the unusual, even accounts +9382|Supplier#000009382|U635Ufh4OsWM8BBR8IbEOFFV0FQ|18|28-592-800-9172|9648.62| dependencies are slyly pending pack +9383|Supplier#000009383|VSXQdtUW4jdfEdQO8n8i,F5|24|34-298-463-2083|467.12| platelets haggle carefully requests: regular excuses around the deposits wak +9384|Supplier#000009384|Z7Txs1yOp4mVCG|24|34-393-170-2099|5776.82|ndencies between the doggedly pending requests serve blithely acco +9385|Supplier#000009385|UW lTJyEmFlJpSxGbi4sv,Q2EORMLEutHP8|10|20-188-719-2360|3098.48|uests engage furiously special realm +9386|Supplier#000009386|vyOL2YUwJuWYh39WsozXA,gFK2HCk4O6q23IVF|13|23-125-515-6745|132.20|ess packages. ironic instructions cajole about the slyly ir +9387|Supplier#000009387|7POl yxTprmpFaFkLy3m|18|28-157-601-9339|8901.67|oost carefully idle, ironic packages. slyly special foxes along the fluffily unusual +9388|Supplier#000009388|Y85UWL2NPa|24|34-514-815-5500|8367.31| regular requests among the blithely reg +9389|Supplier#000009389|D8QYyP4Gcr|22|32-582-109-6408|3453.27|l foxes affix carefully according to the quickly regular platelets. blithely permanent foxes sleep +9390|Supplier#000009390|UEI3Ch1ha0pkxW7dZKFvB5SrjK|2|12-935-196-4340|1060.04|e according to the even requests. even ideas should sleep regular excus +9391|Supplier#000009391|gjRrYc6Sgs0LTPfbI3z90P9ylAhOZW00b|8|18-450-578-2745|5098.01|ns do boost deposits. unusual instructions sleep fluffily carefull +9392|Supplier#000009392|hcvTG6rd7,y,3mZlRwNj6FpgR|21|31-453-312-7832|1345.34|jole ironic packages. ironic requests use blithely i +9393|Supplier#000009393|9dQepFzaR5,KlejyIFMqnKp7EMexTmxhGuky|16|26-874-863-6361|4116.28|ourts across the unusual pinto beans sleep quickly regular ideas. requests wake. slyly daring +9394|Supplier#000009394|ZZcjdMzFRIEOlyJSH|9|19-537-469-8663|1340.79|slyly enticing ideas are blithely after the theodoli +9395|Supplier#000009395|DAT9T,nuGGqPEQ|16|26-812-893-7269|4587.21|ccounts. slyly regular asymptotes haggle idly. carefull +9396|Supplier#000009396|iXhvx5pFwt,AssHirVoyWjfn|16|26-248-592-8049|7231.64|ges. blithely regular packages above the spe +9397|Supplier#000009397|V4vl6KxYjPGAA2BE65tMariCbkL|4|14-789-559-9865|1217.66|. final, pending requests +9398|Supplier#000009398|Y16cjXqC3d3qhALdFeZbchGW2|21|31-417-412-3717|3210.35|ffily final accounts nag evenly? quickly ironic platelets wake +9399|Supplier#000009399|QDf7NxmDrc17I1HlGaIOAZSeq|2|12-259-165-6237|-691.49|auternes wake along the express account +9400|Supplier#000009400|XkWV8h OMFxYQsYUvIuxQKeZ73hJiC9|21|31-540-923-5238|3023.21|usly ideas. instructions cajole slyly. instruc +9401|Supplier#000009401|VfE1W9ZfSHf673xmaOXSB3vv,PGBkAZIVV|8|18-117-294-2978|21.01| ideas use among the slyly final accounts. fluffily reg +9402|Supplier#000009402|1y9Pm8w2Ooa1,l3nE4o2il5H,mEnHh7J|19|29-673-127-9143|-927.36|iously ironic ideas. slyly even requests wake ironic pinto beans. slyly pending pinto beans believe +9403|Supplier#000009403|70841REghyWBrHyyg762Jh4sjCG7CKaIc|21|31-294-312-1056|9859.43|nto beans. silently final requests detect furiously against the even pinto beans. express f +9404|Supplier#000009404|etrfp1TkOyJi1KkZhnX7NYI5YdcUED|4|14-248-635-5968|1577.30|tes lose furiously across the fluffily unusual re +9405|Supplier#000009405|IY1x8B2MaqzWuk1KzA6FJua|11|21-778-728-5704|8717.99|al packages. slyly even theodolites doubt blithely. slyly regular asymptotes a +9406|Supplier#000009406|XvU79GhJcUgAycXY4 d1nJeYgwUJA,|0|10-201-804-6866|1262.50|xes around the even packages boost carefully quickly special deposits. fluffily special instruc +9407|Supplier#000009407|CHRCbkaWcf5B|23|33-903-970-9604|8754.24|e ironic requests. carefully even foxes above the furious +9408|Supplier#000009408|2RsQWoWJEhzXD|23|33-737-402-1856|5650.65|ounts. furiously dogged depths +9409|Supplier#000009409|iKXMUej 5HolL,Ev XIA,igY4YxA|13|23-268-232-1379|5533.67|ets. accounts are. final packages engage against the fluffily final courts. b +9410|Supplier#000009410|E7APdKQQ,F uMjzN1eBtBCNIvDBT|11|21-628-299-2581|8408.75| express packages. slyly even deposits across the ca +9411|Supplier#000009411|y 9HOu3o5fXmQKc7VmA|18|28-902-415-7563|6900.08|according to the final requests. slyly silent instructions boost furiously-- fluffily un +9412|Supplier#000009412|0mEfNNYfNFYEDZM7Nu|7|17-952-891-5414|5757.24|e quickly. slyly silent deposits maintain furiously about the always special accounts +9413|Supplier#000009413|JFTZ2SoojgnJ EhWV8GX9lEO |11|21-233-261-7626|3870.54|luffily pending requests. even, pending dependencies are blithely final asymptotes. special, +9414|Supplier#000009414|U6A0 1BMgkAONf1PT4CqHfwPbuYO,89|0|10-425-901-8610|7161.89|ely about the quickly regular ideas. qui +9415|Supplier#000009415|3fyJj02swxtfgxdPt6LnP|12|22-127-774-5625|5882.37|hely pending packages. quickly express foxes could are slyly. deposits sleep dep +9416|Supplier#000009416|nLxjHC,x5X|2|12-274-652-9680|5968.71|st the ruthlessly ironic deposits integrate about the pendin +9417|Supplier#000009417|Ebq36O1zgEr9HuT22ur3YuAGy2Ey0 wf29aIFLGa|16|26-108-259-3708|5058.97| slyly final theodolites above the slyly ironic platel +9418|Supplier#000009418|chJOHEWwW2Iec5roXAeLiyhe8fKYus|0|10-160-293-8608|5105.25| packages hang fluffily slyl +9419|Supplier#000009419|ltaHUzApJXC3j1YFLoOvzzgG|17|27-444-956-1077|1891.16|ymptotes sleep slyly blithely fina +9420|Supplier#000009420|k88TWsLXugi gAK,dAxxTsVPUaBNxRGT1JzUB6Mc|19|29-892-108-3027|3769.79| ironic accounts. quiet theodolites wake slyly bold the +9421|Supplier#000009421|M6imBfUIHAcsg82pkol6DSeAoxoL|15|25-522-739-5607|7898.04|ackages sleep quickly furiously special accounts. furious +9422|Supplier#000009422|Pqn66BjZKMW2bjIBIGU5Zug6KP5|7|17-343-403-5899|3437.77|- express foxes about the ironic, +9423|Supplier#000009423|N0Fwn7PKT0vlpkRKDm9Bjmw|17|27-624-256-3285|4594.99|oss the regular, regular excuses. slyly busy accounts wake qui +9424|Supplier#000009424|vD55Sjf8apBOAUN,xdSM67l2qu|10|20-447-687-3667|5974.29| alongside of the carefully unusual accounts. fluf +9425|Supplier#000009425|wARLFXaMe3UlIirswe1eeK 8 ocG|8|18-917-996-2056|-21.68|efully. fluffily regular packages affix regular instructions. sly, unusual deposits haggle among +9426|Supplier#000009426|go2c90uF4TsZEHg1aa3,Gacwm|23|33-367-498-1524|7626.56|ns sleep blithely. furiously pending pla +9427|Supplier#000009427|clo4KAvzhCQVT,N|0|10-645-328-5829|5598.29|ges. carefully final accounts about the fluffily regu +9428|Supplier#000009428|R5CHcFZIdnWOvAjRKV4sV0CvfQrtZQKBRzAqaQ|1|11-896-966-5146|5429.37|ide of the slyly ironic requests. regular, even requests solve slyly? +9429|Supplier#000009429|LvcH9y3y5Q2XaN,vIo|5|15-495-957-2715|4469.17|must wake slyly even packages. frays haggle bus +9430|Supplier#000009430|igRqmneFt |3|13-491-379-4658|3345.48|aggle blithely at the silent warthogs. regular foxes cajole along th +9431|Supplier#000009431|mW0Rm,UuI2me9fRpvl,2Mt|1|11-731-702-3171|4537.20| warthogs hang slyly carefully even pa +9432|Supplier#000009432|Pjys2qZZQjh3QH5tFDv,m|4|14-855-482-5723|2793.26|refully ironic packages! ex +9433|Supplier#000009433|Dq3tVIK2Hz3lV43y6EMZe8|18|28-587-206-9649|7225.92|uctions: slyly regular foxes cajole about the blithely regular foxes? requests boost blithel +9434|Supplier#000009434|zBhG4DSvr0YB6eQ47vEa,YQDks|18|28-960-513-5579|4018.30|hely. final, bold packages across +9435|Supplier#000009435|qFOrLCJ2smwJsguigRCMb6Go|20|30-570-980-9428|4574.44|fily regular theodolites. hockey players haggle furiously. blith +9436|Supplier#000009436|RQ20, N7F zgHzM|20|30-533-831-9235|1636.50|dolphins. regularly final deposits +9437|Supplier#000009437|43Of djBBL2oVjDD4X65 Gw9xmTcA8J|8|18-867-673-9947|2572.59|osits above the furiously express +9438|Supplier#000009438|A8udVGwu C6al8s|0|10-348-722-4688|5567.21| the ruthless accounts. blithely regular pinto beans +9439|Supplier#000009439|8WNOOTo42CCztegtMJ|10|20-433-939-8255|98.93|d, unusual deposits haggle slyly. carefully unusual packages slee +9440|Supplier#000009440|p20oDM2OMRYZqjfZt6jQyGA7GA,7T|9|19-324-357-8746|-308.55|lphins near the furiously regular +9441|Supplier#000009441|VOIqo9OoPfEtss6V3FRn,x|23|33-124-494-8728|4173.57| are fluffily. fluffily express accounts wake iro +9442|Supplier#000009442|Dg5Y U0gL R0pYVhnFOfwe2V78,EupzWXz1f S|10|20-558-408-5310|7124.26|arefully about the final accounts. furiously bold courts may promise perma +9443|Supplier#000009443|vjy,6EUB9nTBC6sv|8|18-407-362-3569|397.54|eposits haggle carefully among the regular dependencies. deposits sleep s +9444|Supplier#000009444|K7CLkDGsMENC3|17|27-680-497-6795|2418.87|bold accounts. quickly ironic packages abou +9445|Supplier#000009445|AyEP8mMQ8VpXLwCIj0|3|13-989-455-3374|6988.35|o the ironic requests. bold packages wake blithely along the carefully regular foxes. ironic pac +9446|Supplier#000009446|HwZhB3eURV|10|20-717-662-4961|4281.76|ffix pending deposits. special deposit +9447|Supplier#000009447|6FkKgpERpWPqKt2|2|12-861-266-5178|-498.50|n accounts. permanently express packages wake after the ironic, ironi +9448|Supplier#000009448|GtJQnxMJXtY7avSqfHrnDx0,DSsWyV2|1|11-945-401-8077|1681.00|ideas. slyly final ideas across the carefully regular the +9449|Supplier#000009449|3kGy7IDpszy3dgSN|4|14-608-640-9937|2479.92|ounts promise. quickly even gifts haggle blithely. carefu +9450|Supplier#000009450|0d zNrvhTpg|10|20-729-897-5564|9999.72|totes. bold, blithe packages nag against the slyly regular instructions. bold, +9451|Supplier#000009451|cgBcBoa hRGgwKZD9V0Is0Q|17|27-895-943-7799|5389.53| blithely unusual packages do wake furiously. sl +9452|Supplier#000009452|,xOqIi1xiqjdktPVwKAZapDwEFvzfToWb|9|19-392-969-9048|5878.23| after the ironic packages! quickly final packages after the final, bold accounts integrate +9453|Supplier#000009453| rdngVAY7,|9|19-403-545-2290|2438.33|ial ideas wake blithely asymptotes. express, even instructions cajole; sl +9454|Supplier#000009454|l4BlqfmS,s2NHv7jS4ElH7kXlfqG7iJSNouN83n|4|14-944-860-3507|3959.19|lly express deposits. unusual sentiments are slyly against the dolphins. sl +9455|Supplier#000009455|HsyjLk3pWq9IOKybngOTES|21|31-284-902-1568|1017.05|ironic ideas. slyly ironic ideas wake dependencies. regular deposits haggle carefully idle deposi +9456|Supplier#000009456|7SBhZs8gP1cJjT0Qf433YBk|23|33-858-440-4349|8457.09|cing requests along the furiously unusual deposits promise among the furiously unus +9457|Supplier#000009457|7trQrMbRXGK7bUSJOpn7zYoViyi7Mzwx|22|32-502-244-5147|5298.86|regular asymptotes wake blithely alongside of the slyly final +9458|Supplier#000009458|SEPWjPcp03 JMpsCzShQuYAkCFuLXg|12|22-198-686-8871|-131.51|nts. slyly special accounts thrash furiously. quickly special package +9459|Supplier#000009459|eNcR5W,jns|7|17-230-560-9991|9563.10|. carefully even instructions affix slyly even deposit +9460|Supplier#000009460|j4O6xdqHc029jy5iPBgNv3E|3|13-256-210-9297|4023.16| careful, even Tiresias sleep fluffily +9461|Supplier#000009461|Eweba 0sfaF,l4sAxXGTgEjzsJsNWWIGjYhFkRWV|4|14-983-137-4957|9227.16|lly bold packages. carefully express deposits integrate about the unusual accounts. regular, +9462|Supplier#000009462|ocOJDB,O5q5|4|14-649-880-9503|931.72|ounts wake about the enticingly careful p +9463|Supplier#000009463|63XSrzifVZOvd5L8BmrDebOaCjj455pBzHQ7qRT|15|25-233-652-9638|4300.40|osits x-ray furiously special, express frets. furiously unusual asymptote +9464|Supplier#000009464|93Tm7,TXo80Okugwz|16|26-697-450-7238|4777.12|furiously express courts haggle. slyly regular Tiresias use quickly blithely +9465|Supplier#000009465|d5JrHvIlyzdFoO|23|33-947-207-4786|9071.56|ding to the carefully ironic accounts. unusual packages bo +9466|Supplier#000009466|GkdIYwzK5GCYibM4GJwd XF0e5D2ei|2|12-440-678-4925|3794.31|posits wake furiously blithely even reques +9467|Supplier#000009467|JbdJxWu,7IXIeLdbYYDPqXaBN3MV|18|28-127-496-1536|1924.74|ingly bold foxes are quickly regular t +9468|Supplier#000009468|QRospU5z3JWmvPAgFWmWX2id3zqYYkL13brplY7N|6|16-522-144-8872|7044.69|ke slyly special instructions. +9469|Supplier#000009469|ahcTcVHEtTUzfNNlKnkDV,aBYrjkiO5VQhfOfzz|1|11-934-586-5383|5140.63| blithely even foxes according to the packages integrate f +9470|Supplier#000009470|O5PU2sjqusThzdR3QFGwA|17|27-230-749-2592|2638.93| final requests. final, regular requests are blithely blithely ironic ideas. b +9471|Supplier#000009471|DbfJ 7VNkFOnU4W,7o|11|21-578-514-3601|9517.36|en asymptotes integrate quickly blithely even deposits. express, regular deposits haggle fluffily +9472|Supplier#000009472|ldLdl2JVJeRBaZ|15|25-302-813-4531|5315.78|tes. slyly special ideas are blithely. carefully final ideas hang carefully blithely bold pinto +9473|Supplier#000009473|xsiD1RqV3WNgvy0|8|18-118-279-3166|9393.55| fluffy deposits? slyly special foxes nag blithely. even deposits across the enticing pains are fl +9474|Supplier#000009474|PLSQC8RN6Pvmzi LFy14V 4SL,oQieryN2PD4G|2|12-259-669-7973|3443.58|e carefully furiously unusual theodolites. slyly even pinto bean +9475|Supplier#000009475|VXu9a M8v0EydULIq1j0sp7p3r8AATYn4|16|26-234-208-8814|3161.80|e of the bold, bold dolphins promise slyly quickly ironic pl +9476|Supplier#000009476|EtdRuZQx9Jb8zm8fbOypg1aI1YgcyUCNQ22F|4|14-504-219-3202|8418.69|n asymptotes. silent platelets boost silent dependenc +9477|Supplier#000009477|xJAsagJoSpeqmYvwdX7DeqVkYL9URZCmIq|16|26-838-945-2162|9034.64|ial requests at the furiously special courts sleep fluffily +9478|Supplier#000009478|74FcBmlsV5X9ABR,kiCd25|23|33-723-212-4011|-737.17|arefully bold instructions wake fluffi +9479|Supplier#000009479|tNBWHgO87pZu9oxZHyycVwb73syHwis64g8HLP N|19|29-321-285-3408|5274.87|efully bold pinto beans sleep carefully about the carefully ir +9480|Supplier#000009480|7X9XBzr5m,035H0PKixS0H728TYKL g5|14|24-941-288-8716|3827.96| of the deposits. quickly ironic platelets are quickly r +9481|Supplier#000009481|pVlkK1po67Tw|16|26-369-206-8249|6460.68|ts promise carefully about the ironically pending foxes. special, final instructions integrat +9482|Supplier#000009482|GbYzTER1oFDsOecbywD3SVG|14|24-577-195-4892|1708.01|elets snooze. slyly ironic realms af +9483|Supplier#000009483|x3PUGAy4lrknjrVMUeVnktwvYLBK4tjXAM|10|20-331-258-3889|5516.15| slyly pending deposits cajole among +9484|Supplier#000009484|NduT RdP6FHKsVzdiJiasyFTVhpSFReHbUN|19|29-208-651-5479|-60.23|encies boost carefully busily silent ideas. deposits across the regular foxes cajole furiously +9485|Supplier#000009485|UhGFZ0dCbU1I6 oIgmsM WWpb3,|19|29-507-123-7081|8578.28|l theodolites. bold, even platelets about the bli +9486|Supplier#000009486|1JELMqGNrvKNxpWV7gUHRA9U|17|27-455-715-4230|7186.79|s. accounts grow slyly regular +9487|Supplier#000009487|KhxgDXsSiapyewwkfB9KDFZo1XIEL6QJq|11|21-478-786-6858|6415.49|slyly regular instructions. requests nag +9488|Supplier#000009488|Sh0ia0he7Z9jiOqTgtaiN0Sa|12|22-813-963-1426|1121.19|y ironic somas. blithely stealthy deposits around the blithely even accounts print quic +9489|Supplier#000009489|B1cFg21QWRDxxvpo|11|21-229-305-3169|9822.40|egular excuses. carefully special dependencies sleep qu +9490|Supplier#000009490|pjD9sH0WR 9GL2jYDdXUsrWxOvXx|19|29-373-605-6011|6181.11|en packages beyond the carefully express requests cajole furiously carefully special ideas. closely +9491|Supplier#000009491|M3m6oIzOaFleyE|9|19-202-382-3047|-125.27|across the slyly regular foxes. courts sleep quickly between the carefu +9492|Supplier#000009492|6kPCGz66rAPqwvJuBuVjRQ6VxTRy|17|27-417-928-8655|4521.69|its. ideas around the carefully regular deposits hagg +9493|Supplier#000009493|CTYJSPTsYe5W9IG|17|27-454-332-6489|6093.86|ven instructions are above the special foxe +9494|Supplier#000009494|i8Ym7MV7ftzioDJpZG2JqNcF3La5|6|16-992-789-3674|7597.74|packages. furiously regular +9495|Supplier#000009495|1QldlQ1kvyt5E4dcql1|23|33-181-419-7054|5316.72| silent platelets. regular r +9496|Supplier#000009496|IkZImKwI93PpIeGNTiQzplVN0hj3QUq5RQqkv|17|27-668-429-5083|3182.47|l theodolites detect about the deposits. quickly final theodolites wake to the quickl +9497|Supplier#000009497|xR5T0kfG8aorZiHdj68G7gsJrzg7Y0,l8YLyP4q|10|20-952-509-8484|3777.38| the ironic ideas. fluffily +9498|Supplier#000009498|dDkobGZ5hwO9aUDG0ipVJgJ8|9|19-999-351-5454|3338.80| furiously even sauternes cajole fluffily furio +9499|Supplier#000009499|b7pbLXQCK,3v58O8KpogTTyeTcF5HOLo72,izPx4|11|21-130-844-7565|2194.84|e alongside of the blithely regul +9500|Supplier#000009500|vgmapeI1rH4c7tIp8LaTbd|4|14-577-604-5292|3659.40|usual accounts. fluffily silent asympto +9501|Supplier#000009501|vFug8y0A vcuINZRui3Sgx|2|12-504-265-7327|525.33|press instructions wake slyly finally thin packages! slyl +9502|Supplier#000009502|eGKoX8yZf6qHMJdqQ0XRhH8,9gNs3z4|0|10-542-966-6164|5083.63|arefully regular theodolites slee +9503|Supplier#000009503|,zh2YkK0iw8H7|2|12-214-380-6494|1507.77|refully even foxes. carefully ironic requests along the express, final foxes must h +9504|Supplier#000009504|Rqt07,ANI92kj1oU|21|31-975-106-3065|6029.82|arefully aboCustomer gular packages. idly regular requests booComplaintssly fluff +9505|Supplier#000009505|n1pXScGMekx23nPfg9vgGM4Sy8kd7IQmtU,O|19|29-746-694-1348|9156.28|es haggle carefully even ideas. regular requests play quickly across the c +9506|Supplier#000009506|YqNm9Ml0j,8NhZntYZKzJgWZdrUxX7t|3|13-151-102-5307|8607.27|the quiet foxes. quickly bold accounts use furiously about th +9507|Supplier#000009507|WJFMxE65dV7IiS0s40hTU|1|11-332-398-3201|-471.22|osits haggle even accounts. slyly ironic warthogs are after the carefully +9508|Supplier#000009508|NVr1A8EDasnF6SHLmtRVHc1nOUsFW|17|27-937-803-8397|9067.53| deposits breach slyly up the slyly final +9509|Supplier#000009509|D6z9W8Aux84BAAf81IOn86,Na|21|31-755-704-8601|1962.57|ctions are carefully; regular epitaphs wake furiously f +9510|Supplier#000009510|G5WOxixdya,crcuKGGWg5zDcEh7j|15|25-304-914-8621|4918.03|heodolites print busily special foxes! furiously final excuses nag care +9511|Supplier#000009511|rhk,0T4C OEPLh|2|12-972-137-5914|1291.23|special ideas cajole carefully blithe dependencies. instructions from th +9512|Supplier#000009512|DiSWvB4WPxhpN4hOr5e0WC2Q|14|24-168-367-6579|7589.84|al theodolites according to the carefully spec +9513|Supplier#000009513|qsAMCRV3wenlgA5bjn Vh|7|17-157-731-5380|4501.97|equests about the regular +9514|Supplier#000009514|apzE0QbvdwN6oTQqsq2|16|26-931-519-3357|4608.56|ly. instructions run carefully. bold dolphins wake blithely about the quickly ir +9515|Supplier#000009515|thTcYEdId1T5lsReuSB9XZuMY0Vx,dqK4xB |12|22-100-384-8656|3051.66| carefully regular dependencies thrash above the bold, regular dependencies. ironic ideas integ +9516|Supplier#000009516|EIRGJa39IZtPrz0dBCupUgwLSQnODy4zQa9YZ8PX|19|29-556-916-4227|5298.74|uffily regular pearls. blithely bold requests detect furiously furiously slow deposits. slyly regul +9517|Supplier#000009517|AaX3LSpYHqIFeMksOH2cbgP3pxA,eUXGi|23|33-615-803-7174|8867.66|ts. carefully ironic platelets detect blithely. furiously even foxes affix +9518|Supplier#000009518|,GTEu,DAA9M8H,IDjKvLha80v|0|10-688-366-2571|8796.02|s after the furiously special deposits boost furi +9519|Supplier#000009519|zz6MbTdtLbEZy8yJ3KpNR5Sk4cF|10|20-995-231-6289|3766.93|about the even, regular epitaphs. epitaphs before the blithely permanen +9520|Supplier#000009520|mbkbB86HvAwDWP MO7o|9|19-758-374-1549|3862.29|s. carefully unusual foxes integrate alongside of the slyly final braids. ideas sleep. blithely sp +9521|Supplier#000009521|HBTeAVQRNCpt3ws|2|12-890-995-1809|454.71|se. slyly final foxes above the speci +9522|Supplier#000009522|E6cXgsoyeJCXZBGE|18|28-529-969-5814|6165.45|ronic deposits along the stealthily special accounts integrate according to the f +9523|Supplier#000009523|8011s4G6Je2,iqI60BKkCbv ucFwT,N|8|18-745-759-6087|773.94|. ironic foxes use around the special, regular +9524|Supplier#000009524|bt0azfQEYoLynPBBS|11|21-835-774-3116|3627.83|erns use carefully. slyly busy instructions a +9525|Supplier#000009525|YT2XG1NUqA5JXWWJqD6XKqZv|14|24-824-428-7726|8988.85|fluffily regular theodolites. quick +9526|Supplier#000009526|oVTCHA9Civ9d1CghlwI|10|20-788-250-8252|-248.80|express packages. furiously slow accounts sleep inside the slyly silent waters. furiously express th +9527|Supplier#000009527|RmgAFGfsFttrkvP45QIcfP2|4|14-922-292-1729|7355.08|carefully accounts. pending ideas about the pinto beans wake furiously acco +9528|Supplier#000009528|n,bNmusFFl8LmLRWMSBkDwj2Hr,rhhBGFdV,Tr3r|2|12-181-831-1307|7825.70|beans. carefully unusual instructions haggle carefully even courts. platelets nod. blithely e +9529|Supplier#000009529|e44R8o7JAIS9iMcr|22|32-565-297-8775|8517.23|ove the even courts. furiously special platelets +9530|Supplier#000009530|eFhnjFzyu5yN,4nMFIbxEFT5bs84 RIt3|5|15-563-724-4838|4122.73|sly regular foxes. quickly bold instructions about the daringly regular deposits detect carefully th +9531|Supplier#000009531|AR2f89X8RBUGDAjlCqv0dR96Vx0S3BB|17|27-926-192-5306|-204.67| the unusual deposits. pending requests ca +9532|Supplier#000009532|2BGSfbMcpCfd4Dam|17|27-477-596-8795|1009.43| packages use furiously ironic somas. furiously express +9533|Supplier#000009533|ns0A1FojntfOHziW|4|14-153-655-7458|1432.39|ependencies boost blithely ab +9534|Supplier#000009534|w Q,QHS1dfqs3BBrTlijJuiLQ1I05sWnHWCiiW1I|16|26-519-835-6320|4112.80|xpress foxes wake against the accounts. even requests sleep blithely: furiously bold deposits a +9535|Supplier#000009535|NF2jFE0JxJ9GEUoiTZJb|21|31-233-536-6891|8824.16| ideas cajole daringly regular theodolites. slyly regular req +9536|Supplier#000009536|Vg3xBMS9wMP3h0O|9|19-415-490-9964|3749.78|ctions poach. bold, furious reques +9537|Supplier#000009537|R8itHxMd6ZV94bil04Ln|10|20-394-720-4605|1555.97|ackages cajole furiously. bold instructions alongside of the quickly close acc +9538|Supplier#000009538|Ayts8hIvhFOuCtoGe5G9tbcFg|10|20-950-276-6612|2144.19|nusual, special requests after the quickly daring theodolites should +9539|Supplier#000009539|VuAZtzbmAuNfMzxJh pT9PB|21|31-611-397-9677|-639.76|unusual pinto beans. furiously regular dependencies are +9540|Supplier#000009540|iIq4Ut0C1SZEAF7|3|13-735-588-3785|3440.37|ckages sleep. regular requests haggle slyly sometimes unusual accounts. doggedly even ideas impress +9541|Supplier#000009541|H3LJDN led0rF3L0UlU|19|29-222-311-4609|4296.28|careful, final deposits nod slyly along t +9542|Supplier#000009542|I 6RF2VfTAlFdpmsrA|22|32-300-861-4282|1647.23|ar packages nag slyly. furiously pending accounts sleep along the ironic deposits. regularly pen +9543|Supplier#000009543|VYKinyOBNXRr Hdqn8kOxfTw|20|30-785-782-6088|7023.47|sts. furiously pending packages sleep slyly even requests. final excuses print deposits. final pac +9544|Supplier#000009544|cxrdj8S QG|17|27-239-678-8265|541.66|unts detect. regular, silent requests at the furiously regular instructions cajole bli +9545|Supplier#000009545|8uE8ivAy2tT u867PxlsOdTypcYIomX4ucQqi|23|33-727-629-8262|4535.21|sits use ironically alongside of the slyly express packages. warhorses detec +9546|Supplier#000009546|2eylkElhdzW694N4gROtj4OGq2dHKzY|13|23-909-997-5890|2338.83|fully slyly final attainmen +9547|Supplier#000009547|ahhOA3wIWV9BTEmnI dh2j2z3dRhLANmJuEZNtv|2|12-426-927-6059|3690.55|le pinto beans. even packages nag fluffily pinto beans. regular, ironic i +9548|Supplier#000009548|64AnHprl6 |14|24-231-544-4360|7131.36|e furious, express accounts. expr +9549|Supplier#000009549|h3RVchUf8MzY46IzbZ0ng09|3|13-892-235-6781|9810.82|arefully express dependencies inside the even instructions sl +9550|Supplier#000009550|rsOABl2HDhHUl4kpPsj0gbZPONYWOrfbs8|9|19-471-689-2492|9524.33|re ruthlessly against the fluffily bo +9551|Supplier#000009551|8fW3HZk0UVkr25gA|10|20-260-266-9203|1451.82| final dolphins affix. ironic, pending braids boost. slyly unusual instru +9552|Supplier#000009552|l2CqO,Zj1h1N,BKH, SE7bek2Js|11|21-867-401-8002|685.72|s. carefully ironic theodolites use carefully above the pending deposits. quickly fluffy id +9553|Supplier#000009553|gPmqyLlLtS|7|17-786-490-4101|5915.23|ideas after the stealthily even pinto beans are furiously final asymptotes. carefull +9554|Supplier#000009554|e8De3zk2cRdvxQihN7IArownHTJFTgujVthQbd|17|27-190-154-5202|8219.58|st carefully permanently ironic foxes. +9555|Supplier#000009555|cTdJD CtZ0E|16|26-702-194-6953|6483.82|ly platelets. blithely even requests are blithely. quickly unusual accou +9556|Supplier#000009556|NN Jmwgg0xC0qI9pik9O 5CJj7m qQ NEZjlJpgF|8|18-505-494-6786|5580.86|iously special ideas haggle fluffy, final pinto beans. slyly reg +9557|Supplier#000009557|HGuiT0wySXim,4t26Mn9R3zt0 n|23|33-283-740-7203|9600.65|onic deposits; furiously express accounts alongs +9558|Supplier#000009558|mcdgen,yT1iJDHDS5fV|22|32-762-137-5858|8231.61| foxes according to the furi +9559|Supplier#000009559|8dQO1,H5piQRBDV,96PYVVCXW2dunI0yKXJJ77Bh|15|25-566-702-1125|-194.72|ironic accounts sleep. courts cajole slyly. accounts sleep final deposi +9560|Supplier#000009560|c39bMVAzQ kCPl|2|12-502-695-6531|3352.75|kages sleep fluffily atop the carefully unusual accounts. final, final ideas sleep. furiously unu +9561|Supplier#000009561|f,hPXkNNv5bFThwlF|20|30-146-794-2416|5708.47|ld requests against the even packages haggle slyly final excuses. regular +9562|Supplier#000009562|073rt4aydFLO,48ecGgo1vkZ27,oQWrWyP1UKw0e|14|24-474-154-8020|9289.10|uickly across the quickly unusual accounts. furiously even foxes serve furi +9563|Supplier#000009563|fl dNSU309HEd1K|9|19-638-205-8748|7982.50| dinos according to the regular instructions would maintain slyly against the enticingly r +9564|Supplier#000009564|K ezA5Ud86lp6Tnc4ynd90RNu4|20|30-395-764-3694|3894.15|deposits! blithely express warthogs detect slowly regular deposits. special dolphins are. even, pen +9565|Supplier#000009565|mP7SmIA,AAgm5usSVke|23|33-172-774-7000|8031.56|inal excuses. regular deposits cajole above the carefully ironi +9566|Supplier#000009566|gFy8fwG36c1Ri|9|19-198-116-3424|3516.51|its. regular, ironic courts use blithely depo +9567|Supplier#000009567|r4Wfx4c3xsEAjcGj71HHZByornl D9vrztXlv4|3|13-816-610-4613|7911.07| quickly pending accounts about the fluffy theodolites are permanently at the slow realms. regular d +9568|Supplier#000009568|YnMsjNT7FDi|23|33-436-335-8486|2359.12|across the ironic waters. slyly bold pinto beans are carefully express deposits. f +9569|Supplier#000009569|6cZ2S,AnpIftS2xlb|16|26-497-997-4339|3646.28|kages cajole after the carefully ironic do +9570|Supplier#000009570|WquWCAaKMObu0|1|11-524-770-2559|82.60|ithely about the blithely +9571|Supplier#000009571|TeIMA2rqtweBI5cKTb9LPqtgXJ2Ft6H4|17|27-918-538-6528|138.62|ronic pinto beans. decoys are slyl +9572|Supplier#000009572|dRAhJ GXEdFjiZ|9|19-760-265-9900|6769.67|he pearls. even foxes haggle slyly? regular +9573|Supplier#000009573|cz,oBNKLoI,9hE1u|14|24-293-594-6031|2953.08|ns. furiously ironic deposits use evenly special, regular +9574|Supplier#000009574|zjx57 niIV6 ENacO11W6aE,aW0ohfHum|0|10-210-767-2649|9324.34|uriously after the final, pending deposits. ironic requests integrate unusual, ironic Tiresias +9575|Supplier#000009575|qAdrnjTmcGUqpcVQy CdX PIiFGea,TwFl|24|34-882-374-7778|6338.34|ackages nag slyly final packages. slyly bold pinto beans haggle dog +9576|Supplier#000009576|uNdNymD,2MKa|12|22-213-509-3533|2536.76|he furiously express courts. accounts play carefully regular requests. finally final d +9577|Supplier#000009577|u83dA14yM5SC8D|2|12-492-697-9085|8328.31|s behind the instructions hagg +9578|Supplier#000009578|n4jFVMArQUVmowZ|14|24-175-111-3909|7984.37|e after the slyly special instruc +9579|Supplier#000009579|qaw1e189AHujl,dH4FONtwNIrXhAFlq96 |11|21-168-505-3652|2305.49|uriously ironic pinto beans against +9580|Supplier#000009580|xdCOHsFbZAnBNDHjV|9|19-206-374-7815|4364.26|ress platelets. carefully pending fo +9581|Supplier#000009581|wwFM3EPThCi87xaXYbnYaiR|0|10-968-746-9540|122.88|lithely bold deposits mold furiously carefully even dep +9582|Supplier#000009582|qqlRS8h9MtF5KTkRXNl,g46SPVJ5qL4uunj dA|9|19-583-131-4953|6140.45| alongside of the foxes haggle slyly according to the regular dep +9583|Supplier#000009583|mD7rK64MLVWTE sZ1wPtGVz76a|13|23-970-824-2232|9518.41|gs cajole. bold, regular accounts nag blithely against the spe +9584|Supplier#000009584|TCXXzBEakqzEITAtIRrg6UoX2|21|31-303-396-1173|1033.55|uffily unusual deposits cajole carefully. packages doze. accounts above the regular excuses are slyl +9585|Supplier#000009585|hoefu6j59L8HoexLEFQ8N7Rj54zOEu2|19|29-491-422-1223|-127.07|ts according to the carefully special epitaphs haggle boldly regular accounts. specia +9586|Supplier#000009586|HHhopA7IfRzMwhmr5LQgIkuBjmqdAssWY Ko|19|29-491-553-2254|7330.25|furiously. blithely final escapades nag after the slyly unusual packages. bold theodolites +9587|Supplier#000009587|tWLWLuqaJuIPELHYIeSs6B4VG95IcvUSti15|7|17-306-558-6416|8556.20|ymptotes need to nag daringly acro +9588|Supplier#000009588|TQdnmIKYOFYBUrPt5 QaCli|19|29-438-189-2062|-807.05|ect quickly express, regular instructions. bold, bold dinos s +9589|Supplier#000009589|yw AjjynmKr9KZqx5ifhTUym|15|25-341-250-6793|7883.41| furiously express, ironic instructions: carefully even courts are fluff +9590|Supplier#000009590|yV1diMoLMrzxRpCTW|15|25-127-666-3481|-547.53|ons sleep regular warhorses. fina +9591|Supplier#000009591|4ksWQNho8d7UzvjZ1Ea5N40tI8Y ZqDLHqC|23|33-464-958-6370|2558.48|d courts. regular pinto beans need to x-ray slyly about the fl +9592|Supplier#000009592|nDJmoHuCKTDyED|18|28-477-309-2947|-607.20|ly final asymptotes doubt slyly ironic, final packages. even i +9593|Supplier#000009593|214GzPRCG1yeCKBVe|17|27-374-794-4187|646.63|lyly pending ideas cajole. blithely +9594|Supplier#000009594|t5POxK43OIjIfHTB90|21|31-703-320-6583|8803.64|y carefully express platelets. bold, +9595|Supplier#000009595|iaM0r6M1JBwC|4|14-968-467-7333|-132.86|d the quickly ironic deposits wake furiously quickly regular dolphins. carefully regular inst +9596|Supplier#000009596|tj7GQwf5bdW|2|12-121-359-3545|8993.38|r the even sauternes. dolphins sleep carefully blithely special dolphins. stealthily +9597|Supplier#000009597|NzBmqXeWCAaQcPnn,nldM5XeD7md6G5qBqGQ|0|10-525-491-6656|9823.56|refully ironic dependencies. blithely ironic pinto beans can nag slyly above the permanent dolphi +9598|Supplier#000009598|PnTAz7rNRLVDFO3zoo2QRTlh4o|21|31-420-284-5692|3576.60|odolites are fluffily silent requests. even requests are quickly across the requests? slyly pen +9599|Supplier#000009599|Bxx52Q,TBT9D12IhEhUJicG83Oay6|22|32-554-210-6590|5846.67|uriously ironic dependencies-- pending deposits kindle evenly? ca +9600|Supplier#000009600|bNbzMM5yGeouk5a12O3Wb7keyDsB3tp1 A|2|12-989-200-4223|10.08|f the quickly pending grouches use within the spec +9601|Supplier#000009601|51m637bO,Rw5DnHWFUvLacRx9|3|13-173-966-5906|2399.75| instructions. carefully even accounts u +9602|Supplier#000009602|JljsuUqEqqluhObJMMM5iHO6jTeLLwKgfa|16|26-137-116-6920|4619.15| blithely final accounts wake furiously above the blithely express deposits. c +9603|Supplier#000009603|mieAE7uLaJHrNZa3j3JJ5iXz,RqIGS|4|14-442-570-8298|5008.34|ording to the unusual accounts sleep +9604|Supplier#000009604|oujCr,AcGJzs8ThIc,MGYjwbkZqkipn0|9|19-711-135-4502|1232.40|g warhorses. bold requests boost blithely carefully regular pinto beans? permanently final accou +9605|Supplier#000009605|GkKVjuaqxBJkLtfaldYevgISGjg1eN0es|22|32-570-408-9915|8932.60|ideas wake fluffily. even, special instructions snooze ironically express excuses. slyly special wa +9606|Supplier#000009606|vgEUU5IA1h|12|22-596-455-9041|315.56|uctions. pending, silent theodolites sleep furiously +9607|Supplier#000009607|Y9jmgrgp0gHXzDn0PkTsYzy|16|26-250-610-6432|-372.50|n carefully along the pains. slyly pending +9608|Supplier#000009608|xYoUMT40amiTZpsYQ|11|21-776-627-8404|7328.38|ts after the pending, slow packages cajole sometimes sheav +9609|Supplier#000009609|LV2rJUGfr0k3dPNRqufG1IoYHzV|21|31-665-869-6806|-874.84|ven theodolites. ironic, even packages are pending, ironic foxes. special requests +9610|Supplier#000009610|LF3yXRodMjSJONvtGlKnNR2BgVvXSRDAuLt8AE|4|14-798-991-9397|1516.57|leep quickly alongside of +9611|Supplier#000009611|QkgF16C9A,OnUZPvWncMeD|11|21-494-489-3646|-92.76| final accounts. slyly final ideas above the silent, final dependenc +9612|Supplier#000009612|oZpGwogaNBBnfzZUQVoobSq4T9m|2|12-647-830-5763|883.41| the even, even dolphins. carefully final asymptot +9613|Supplier#000009613|f8IKA,xaPip7vU0NPbqTBzfn|24|34-265-538-6774|6358.51|kly special deposits haggle. slyly special deposits haggle blithely against the depos +9614|Supplier#000009614|,rSrq2IiWb5VGH,sRw|17|27-780-247-8960|447.34| quickly asymptotes. furiously ironic deposits according to the quic +9615|Supplier#000009615|cfZ4WWa4Pm|11|21-198-874-8274|3350.90|ermanent deposits. carefully b +9616|Supplier#000009616|1SnRLf5JOmleta|11|21-570-969-3630|7551.79|nal packages are slyly. ideas thrash. carefully pending ac +9617|Supplier#000009617|HyhsQSSE2k2K825gYP3PYoZ7pr|9|19-741-860-7232|2932.96| requests about the regular pains ca +9618|Supplier#000009618|5WQK0isGUjwwe,f0hSV tvg7uSKLW3,|9|19-592-948-7307|1262.11|urts integrate furiously furiously silent accounts. even sheaves engage blithely carefu +9619|Supplier#000009619|K0RwcJ9S75Xil jqKukFoDNkD|21|31-359-794-4194|2222.77|ts integrate. quickly ironic dolphins are carefully. regular theodo +9620|Supplier#000009620|MyadI08IbsyJeS59DpXFjDWIrR7g6WzDUOBFb|10|20-666-918-2616|52.64| even tithes boost blithely silent deposits; bold somas nag fluffily. slyly fi +9621|Supplier#000009621|gDrxt1WAuR5LHTrZB4L|11|21-306-998-9832|2623.25|final accounts are furiously around the requests. final packages affix blithely. carefully ev +9622|Supplier#000009622|nEfWHUKQ9VsiM1JdiwRi0P8oCcR|5|15-676-570-3786|9858.45|fully ironic courts haggle final packages. ideas +9623|Supplier#000009623|gzSBhAv86RXlqw4d2OArfp6YygO9qrHzL3MSlE4|1|11-796-392-8340|7067.64|uffily pending packages. ideas are quickly: per +9624|Supplier#000009624|yZMNk5VqyzFX88 r0LAl9oWx|6|16-341-828-5113|5806.20|l packages nag above the slyly pending +9625|Supplier#000009625|hx6phLn8,mPBhKs3m|6|16-706-357-3708|1880.17|refully carefully final theodolites. unusual deposits about the carefully final deposits boost +9626|Supplier#000009626|Nm1FnIh4asUR3EnXv2Pvy3gXqI9es|21|31-795-720-8984|5970.16|al dependencies wake bravely. dolphins poach thinly final dependencies. slyly busy orbits above +9627|Supplier#000009627|uhVxlR2ZdXa|1|11-459-329-1759|7425.47| instructions. fluffily regular realms sleep. furiously quick forges +9628|Supplier#000009628|W42Aqclp4Ov |19|29-745-342-4598|2881.79|ccounts haggle furiously bold packages. slyly even theodolites a +9629|Supplier#000009629|Gk,sFITaUyqNIpjnJe76YvHGkkOUNuM1fE9mWmE|10|20-424-554-3707|-832.79|e fluffily against the packages. carefully furious dugouts throu +9630|Supplier#000009630|7274mfBUihldBjVPROmLBMZbR2|20|30-677-856-8591|4210.77|of the even platelets sleep slyly after the regular, regular deposits. blithely iron +9631|Supplier#000009631|FJECow3o,khLY5f2sc|24|34-465-941-2862|5423.49| dependencies sleep furiously among the furiously regular pattern +9632|Supplier#000009632|4HMeDoqCq0ciYzdyyKNtz8kyTJcelWaSG32|14|24-450-231-6125|4684.10|against the regular accounts use slyly alongs +9633|Supplier#000009633|0gHBypLqJuZLoPKB8zjB9WyK|21|31-131-365-3836|7597.44|slyly final depths cajole slyly according to the regular, ironic pinto beans. +9634|Supplier#000009634|4sY25wAikcb9Ns UG3|14|24-110-493-3099|200.83|usly regular packages cajole slyly among the bold, bold requests. iron +9635|Supplier#000009635|UkKk7adjonMEy|5|15-218-564-4264|7218.80| ideas engage carefully slyly quiet ideas. regular, regular pinto +9636|Supplier#000009636|sNhSZshiwBtouuDrS|7|17-781-374-3572|5468.39|ts cajole carefully regular, even deposits +9637|Supplier#000009637|4,ZXKxiy0TJ|12|22-215-148-8784|213.82|nusual requests! carefully ironic pinto beans are slyly slyly final packages. quickly expr +9638|Supplier#000009638|giy0VurBGyQzPT|9|19-675-940-5244|5570.92| dependencies integrate quickly against the sp +9639|Supplier#000009639|s01Uzq,oShUyK3EGSNVlTBT439,B|16|26-240-990-3131|7513.27| packages. regular, regular instructions haggle blithely unusual theodolites. final courts are fl +9640|Supplier#000009640|YVi8zEFPwnWxDBHbAz|17|27-154-718-7001|4635.82|above the special, regular deposits. final ide +9641|Supplier#000009641|iycAANJYZaqs45tV XhuSNXVV7,fOe2v2GPH|24|34-829-846-8966|-810.17| furiously special requests impress furiously. quickly quick accounts boost qu +9642|Supplier#000009642|H6Mp3wghJzc61Rb|0|10-456-788-2625|-570.31|beans. furiously regular foxes hinder furiously. accounts nag accordin +9643|Supplier#000009643|KcAkQKPNPKu1Re|20|30-687-239-4722|4651.06|xcuses among the fluffily regular dependencies play idea +9644|Supplier#000009644|aNmbaoJ9cHH|1|11-481-644-1220|2435.78|es. furiously ironic pinto beans sleep carefully. furiously even deposits sleep tow +9645|Supplier#000009645|6CmMjDVLD5mzK5k19CaL|7|17-170-281-6088|5110.27|al deposits according to the furiously final asymptotes use carefully across +9646|Supplier#000009646|PNLwcL8HIKRjIx7|9|19-306-917-5939|9256.15|ges. fluffily regular packages sleep slyly on the furiously special foxes. furiously silent pin +9647|Supplier#000009647|b7w2eVKSZNma3ysmbeHMLt2,jYdhwfHFA9zlaF|20|30-635-807-4009|4143.18|o the blithely final platelets? furiously regular dolphins are: furiou +9648|Supplier#000009648|1dverSNf UlXPKteAbTV2Qq|20|30-861-462-1621|2428.28|ndencies sleep furiously quickly regular accounts. slyly regular foxe +9649|Supplier#000009649|1dx4XgRuG6aJRM4Cahp0Z1QFG5n|15|25-223-711-4173|144.15| among the ironic, unusual accounts ca +9650|Supplier#000009650|JXwPTk5ETBngKH 1gLIw8OVpfcm6wvmzE|14|24-143-474-6020|717.59|jole. carefully regular pin +9651|Supplier#000009651|wAez7p,lbPHySWeSDNRbfl4|24|34-829-936-1955|2477.97| instructions cajole carefully. furiously special accounts about the caref +9652|Supplier#000009652|RCT1dmIAXESWziBNOWqkMNngI7ThoBqZ|8|18-413-577-3110|-723.53|to beans: even requests shall have to wake around the carefully express packag +9653|Supplier#000009653|iZSb1O 0olEQYZjo4exXvOuQS91yHYMtpp|14|24-145-810-1602|3508.95|ly express packages for the quickly unusual frets wake blithel +9654|Supplier#000009654|5MNils0QhV2F3Sv,IfxOvuNBvssyh3|24|34-841-834-4591|1907.31|platelets cajole furiously carefully ironic pinto beans. pending hockey players about the car +9655|Supplier#000009655|,ZpiRbWDArIS7|18|28-199-383-8258|145.99|kages. packages was. silently careful packages sleep quickly. blithely ironic instructions integrate +9656|Supplier#000009656|vKPPjhrw6lnQmIvuAj70e|9|19-362-745-1503|9560.64|refully according to the +9657|Supplier#000009657|FotUbqfNjurHDJtawD02Cvgc|18|28-660-581-1006|6720.33| haggle fluffily among the regular, express tithes. even, ironic packages boost blithel +9658|Supplier#000009658|oE9uBgEfSS4opIcepXyAYM,x|19|29-748-876-2014|9624.78|ronic asymptotes wake bravely final +9659|Supplier#000009659|3hM5RwmOcAy|10|20-668-283-2499|6693.60|ometimes carefully thin foxes. quickly express packages above the blithely special asymptotes a +9660|Supplier#000009660|ex4DrVoJVSMmeHuypWIK1bG96XXnDmpo0F4|11|21-175-384-9834|304.74|out the final pinto beans. even, regular packages sleep blithely. slyly ironi +9661|Supplier#000009661|qftzAjWQj2JTyRDg5q3c|22|32-243-423-7966|4687.77|ests boost fluffily ironic deposits. carefully express asymptotes sleep blithely +9662|Supplier#000009662|,LewPPox5XnPT2eytXmopVIvM|9|19-926-402-3468|3394.35|ng the regular packages haggle alongside of the theodolites. slyly ironic accounts engage. re +9663|Supplier#000009663|VNN6byIi5Ad1LPgRo8JcqwLsB8kb6ajig5s|0|10-141-867-6340|866.01|ourts. excuses promise fluffily furiously bold platelets. fluffily unusual sentiments nag across +9664|Supplier#000009664|Esgu6FpdMOwbcR,c0wTKgJgsuVBwGfe|0|10-893-134-6539|19.99|y quickly ironic requests. quickly ironic theodolites nag. regular, pending pinto be +9665|Supplier#000009665|KhImEUr9ca2 S1X51nBeIyqBW6bd4AqtrvOUU4|6|16-939-721-8437|3947.61| the furiously express theodolites. busily final deposits sleep carefully above +9666|Supplier#000009666|UcRhPpMLLVceTnrbBkIrg|15|25-489-460-3462|4394.45|lar instructions according to the quickly even +9667|Supplier#000009667|B o GgdMLvkvba0eV9fMYglO2k2jlRnZ3Vi|11|21-650-347-7176|1459.70|ithely unusual accounts among the carefully even instructions are slyl +9668|Supplier#000009668|nntt4TuvakUBxhfNuFg3 sQ3d8|10|20-933-894-8052|1191.33|o beans. regular platelets above the ironic realms sleep furiously alongs +9669|Supplier#000009669|FR7KpOYuSAbzO |24|34-624-833-3429|614.97|lar packages. regular accoun +9670|Supplier#000009670|0gnjR7sFZm7uVVx|19|29-608-839-2901|4779.46|the pending requests. caref +9671|Supplier#000009671|BAKZ32zS2iE0d9K7EwS|5|15-683-792-3902|4385.02|efully express accounts wake blithely final theodolites. quickly expres +9672|Supplier#000009672|fupxQmsgDlv|18|28-641-826-1304|8069.39|ial accounts. furiously regular packages haggle according to the final a +9673|Supplier#000009673|vQNWXjmUyVpce2yMWmtQXCBML2r|14|24-343-921-5889|1357.22|n requests need to use slyly regular packages. quickly regular accounts wake fluffil +9674|Supplier#000009674|ByUyEOSDPbK3hpH80EevmJsMYzbcJ|4|14-474-925-8015|5680.62|into beans haggle slyly ironic foxes. pending requests wake about the unusual, sly dependenci +9675|Supplier#000009675|psKpKc78VgI,lDggxaDinguyMnF|4|14-839-438-2833|3191.44|n, even pinto beans wake carefully along the quickly ironic accounts +9676|Supplier#000009676|DveMRsJilUpIwD6zaJATQqhZUE4qggK5QJ430O|15|25-550-973-9295|518.49|try to cajole slowly fluffily final theodolites. +9677|Supplier#000009677|LKL37sM9RHu5j0138,jQCNnBjo1DM1Z5,|17|27-135-372-9596|268.76|ly final deposits impress carefully. furiously bold accounts are blithely blit +9678|Supplier#000009678|wQMgFCRlfs7KPO0liWtEhn7XMxkJM|24|34-651-991-4386|4955.88|ss, even instructions above the slyly thin ideas i +9679|Supplier#000009679|ueTQ9bUt0 iZfFRxiF8hFPwpU4PzVq4SUi8|12|22-592-240-3104|-504.16|uests after the slyly express accounts sleep carefully careful platelets. bold de +9680|Supplier#000009680|IKlL2Nkwu IqJV6kF OC TywoDxLDaBuavlXQBTF|8|18-956-814-3457|5471.93|ng instructions detect final account +9681|Supplier#000009681|212RJJdqbKy88htaeQJXptoqJaH|0|10-440-827-7540|4714.96|ckages wake slyly express deposits. even, final deposits sleep blithely after the fi +9682|Supplier#000009682|9lp047LVW3K,G1HG|5|15-435-370-7884|9335.95|ress instructions cajole carefully according to the carefully special +9683|Supplier#000009683|xqKTXr6qGoNbbjKel2ukWNeR|16|26-929-200-3310|3316.06|ns unwind carefully regular requests +9684|Supplier#000009684|LrxVt1hWxO24Mii|15|25-526-657-1906|5061.29| the final theodolites. special theodolites according to the fu +9685|Supplier#000009685|EUfFMAwEQdmFBC3fTQpltg|7|17-329-354-8238|27.34|ests sleep along the pending +9686|Supplier#000009686|MziS,BoJFgLZTV30P 9ugzgy q6oDwwlUgb1nU0|9|19-592-751-6799|6487.62|f the theodolites cajole furiously among the carefully express dependencies. quickly +9687|Supplier#000009687| Ik4 Ef8rGuF0ixhIfH0v9RgzEI|24|34-131-855-8498|3402.53|requests. express, regular packages cajole quickly. blithely regular excuses shall nag +9688|Supplier#000009688|ohakN polcEftV2GqPl|17|27-810-975-4861|6275.47|s. regular deposits should wake carefully. carefully unusual warh +9689|Supplier#000009689|ZAQw,vT3q8Ry MOyRSOulj|4|14-558-155-2781|1930.10|ic accounts nag against the carefully silent excuses. final accounts in +9690|Supplier#000009690|CB BnUTlmi5zdeEl7R7|23|33-121-267-9529|9201.47|e even, even foxes. blithely ironic packages cajole regular packages. slyly final ide +9691|Supplier#000009691|K IZD707NHR6Icrq3vpAiFRE75B|6|16-398-275-5960|9212.60|. quiet accounts cajole of +9692|Supplier#000009692|fUF1Z6esCDHpTzdmawL9Q|1|11-597-962-7139|9309.38|. furiously furious Tiresias wake. special packages cajo +9693|Supplier#000009693|,NtpOfxQv JMEv CJZjPvD,YFL|11|21-884-330-4663|7832.77|ly final instructions wake slyly according to the carefully express asymptotes. fluffily regul +9694|Supplier#000009694|BUWsBsNeRTp8vAEgKjiOGTsYcwW HtQ62mAkec3Y|7|17-189-183-6421|6373.69|aggle fluffily above the carefully express requests. blithely final accounts among the furiously +9695|Supplier#000009695|a5jfiVzGrKnNxceRfJXWjy0l7kSH,tsvLu3C|17|27-219-832-3236|2577.65|l packages after the silently express pinto beans sleep according to t +9696|Supplier#000009696|w4MKscCj,3U1kSA|15|25-922-978-3990|2451.32|ts nod slyly. furiously daring packages cajole blithely ironic theodolites. carefully fin +9697|Supplier#000009697|o2X 3GLhipvp1ReLO7EcBlz13MI|23|33-323-107-8373|2373.81|al requests: slyly regular foxes must detect carefully pending, eve +9698|Supplier#000009698|vAXg7cTjmOnY7Z|18|28-314-282-5827|8522.70|elets. furiously final reques +9699|Supplier#000009699|5UPSlkRQySRH9Guzku0mT|11|21-278-629-2954|8143.19|y above the slyly unusual asymptotes. quickly special theodolites nag fluffily. carefully fi +9700|Supplier#000009700| UTdFgLjpAb3|1|11-622-840-3751|9697.71| quickly even foxes. deposits around the regular theodolites use furiously blithely r +9701|Supplier#000009701|55EOMD3NDLStlMAIwjXZh7MCe|15|25-257-268-9350|464.05|s are quickly. fluffily regular foxes cajole blithely even theodolites. carefully regula +9702|Supplier#000009702|NGe7jpW9xfuW7RbiogsNa3ohqIEz29nPow|17|27-868-756-3613|9186.40|furiously among the fluffily regular asymptotes. quickly final requests engage. excuses +9703|Supplier#000009703|461W023tsamk5gRq9x U|17|27-841-617-4653|8967.41|nal deposits sleep blithely at the final theodolites. quickly final re +9704|Supplier#000009704|LoEsylAvTgQ90Jzi3kF5iPCSFDShUPjf 3Juu|19|29-281-504-6257|2321.38|kly even deposits sleep carefu +9705|Supplier#000009705|RUQ YXeE7mEogt|22|32-756-237-8275|7397.89| alongside of the carefully u +9706|Supplier#000009706|VHshTcwi8JFa2O2|14|24-269-405-2326|1465.27|even pinto beans hinder blithely carefully even requests. daring instructions use blith +9707|Supplier#000009707|Yi7VYA90iDUyiAfq3|7|17-541-100-1953|6742.73|packages detect blithely. furiously unusual requests alongside of the final instru +9708|Supplier#000009708|dSIM1ZOS2CiDIezYjHL|12|22-911-312-8577|1018.71| excuses. pinto beans serv +9709|Supplier#000009709|rRnCbHYgDgl9PZYnyWKVYSUW0vKg|3|13-417-841-9464|3457.97|ully unusual deposits along the slyly final pinto beans cajole against +9710|Supplier#000009710|TrgmM5M7RbLEAMMMrPmlo9BzrSNBR|12|22-378-617-8537|3067.30|t the accounts boost slyly according to the blithely unusual packages. carefu +9711|Supplier#000009711|Mk52BBqf4I17eHP7YhI8Q756Gr4aG9Irbx|11|21-848-779-8834|4121.62|against the slyly regular +9712|Supplier#000009712|bUoHY,NdYjKw4NL8PIZH RgM9oQ 0Dq0X|13|23-374-169-8901|3117.74|excuses affix blithely expre +9713|Supplier#000009713|RMZd04phxn6j9u|12|22-637-786-3479|4867.25|packages boost quickly pending pinto beans. pending instructions detect around +9714|Supplier#000009714|v9KJ,mGzvaEL9iAT 112ZJT4rKcOew|0|10-347-780-2647|5864.60|ecial theodolites sleep carefully. requests about the blithely bold requests boost furiously re +9715|Supplier#000009715|4uKqVHtYC40kYYp,YdLtQxd xSBIsHqdScD2RwR|21|31-490-601-7438|4324.87|ully permanent foxes use blithely busy, un +9716|Supplier#000009716|VXVt87Jc Eg2w8x8QGk,c3|18|28-427-307-2561|9008.68|ly unusual pinto beans. sometimes express foxes affix fluffily. bold deposits unwind fluffily r +9717|Supplier#000009717|YbtmKJqGaNn6OfiRw7PNSE1xkkhmUHILp78u31Z|13|23-182-320-2647|2965.36|ructions: ironic requests alongside of the regular +9718|Supplier#000009718|TJcCKxiE0v8hVqydfBW8EFuRqNDeKoD|3|13-817-490-5383|6861.64|onic ideas are blithely carefully pending deposits. deposits are blithely regular pa +9719|Supplier#000009719|BZ YzlLJU,CqlAhflWqzxr4jP|8|18-920-315-6826|8570.57| deposits boost carefully packages. t +9720|Supplier#000009720|leK82UIpE4GmuwkGpg3stx48r2WGr neoG|10|20-132-178-9010|9563.48|ses. carefully regular ide +9721|Supplier#000009721|FTLhkV3IX4USotx5rXJHymFYrvI,Xt3y|20|30-345-489-2347|-547.63|: deposits haggle slyly around the ironic, final d +9722|Supplier#000009722|3thnavdvDAOhdsGVYWyXY,W|15|25-982-411-3681|889.37|e furiously foxes. slow foxes +9723|Supplier#000009723|Q1CkkpDdlLOpCJiV,zIf,Mv86otWhxj7slGc |13|23-710-907-3873|7591.79|e fluffily even instructions. packages impress enticingly. +9724|Supplier#000009724|ICG8T,65SKiEHuEM7cuXDdb1s UACT|0|10-375-108-2181|4210.77|es: quickly final platelets use fluff +9725|Supplier#000009725|mxKnVSct0MMwrPJkYR aBxtslzCtdAo|12|22-258-423-2896|5407.31|es poach final, careful requests. furiously pending packages haggle +9726|Supplier#000009726|kmuggXTMhmZZt4nPS1umvzBY|17|27-516-198-2015|7663.95|! courts haggle furiously ironic, regular deposits. even foxes serve. ideas maintain slyly +9727|Supplier#000009727|jKjZMw6CMxw,qjGKZmlUZqP5oV|20|30-580-990-8437|7005.63|ly final accounts. furiously e +9728|Supplier#000009728|3KEvcbj09c10CIgjyMsURqzkOBYrxqiUaR2dA4|1|11-408-583-6730|8712.35|lithely express theodolites try to haggle unusual instructions. +9729|Supplier#000009729|7sXQXOGUiV5fqFULpGfLDDPeAuEnwnVJ7yQAD|16|26-307-939-5254|5456.16|ts after the final ideas affix furiously along the ironic ideas. si +9730|Supplier#000009730|8DQwd0yTyFHZAGJ,IJ2V|5|15-834-304-3803|3660.79|final, final somas. carefully dogged deposits nag furiously about the slyly ironi +9731|Supplier#000009731|HmHJZeVU1oft |8|18-727-593-9134|1925.55|ently special sentiments are enticingly. furiously express pack +9732|Supplier#000009732|KbR u5v2huzhHzc|0|10-510-815-1655|9678.68|s packages cajole across t +9733|Supplier#000009733|XIkUGlZFKq4IiZsAIRxFwzVBw7D|7|17-789-292-3060|-271.69|ions. boldly regular requests play furiously. furiously busy +9734|Supplier#000009734|d8BVmlRLNQSKaVA0vtse2gP2k3|23|33-359-235-4247|7690.91|ckey players are. evenly final accounts use. pendi +9735|Supplier#000009735|M1ejLn7K874MMc7Cp7xHwAj|15|25-920-872-8697|9107.53|p furiously requests. ironic requests boost carefully finally regular instruc +9736|Supplier#000009736| vGscTUEx,Ssr3vDDPtYTnmEyVRmNoVD|1|11-524-874-6454|3276.62|as are quickly. even dugouts against the busily pending dolphins are blithely about the iron +9737|Supplier#000009737|otDIDzmSrCXe1A6MTSBzkepuQjdWM0|1|11-322-896-9181|2214.73|y at the theodolites. even, ironic asymptotes cajole quickly ironic, ironic deposits. +9738|Supplier#000009738|15RRSVTuOzwdMP LmfCtIguMGXK|21|31-448-757-8378|7940.65|ckages haggle blithely slyly even accounts. care +9739|Supplier#000009739|k,4s66CcySj7J|12|22-767-176-8701|4551.61|uests. sometimes regular asymptotes use furiously +9740|Supplier#000009740|lDJbANGv2GuHxesULzNFf|4|14-292-957-5331|2836.15|nic platelets are platelets. carefully regular fox +9741|Supplier#000009741|K3bZ5BP,9aITan|9|19-822-261-7498|2206.26|e slyly about the regular packages +9742|Supplier#000009742|xs01ehPXrFQHZ,hDLFFT2653dHeMJzQa72fy|7|17-550-690-6902|8508.43|p furiously against the slyly expre +9743|Supplier#000009743|XbxgQyz5W4xXOKTfKoNjsMqZkOnG|13|23-135-741-1411|-759.41|ounts sleep carefully alongside +9744|Supplier#000009744|JwOGS,Zk gNI 2PE9LSfX7L|24|34-670-721-1147|5487.84|are bravely quickly bold ideas. furiously final theodolites sleep +9745|Supplier#000009745|sd1c0aMIhk6s4aThoGjs9fL OHClSTc5Eec|16|26-785-236-9117|8116.97|re carefully blithely pending ideas. ca +9746|Supplier#000009746|bnw7Yv0z3ZV8ku|15|25-701-923-3340|9030.49|lyly even, ironic instructi +9747|Supplier#000009747|jdub6FZMEJIwV3uO|19|29-910-833-4121|6505.25|nts are furiously. blithely unusual requests accordin +9748|Supplier#000009748|nVMMx1LjFnS3ADCfGlshld90Ekj8N n|7|17-607-105-1620|2764.64|rding to the blithely expre +9749|Supplier#000009749|Jv1ykSfKRNgmGM4gauJhQgkN2qdbCG1YHLXZabI|23|33-366-998-2159|-727.14|l Tiresias. final, pending asymptotes are qui +9750|Supplier#000009750|KZspEZRr9rc8,NaS|21|31-840-622-1678|8871.93| ironic, regular requests integrate above the caref +9751|Supplier#000009751|AFQ3U19Z1v|9|19-788-784-4046|3536.94|ding accounts. foxes sleep according to the never e +9752|Supplier#000009752|bTT59LszrEY|2|12-555-594-5972|-679.63|egular requests. unusual sheaves haggle carefully. fluff +9753|Supplier#000009753|wLhVEcRmd7PkJF4FBnGK7Z|3|13-724-256-7877|4406.93|nts nod carefully. silently +9754|Supplier#000009754|EeJyCPZ5qf0BLodby91xROEJPvfGA7ayI|0|10-841-520-6539|2242.59| express instructions. furiously unusual dolphins along the pinto beans snooze among the carefully u +9755|Supplier#000009755|lnSXCuqeKwR|0|10-523-874-6826|3144.30|tect furiously express pinto beans. furiously ironic deposits over the ironic dugouts kindle about +9756|Supplier#000009756|WgIcZYJP2W1bmBy0Pz84TIDPzljBLjKhqq,|18|28-964-897-9693|168.81|egular dolphins cajole blithely. slyly final ideas use by the unusual, bold ideas! blithely fi +9757|Supplier#000009757|XcD QT7rUA|4|14-597-813-8918|8113.22|aves dazzle slyly according to the +9758|Supplier#000009758|ammQN61k,nhUCxON5KO2DcjjwmrnmH6|20|30-626-624-5534|863.38|s. final, bold accounts haggle carefully final instruction +9759|Supplier#000009759|pm yLyvOS5bvpO1OZ4FwN262aLE21Hhjxv5|1|11-822-433-3967|-269.73|ecial instructions above the slyly stealthy attainments boost about the regular, regular d +9760|Supplier#000009760|pmtH2u1xfEjW5EEq9kjB8mMfqr1DCzfv|14|24-294-975-4599|7606.06| special requests. furiously ironic packages haggle bl +9761|Supplier#000009761|720mn AByioLe3YiA,nrybkVUX|5|15-466-674-2289|2163.14|nic pinto beans. blithely even deposits among the slyly regular pint +9762|Supplier#000009762|sszbnlE,8rTVV1n5PUnUaX|7|17-758-315-1893|9934.44|uriously unusual foxes. bold, ironic foxes according to the blithely final +9763|Supplier#000009763|5014E0NVyt1ktVX,8MCrtaGpDhl|14|24-527-145-4543|8149.65|unts nag courts. quickly ironic deposits doze blithely across the slyly e +9764|Supplier#000009764|pcQj5Q07Rq5iEGSHa8BBpMGFB|17|27-202-590-1209|8259.09|refully quickly pending foxes; quietly even requests use. fluffily expre +9765|Supplier#000009765|c5iTiLQekVj135nld6sJuujyk7, DXo0|23|33-187-453-7726|-491.39|riously about the slyly bold asymptotes. accounts nag quickly special deposits. regular depende +9766|Supplier#000009766|0gzxa,ZVKL7OMFTWwyW|12|22-375-740-4590|155.41|lose slyly bold deposits. final ideas sleep slyly above the carefully s +9767|Supplier#000009767|mpESLo1ZnAIbowUO5C|1|11-892-524-8624|2895.51|ithe accounts haggle quickly according to the fluffily final deposits. blithely silent acc +9768|Supplier#000009768|,1vdQhM1gDmOI09jy2XqgsUYcL1BSiG7|14|24-751-943-8632|9740.75|eep blithely among the quickly ironic r +9769|Supplier#000009769|Y0wv,9iqgEy7|5|15-701-408-9313|8851.02|ronic accounts. even request +9770|Supplier#000009770|Ag, SZfowit580QPDdbP8kmFHdpZ9ASI|21|31-571-664-8294|4204.08|e unusual packages. dependen +9771|Supplier#000009771|s38uMYcXku6|24|34-105-641-1996|3048.16| to the quickly bold packages. regular, ironic accounts use quickly against the pend +9772|Supplier#000009772|J7uGlVBUNacsmRXSc4, O4a|6|16-648-452-4698|1381.54|accounts. closely ironic packages use f +9773|Supplier#000009773|KvqGUov1WSL3k4Ir|5|15-563-235-4911|7531.62| detect. regular packages along the slyly bold instructions integrate carefull +9774|Supplier#000009774|nsZnXAk9fXQTVoI80xgG17AaZSjlnci|4|14-246-462-4723|4973.14|thogs after the blithely pending dependencies wake slyly after the carefully ironic foxe +9775|Supplier#000009775|30kUV46mOw91fKbZOLgxrss2UT E|18|28-925-275-7718|3603.76| packages. quickly regular requests among the quickly blithe foxes haggle about the even instr +9776|Supplier#000009776|,E0Fmh7eiNfAlTe4uxGHP4typvxn1Cmtn8oC|2|12-558-528-6307|386.66| ironic pinto beans above the furious +9777|Supplier#000009777|EdtUwT6BBgUVlx7Tqq35UG3f359UgC0|9|19-406-999-1389|6598.53|gular instructions nag blithely quickly final +9778|Supplier#000009778|TqQ10AxwOxCYgytM2wk|1|11-503-621-2157|8553.94| pending, special warthogs are quickly +9779|Supplier#000009779|qsk hPHPYBkfAqpVCqE|11|21-699-730-1839|5067.36|. carefully pending deposits haggle. slyly pending deposits wake slyly. thinly express requ +9780|Supplier#000009780|uukEUY7b0iw|13|23-963-126-8388|3150.73|furiously unusual packages cajole. +9781|Supplier#000009781|49NAd1iDiY4,|4|14-410-636-4312|9284.57|its. ironic pinto beans are blithely. express depths use caref +9782|Supplier#000009782|w z9a2x4sjwQVALB3QYQho0T,7o Gc,|16|26-477-762-5288|8922.27|ular deposits should are express foxes. furiously express excuses haggle busily. furiously +9783|Supplier#000009783|L9ji1BrzzHZHwIHVyZBz7O,Ng8f3g06WR|8|18-924-397-9573|2687.75|lly regular accounts cajole always slyly express pinto beans. pending deposits sleep +9784|Supplier#000009784|1Hc8s1MUWHwmRZ|16|26-434-843-8501|4954.45|ely furious asymptotes nag qui +9785|Supplier#000009785|G6Orhozp9rJ ll1u|7|17-474-990-7648|8390.01|haggle furiously regular packages. special accou +9786|Supplier#000009786|AEwr6sY4ryp7xOPZrpY4FYY|8|18-437-375-8170|-43.96|y carefully ironic pinto beans. quickly regular requests +9787|Supplier#000009787|h8du7pPO9dWD17btMv,x|20|30-554-219-3045|7140.99|deposits along the slyly unusual pinto b +9788|Supplier#000009788|m4eWogtlGCzvMfAtEiehqr030E5rAlZV|13|23-365-597-2009|6613.56| fluffily regular requests +9789|Supplier#000009789|3QdFPhczlVsQCcMII1MExj6U9OcGB|22|32-776-272-3122|8201.75|ainst the ironic ideas. deposits are carefully. deposits boost i +9790|Supplier#000009790|Qr8fuKN0LnMZ7w1|6|16-348-817-5382|3907.54|he pending packages. pending, pending pinto beans wake. ironic, bold foxes +9791|Supplier#000009791|rF3MHIIRT6YboDXZ|19|29-393-736-4035|1986.97|s. even, unusual accounts wake furiously special depths. blithely regular pinto beans do n +9792|Supplier#000009792|Y28ITVeYriT3kIGdV2K8fSZ V2UqT5H1Otz|7|17-988-938-4296|7887.08|ckly around the carefully fluffy theodolites. slyly ironic pack +9793|Supplier#000009793|ftjRWD5yKM8JNwUAEJCi9jAb,Pu5LVDrsKEq|11|21-292-156-2304|6166.73| frays after the fluffily even packages are blithely against the packages. carefully r +9794|Supplier#000009794|Iy0biqUnW8SEy0wUDtcJVDqWauFFgUarPO7PG5c|8|18-249-137-4735|184.63|d the permanently final courts. eve +9795|Supplier#000009795|8OTciWrTnRVbFAUUrqEg|13|23-628-902-1062|-998.22| furiously regular notornis unwind furiously. blithely regular pinto beans boo +9796|Supplier#000009796|z,y4Idmr15DOvPUqYG|3|13-582-882-6092|6780.44| haggle blithely. final, silent theodolites haggle fluffily in place of the carefull +9797|Supplier#000009797|ZYdHwgx98XwATW,oR,87,zuSb9|9|19-484-954-3272|7962.25|al pinto beans. silent ideas haggle carefully careful foxes. boldly even accounts wake carefully ac +9798|Supplier#000009798|VaBb,n0TkZBkB,r,UBDqK241uWZDcEb|5|15-283-773-6941|4546.91|sits are blithely pending accounts. dependencies across the eve +9799|Supplier#000009799| 4wNjXGa4OKWl|3|13-717-932-6533|880.29|deas affix carefully after the accounts. quickly +9800|Supplier#000009800|VFwKCiLwHI8u d4W8v 7T6sIZvphLeSRUDBvM|13|23-392-795-5960|8991.30|nusual deposits sleep final, special deposits. carefully regular packages nag even, regular do +9801|Supplier#000009801|loKjUMxr,JZC|12|22-293-385-2049|7796.18|ins affix. even ideas cajole furiously ironic theodolites. express theodolites promise quickly pendi +9802|Supplier#000009802|y6wLN KiZuTf5HT9Hbm0BELn1GUTD6yl|11|21-116-708-2013|7445.03|nic requests. pinto beans across the carefully regular grouches snooze among the final pin +9803|Supplier#000009803|GKARmvI6Npvph0Qm5WoqY3Q373gtzZT0xmh8fES|2|12-960-282-3252|1646.48|ly. slowly special packages detect slyly. final braids x-ray blithely slyly even deposits. slyly bo +9804|Supplier#000009804|VjyFXxtpo6Bbi6MB9ygEdD2TPO|20|30-586-486-6852|5284.30|ngside of the blithely ironic packages. ideas sl +9805|Supplier#000009805|dTTmLRYJNat,JbhlQlxwWp HjMR|11|21-838-243-3925|8195.44|lets. quickly even theodolites dazzle slyly even a +9806|Supplier#000009806|xKlLw13LtRL APZyOjSz0H0Nhclxi a60KW|13|23-165-182-4544|7862.14|ending instructions are. caref +9807|Supplier#000009807|FwJP993ZxQ|15|25-585-756-5943|5052.84|fily regular ideas: deposits across the b +9808|Supplier#000009808|XuqqRVaNwiJ anv5EKpXolaJc8AKrJyL|18|28-676-614-4258|1139.04|. furiously even foxes wake quickly against the regular pinto beans. silent warhorses nag +9809|Supplier#000009809|M,NoXai5TjhyvOH,R|17|27-231-166-2926|1789.80|y regular accounts affix slyly against th +9810|Supplier#000009810|vF,AdvbHMP vU8ARKeN34fNZWdrhV 3TWQ2n|1|11-888-435-7298|5112.90|structions sleep blithely final de +9811|Supplier#000009811|E3iuyq7UnZxU7oPZIe2Gu6|3|13-468-254-3805|9453.06|ular, regular instructions across the quickly expre +9812|Supplier#000009812|APFRMy3lCbgFga53n5t9DxzFPQPgnjrGt32|3|13-298-591-2357|2709.63|y final accounts dazzle quickly. express theodolites cajole carefu +9813|Supplier#000009813|Lv2yLejtYVUOvp|24|34-756-129-9896|9691.67|fully express deposits sleep. ironic, pending packages are. furiously quick epitaphs wa +9814|Supplier#000009814|af0O5pg83lPU4IDVmEylXZVqYZQzSDlYLAmR|19|29-995-571-8781|8996.14| dependencies boost quickly across the furiously pending requests! unusual dolphins play sl +9815|Supplier#000009815|qFvPqh2RBf5rbTxg0O,|20|30-248-440-7116|1205.36|even requests. boldly regular deposits sleep-- courts sublate. regular requests x- +9816|Supplier#000009816|OnrqELz5arn3HUQkTAv0B6zkDAPneH,S|19|29-661-606-7455|5415.82|ronic accounts cajole slyly a +9817|Supplier#000009817|MwQo7JKIcbIHc1T4dKI2qkb7UrBpIbTd|3|13-165-187-9639|7692.25|eposits. ironic, regular pinto beans sleep. slyly ironic requests sublate along the +9818|Supplier#000009818|6jPfNtL1KnMUnRZdI0,TfvsiGmJbSTM4SCzEA3L|0|10-612-146-1499|9760.75|ly special instructions. final, ir +9819|Supplier#000009819|gHrN6oD,5Bv139L4gTTnRPZWpQ2BKmnkIKHe|24|34-547-770-7019|3984.42|sly unusual instructions maintain. even requests haggle slyly near the blithely ironic pin +9820|Supplier#000009820|TNGtp51Xn2k6WhsV |21|31-514-215-1317|7174.08|er the pending ideas. fluffily even excuses haggle about the blithely +9821|Supplier#000009821| LWcfymaxdRf|2|12-228-906-8467|1822.60|he ironic foxes. blithely unusu +9822|Supplier#000009822|AUkDrXor8W|13|23-590-235-5159|6468.35|ses nag! closely special foxes use carefully. carefully regular escapades after the regular +9823|Supplier#000009823|DrAMnSiTsupxt3NlcEYFmLxcyZwrP7e3GApK2p|13|23-771-449-9542|9508.04|nding deposits sleep furiously about the slyly unusual requ +9824|Supplier#000009824|bOyLQpQE2vDP2TM6k7IPvsIrO7SiWMqN|19|29-600-465-5561|2624.73|theodolites. pinto beans +9825|Supplier#000009825|MtmjXZzwLYcF5Sdte8KYKp9pzF|10|20-610-765-9935|1780.93|n frays wake along the furiously unusual accounts. special, regu +9826|Supplier#000009826|6kzxRewM0Ab538JKz0|5|15-807-809-3623|-974.42|y across the furiously pending theodolites. foxes doze care +9827|Supplier#000009827|Wly7WXJFbTyZb30yfgZ4kIqrTF|23|33-427-591-9490|-226.87|s. final packages are against the packages! even, regula +9828|Supplier#000009828|eWld etKTGZqvR1igxQc0ZTKVfCxpR9Y3aEDn|17|27-853-353-5536|1909.22| the blithely regular packages. requests doubt furiously pending requests. t +9829|Supplier#000009829|a MzUAIbIxqhIuwTZN2w ysqZHoTXipJpWMo|24|34-526-678-9396|-785.38|al packages are blithely. blithely even instructions sleep quickly. blithely regular packages ab +9830|Supplier#000009830|ogbH63pS,VAoiGpDaMDRTSq7coA|9|19-230-592-9116|9094.47|ts. quickly pending packages detect carefully among the furiously regular instructions. ent +9831|Supplier#000009831|l6madwcmMnZkGWv0 e|19|29-635-483-3056|3839.69|furiously slyly dogged platelets. express requests cajole. blithely bold pinto beans a +9832|Supplier#000009832|pMhEmp70VMlpkPRF35oA,j2omIH|21|31-247-775-2786|5585.49| furiously furiously silent pinto bean +9833|Supplier#000009833|6ZxOK9QRt56jUE|13|23-840-522-2306|-690.66|riously ironic requests; express, pending deposits sleep daringly according to the ironic pa +9834|Supplier#000009834|PG,sk387oaQtgS1x655m7JLZfLdrH|15|25-258-791-7631|5979.26|final requests after the regular deposits boost regular pinto beans. pending r +9835|Supplier#000009835|MbMZ9KZ4B2GFPAtkTJESMS3mLs1CiMU1F5emUk|20|30-205-790-9745|6520.20|ges. furiously bold instructions cajo +9836|Supplier#000009836|vKVMlivaUCeWpMkzYafVd|22|32-457-649-5465|1039.23|. ironic, final instructions nag quickly. bold instructions use q +9837|Supplier#000009837|hQIOd6pW88L0S wNX FKZcldmLkYV6|16|26-259-904-4342|7361.18|ding packages after the quick +9838|Supplier#000009838|PhI5kjhtQBYwDpwmN |18|28-748-919-5438|442.73|nal instructions. slyly bold asymptotes +9839|Supplier#000009839|pyjNb8DFD7Xl|18|28-153-513-1311|2928.64|ts! final foxes solve slyly bold packages. ironic, bold packages wake blithely special +9840|Supplier#000009840|cneP2OiECwqb4dbfEb4kI4RdmIKbOds|5|15-987-639-3405|5692.57|sts behind the slyly unusual +9841|Supplier#000009841|6jbXloR1WIfLGyD137gH6jDFVJ1JXgwR3rU1|1|11-889-942-9631|6106.01|dencies are above the blithely unusua +9842|Supplier#000009842|SFSzJsFlLd,SU8hgxrm|8|18-593-121-4233|2794.51|latelets sleep furiously: carefully final asymptotes serve furiously up the id +9843|Supplier#000009843|EeRt5VTXbxQ41z9BapKaBgEU|13|23-923-281-4515|6252.64|carefully ironic requests sleep quickly carefully even foxes. +9844|Supplier#000009844|WTMchXIp1OXcJ8Qi|15|25-512-773-1922|3148.35|nusual accounts sleep blithely accounts. fina +9845|Supplier#000009845|5N29767EUnpB4ry NXkOEJ,C25jKgl4GSP|1|11-847-103-9663|2368.92|nal ideas. special requests could affix. excuses promise +9846|Supplier#000009846|57sNwJJ3PtBDu,hMPP5QvpcOcSNRXn3PypJJrh|3|13-388-969-1922|3472.08|ideas. blithely pending pinto beans sleep furiously past the c +9847|Supplier#000009847|tJDHVORZVvh9CgVB3|2|12-876-329-6513|9203.75|xcuses according to the pending +9848|Supplier#000009848|A,jN riqwo8sonCm61jIKhB5t16V6EF1|6|16-590-365-4635|3484.12|uriously pending accounts are silently. foxes are +9849|Supplier#000009849|ZXVOEhgwZ18xd85|8|18-241-969-1960|2564.13|s doubt quickly carefully final packages. carefully pending platelets are. carefully ironic depos +9850|Supplier#000009850|d4FjKHfQbjAcCb8KikyxR|20|30-844-242-6342|6261.89|ully ironic packages haggle. accou +9851|Supplier#000009851|Tqq93iG5UsYWC1kFNIuRMsfbXJX86HVCZn0|4|14-139-773-9129|4541.60|refully bold platelets cajole carefully final, ruthless packages. slyly specia +9852|Supplier#000009852|J 1Nz9NgWA0 uPIqjY5LVHLAHy|7|17-539-104-4483|-272.17|s are blithely alongside of the idly furious packages. ironically r +9853|Supplier#000009853|FQdvxIB2GcP6Z|2|12-595-826-2083|-362.47|egular, express asymptotes kin +9854|Supplier#000009854|DRGKS9JjAbYhTtN7bLB4rxOPs,Ia6KJoMDXQdg|7|17-873-530-1808|-65.69|ons wake carefully beneath the packages. carefully ironic pinto beans wake after the furiously fi +9855|Supplier#000009855|jK58xgC6rj0XEAN|20|30-298-120-9418|4866.45| furiously regular packages boost-- quickly unusual ideas +9856|Supplier#000009856|OGYDi21Su3MPG|5|15-590-600-1062|3477.08|the fluffily express ideas. fluffily ironic re +9857|Supplier#000009857|7gtWdfPIZTL0LMtKOhSOLyDAUOqWU97oT|7|17-860-910-8795|5256.75| blithely final deposits pla +9858|Supplier#000009858|HyYkus8tIiRgfYu8|8|18-616-616-6528|7235.12|its boost carefully regular deposits. carefully final packages are furiously. c +9859|Supplier#000009859|QR0E5htVvlb4qLK50|24|34-678-888-8655|4643.80| requests. ironic ideas wake against the fluf +9860|Supplier#000009860|pb99m4Ua9WbXcbAGQcOwdKxLAN5LGFLxYx|22|32-480-860-5922|1339.41|gly. fluffily regular deposits cajole blithely re +9861|Supplier#000009861|eGGB4oHunaB lYreIWG846uc9I|20|30-306-312-6601|3431.05| unusual deposits cajole. bold, regular +9862|Supplier#000009862|rJzweWeN58|3|13-653-546-4486|1642.51|ideas. furiously regular ideas affix against the carefully express pinto beans? regular requests h +9863|Supplier#000009863|kuoyUX809PXDD3D9LjAikJ5B3h|6|16-569-653-3211|4549.09|r packages. blithely regular excus +9864|Supplier#000009864|8uHZOeJqglR6upaX|23|33-682-554-7774|4553.62| accounts cajole blithely final excuses. silent, +9865|Supplier#000009865|extcOh9ZrdDCMsHhhsFTkTUAh,HM2UQ2qa8sRo|21|31-357-360-6063|3703.55| regular foxes are blithely. carefull +9866|Supplier#000009866|Auh6aZnOnQG1pPYKZ5o9ATramJBA|21|31-971-580-8950|5596.09|the fluffily brave theodolites. bold foxes +9867|Supplier#000009867|eWuVVObffkj xsIxj2O|2|12-553-972-8635|1421.41|luffily final orbits. dependencies in +9868|Supplier#000009868|ROjGgx5gvtkmnUUoeyy7v|3|13-616-429-1063|6595.71|ly above the slyly blithe packages. caref +9869|Supplier#000009869|ucLqxzrpBTRMewGSM29t0rNTM30g1Tu3Xgg3mKag|3|13-712-726-2032|5128.82|blithely even excuses around the +9870|Supplier#000009870|gW1ip1kGSdMGjfoj,8Vrf9ze1n1c|22|32-241-357-5627|2057.14|y players. pending, regular requests sleep after the carefully ironic theodolites. pending depos +9871|Supplier#000009871|xTocQE8vq8YkkBzCx7 z8b8|9|19-125-786-9743|-684.75|the fluffily sly epitaphs. carefully express packages affix +9872|Supplier#000009872|mRB haH7m nYrYHYWdUUF3WVwy7VFhGie4rse8|2|12-386-580-4248|6343.47|cajole ruthlessly. never express requests are boldly along the slyly unusual packages. quickly enti +9873|Supplier#000009873|Wimi5d8,sllO,5K8I|19|29-173-736-8712|669.96|kly. slyly brave packages acco +9874|Supplier#000009874| RdRSenOKZFnkp|11|21-169-160-7132|4902.64| platelets. daringly ironic requests h +9875|Supplier#000009875|9mlAzUnB8x7S8lpX,|1|11-948-749-5177|6669.44| beans sleep slyly blithely regular instructions. unusual, regular accounts wake. final +9876|Supplier#000009876|PFk3OQvPDIlJfNVOZc,|1|11-126-845-5361|1377.53|s haggle along the blithely regular dependencies. furiously even theodolites alongside of the +9877|Supplier#000009877|vo5NU9Dwzf6LbdRAz290f39Z56xJ|2|12-133-847-2414|6056.04|. foxes haggle! furiously ironic accounts boost furiously. +9878|Supplier#000009878|2kwsctKFEhB9inlp,hvlJ|16|26-864-214-4496|9856.55|quickly. unusual, final ideas nag slowly at the ironic accounts. slyly ironic instructions according +9879|Supplier#000009879|gpQDVtFStUZkG4jC9|10|20-938-342-4937|8181.68|the excuses. quickly regular accounts sleep blithely sly excus +9880|Supplier#000009880|CQzqP0YiUFIvgwHsVPbbq|6|16-486-273-8984|7627.42|structions nag quickly carefully daring requests. fluffily unusua +9881|Supplier#000009881|wBAZjc3w4ptPzXHi1xyjM2mddn|2|12-206-786-3708|9100.24|telets wake across the fluffily regular accounts. ca +9882|Supplier#000009882|toBXAWJs3dsv J|15|25-427-215-4592|1820.83| regular deposits. packages unwind quickly unusual instructions. fluffily eve +9883|Supplier#000009883|w8IytP4iV4uVGwAEy4HFZdysrv6,tykVZALiS0H|23|33-456-537-9468|9662.59|e according to the ironic packages. regular, bold accounts unwind. slyly e +9884|Supplier#000009884|wMaTkKJ9lFU8dJoSsmXdx|21|31-831-603-2305|4152.58|y even deposits. slyly permanent ideas haggle according to the fluffily bold foxes. +9885|Supplier#000009885|FKyOc lfEOB7CViGKURtSWiR|10|20-121-198-8026|6450.21|ly blithely final deposits. quickly bold excuses are furiously after the blithely regular ac +9886|Supplier#000009886|0pbnsB2fLaDoJY0LG0whSFvjISCI|16|26-528-380-8286|1626.11|out the regular, even ideas cajole blithely against +9887|Supplier#000009887|G1CNHwoABuHO1zBjLEKTpW7ZE3QXjU|5|15-832-242-2522|4065.02|ges cajole carefully! special, unusual tithes do poach always bold, iron +9888|Supplier#000009888|xTvFi75MqmFoMefzp0b|21|31-556-379-4280|8862.93|dencies. bold ideas use blithe +9889|Supplier#000009889|tpaG8SpuSZdTohqedcscLIx|15|25-396-991-3423|6321.90|ironic packages. carefully final foxes mu +9890|Supplier#000009890|izJXemCM Ikpgxk|21|31-951-623-4301|1119.65|instructions sleep. express the +9891|Supplier#000009891|I1olHN oxn4PHWUCytQ8kpKwsYLwZ11mpeR7|20|30-292-954-7155|5965.60|ly final accounts affix fluffily. regular pinto beans across the bold deposits wake +9892|Supplier#000009892|ZPxApGOpmemWejte71QARSdS|6|16-326-561-7103|7847.41| according to the deposits. pending grouches sleep furiously express requests. car +9893|Supplier#000009893|b85XhVuxBTTZTychS|0|10-364-259-8492|-522.37|tegrate quickly after the quickly regular requests. carefully ironic +9894|Supplier#000009894|pFIDobSjKG90Y8l zJbyu2VJQWWrbMgxilNqhn|15|25-180-456-5897|7463.94|c foxes use blithely carefully ironic accounts. asymptotes are slyly carefully pending instructi +9895|Supplier#000009895|FrlMSJxhOXbIG,XfqxJS|18|28-409-445-5933|498.30|ecial deposits cajole slyly even ideas. furious +9896|Supplier#000009896|Y q8F1vrW7i9FJ7evCYIyLRcgFe10wYQJ2|20|30-209-710-1983|1924.20|refully even excuses sleep +9897|Supplier#000009897|vv,stJWxADIU6NnC2oNC6uk7u9PEgkLd|10|20-811-736-1498|9031.40|s sleep slowly up the regular, even requests. special requests +9898|Supplier#000009898|H2TAKaf9w3,eEpJ,qrrgOP9ZiIw66|20|30-206-261-9917|-142.30| asymptotes. ironic packages nag blithely. slyly bol +9899|Supplier#000009899|7XdpAHrzr1t,UQFZE|3|13-200-446-4635|4235.99|print blithely. carefully final theodolites alongside of the blithely even +9900|Supplier#000009900|r5JV9d3esrDXCI|13|23-829-789-6887|341.04|ding packages. furiously ironic foxes boost. requests are furiously. ruthless, +9901|Supplier#000009901|8uN0zqb46 Bve|11|21-981-827-6340|9730.21|olphins. slyly regular platelets doubt slyly. +9902|Supplier#000009902|MM5vwVRcB4v, VbKSn84P|18|28-749-280-2809|7732.86|ins are according to the careful, regular foxes. even foxes haggle blit +9903|Supplier#000009903|2uwU9fLHYsZCGOe y95Buzgci0Vefh4RvjSC2Jfx|13|23-189-461-4618|-770.95|uriously. slyly ironic de +9904|Supplier#000009904|azEeVY7d1tqsytCwYg0Zqq6BSCg0uRW|1|11-504-889-1729|9537.43|after the furiously pending requests. bravely regular packages are carefully among the bold asy +9905|Supplier#000009905|1cZJO9Z2dPjlc7eX6ES2RQocyMVyf|9|19-632-326-5760|1637.14|as according to the slyly regular asymptotes detec +9906|Supplier#000009906|gYu, 3VzqrurtbsOpnt5,CQHy,hkRHOc|24|34-505-369-5501|2352.88|orges. slyly bold decoys use quietly across the unusual packages. furiously stealthy r +9907|Supplier#000009907|PsLwoo7rj23MGQVg0tTC|11|21-214-326-5231|8794.27|ong the slyly bold platelets. reg +9908|Supplier#000009908|YCdG0YZN6k8GLR4z8bD8rFI3JFOuchhSbJ q|12|22-339-638-5283|7293.48|nic platelets. thinly final foxes are along the even, special packa +9909|Supplier#000009909|fKWVWN3MtMmulZ,n|9|19-858-188-5399|6601.31|lyly even dependencies. blit +9910|Supplier#000009910|yEcU23vFDVQ1,Hc4sSrUGdDW|17|27-655-920-1663|282.17|slyly carefully regular theodolites. carefully regu +9911|Supplier#000009911|6SK4QgNHPMoCYs6hmGTvb5x,sU3k8HG17cW2R5|0|10-522-664-2103|-704.75| dolphins detect furiously ironic foxes. doggedly silent requests poach fluffily ironic packages. ca +9912|Supplier#000009912|pls50PhMjlOjUMVoGY05D|14|24-720-153-3561|110.30|slyly express ideas. special asymptotes serve b +9913|Supplier#000009913|Xv9xCmjx5N ms|0|10-913-665-6524|3724.69| after the dogged deposits. express, final courts wake. carefully even excuses cajole fur +9914|Supplier#000009914|0SpbbEPTOvEQ4fO|1|11-822-973-1163|8915.94|luffily even deposits. carefully even ideas are about the final dependencie +9915|Supplier#000009915|mhdhBkL2PL2Qs,Ny5FGPwciUPJggpK|8|18-682-553-2638|6252.10| haggle fluffily. blithely regular theodolites are. platelets wake. slyly express accounts al +9916|Supplier#000009916|Q2iGrNxzf3Q|13|23-476-236-9601|5151.60| use regular asymptotes. regular, bold requests wake slyly f +9917|Supplier#000009917|sBMs8OdxAXN1HQwfJ4YNpa9z4ZI3WD92qdIEvAKr|1|11-538-748-5875|-106.21|gular requests against the slyly enticing +9918|Supplier#000009918|CIoHYKkKdjMBH6|24|34-348-290-6027|5907.58| carefully. quickly regular foxes against the slyly ironic dependenc +9919|Supplier#000009919|MkrNYlbFA5Z,2C jSE4aEHPdcLoQ92oBPy 0|11|21-515-740-8455|696.51|sly. furiously unusual theodolites in place of the furiously regular pinto bean +9920|Supplier#000009920|INb4BeiKPkY1oI|0|10-214-567-7809|102.96|fter the requests. carefully silent accounts unwin +9921|Supplier#000009921|ZPxCI5ws CDbETjTqF603AQIlNsgh|2|12-986-966-3363|1345.24| ideas. slyly pending excuses wake slyly. regular, even braids doze slyly alongside of +9922|Supplier#000009922|2xl71djjzI5lnxhIPuQgnMuMc|9|19-516-130-6575|2405.59|s. quickly pending foxes integrate. furiously ironic +9923|Supplier#000009923|gk7PQ tkI8d roQRTYJy InZr2,X4okkbGX|23|33-231-694-4222|3679.52|regular, ironic instructions. special, pending requests eat +9924|Supplier#000009924|M0WQBYc3sqPmWcXBBxSBBhtp,WgSmRbyX|23|33-315-586-5453|8620.28| quickly regular theodolites sleep blithely i +9925|Supplier#000009925|1bFKWACLGwkA|4|14-172-754-5545|9027.15|heodolites affix. instructions use deposits. fluffily bold dolphins sleep slyly ironic accoun +9926|Supplier#000009926|iZRB7iQrJIbx2fY2aluTN3Y5Ah8|15|25-384-564-1345|5422.14|express accounts are slyly even deposits. ideas cajole special accounts. even accounts solve slyl +9927|Supplier#000009927|LMkskBnhkqzG7t11PVb60XK1 DF|8|18-723-814-4702|3892.44|iously final requests are fluffily after the slyly special requests. instructions i +9928|Supplier#000009928|cwQWbd1kve3|14|24-802-656-6504|1249.25| express deposits. carefully express accounts use. busily final deposits ab +9929|Supplier#000009929|iKDmtUon zqB8VybVsDgeVH,JF|0|10-140-262-3951|6094.53| accounts. even packages wake blithely. regularl +9930|Supplier#000009930|2 NaW1Xh eU1 ypRgGz3|12|22-376-168-3177|7733.32|y pending deposits are blithely packag +9931|Supplier#000009931|peQYiRFk G0xZKfJ|4|14-989-166-5782|6964.75|deposits according to the sometimes silent requests wake along the packages-- blithely f +9932|Supplier#000009932|ukUI0tgHCkHmjn37YBrsjLTim1ThRU02dgtr|4|14-556-799-9666|3591.84|ts play alongside of the furiously regular theodolites. accounts along t +9933|Supplier#000009933|NoqVo,uuCj2k|14|24-748-539-4361|9027.28|. slyly silent deposits lose carefully furiously regular ideas. +9934|Supplier#000009934|1Uc6NNlezVjNUaTe85dSc,JYb YhZ0ESW|7|17-163-337-5853|7481.67|y along the ironic requests. furiously unusual +9935|Supplier#000009935|GbU8jTE w4NOHU5ZkeBOmc XVKnz|15|25-154-657-6246|1456.01|h along the regular accounts. even, regular theodolites nag quickly slyly final deposits-- st +9936|Supplier#000009936|eNFpCM4CN5sDiOIKGTxT1|16|26-116-140-4412|5056.60|s nag. slyly pending accounts +9937|Supplier#000009937|edZ9HQJ0KJAU6EWknTiDghKfRLHq6vtFqdey,0l|21|31-531-982-7104|9252.69|n the regular, even requests. instructions along the ideas sleep final pack +9938|Supplier#000009938|9yJJaUT3ZP7FgKl Yw,I0k6E28eH|5|15-888-732-4995|1030.15|regular accounts cajole fluffily among the slyl +9939|Supplier#000009939|wFs,fIjp0vKa4|18|28-941-409-4935|8731.51|ns haggle carefully among the special pinto beans. even foxes kindle. +9940|Supplier#000009940|SUhqhESIjc|9|19-261-809-2619|8523.73|ross the furiously ironic asymptotes. ironic excuses above the pending dolph +9941|Supplier#000009941|nt5XPlHwYecqnkW|18|28-251-197-4432|3314.12|its haggle of the final dugouts. carefully ironic instructions boost furiously blithely thin +9942|Supplier#000009942| Jk0lKne38S4UN6aiEFuJW|12|22-683-412-5620|5368.76| ideas affix. carefully even fr +9943|Supplier#000009943|kthH3uOBsvd5jihNQSgsakbTiyMozuOfPZummva|10|20-405-661-5377|8695.65|express instructions. blithely express accounts cajole quickly accounts. ironic, final a +9944|Supplier#000009944|mVupJj3G4BpYI5ldojDgL|1|11-170-574-6579|-221.20|rets. express, bold excuses after the slyly regular ideas haggle above the unusual, fina +9945|Supplier#000009945|iq0VGL3Q1eKbFk27A|11|21-193-447-6870|9229.25|ounts believe slyly furiously unusual dinos! f +9946|Supplier#000009946|NRzShyy3ElxPtwgG 8SYQjmHpqMImdYV7e8|4|14-514-133-1173|3053.88|ependencies integrate. blithely even deposits are carefully among the requests. blithely +9947|Supplier#000009947|9xZDbMjIUQRgvE5dtZgiOvz1DB|4|14-628-127-7953|9486.11|blithely regular ideas. blithely regular Tiresias are accordin +9948|Supplier#000009948|w1PNn1DPWwTuSTYO,cNw3|1|11-573-223-6885|8113.37|ar theodolites sleep among the quickly unusu +9949|Supplier#000009949|AlVPaaSP3dB|23|33-216-624-1799|3509.09|thely final accounts wake furiously never unus +9950|Supplier#000009950|nD62ex9fraU78CRvRf6t,WEHj|13|23-587-598-5887|4342.63|requests. furiously regular theodolites haggle! blithely special dependencies are above the ironi +9951|Supplier#000009951|jnr 3C21ejBLgO5Mj7RoflkUFzC|17|27-804-979-6498|-503.55|sly final, even pinto beans. express, final instructions boost beneath th +9952|Supplier#000009952|NB6L,FgLmM CAreXQPOJjX,UGnTN6c QHrcRjy|22|32-898-398-3819|4757.54| ironic, regular pinto beans along the ideas dazzle carefully +9953|Supplier#000009953|ET85SH4iwBxacFCTFqwTt5KW8Ih,9|12|22-258-738-4585|9964.58| slyly express packages sleep furiously carefully regular deposits. unusual deposits +9954|Supplier#000009954|VzElx9ihlXFJLIQw2Hn4bC2|21|31-657-448-4812|8740.51|s sleep against the carefully regular packages. final instructions poach express, final sau +9955|Supplier#000009955|Em4LgL4i7,9krrhE 2yUrVxhqLSM|16|26-188-652-7749|5785.69| silent platelets. blithely ironic r +9956|Supplier#000009956|UjhY8PHXciKIqo6srufRLZP,p,t7N8|2|12-596-165-8425|9083.87|t theodolites are blithely carefully expr +9957|Supplier#000009957|QHkY,ei871xpBgSTjPjDyBee|23|33-206-683-9213|1663.99|packages haggle carefully +9958|Supplier#000009958|ggiiSA4CSyvhwQUYjdJhWlKEY9PAfs|21|31-196-571-6111|2698.36|uches haggle quickly. fluffily express d +9959|Supplier#000009959|HUAzfsJeRbMc1leYMY|23|33-638-887-9523|7385.05|ve to nag alongside of the bold, final instructions. dependencies boost. slyly fluffy accounts +9960|Supplier#000009960|pw2dYdX5dGYb2jWyRgz3IuuuxPXfmakaCRoTU|19|29-825-993-4837|5594.44|ly ironic foxes boost slyly above the ironic accounts. regul +9961|Supplier#000009961|mzznwxAlU97eIjL5Lj87oXPVmmJpANU|3|13-240-124-7463|6701.27|ar courts nag furiously. regular requests cajole +9962|Supplier#000009962| YlK6u3oLVeIPXQWyeQSUvNZVsCk,Z0D0,Bo|18|28-672-705-3701|8219.26|bold packages. slyly ironic escapades sleep blith +9963|Supplier#000009963|yms65U1v,uJoBIMrSyWYVb61wKFv,,lpd|14|24-279-658-5755|1194.19|ag special requests. carefully final ins +9964|Supplier#000009964|t5JNI1qn yf0LZWBnbLrOMvU82nc8OPOOAHTPF8|1|11-130-318-6328|4741.38| pinto beans integrate above the deposits. carefully even ideas snooze. +9965|Supplier#000009965|fV2VHGe8i,sU2KMw7SxUWjghGs0vi|9|19-509-772-1312|2364.73|ckages play along the carefully pending foxes. pe +9966|Supplier#000009966|f7UB YWpwxkecW40I07kFYA9Ytnwg|10|20-100-554-8091|9991.00|ts haggle quietly. quickly regular patterns cajole regular, bold pinto beans. excuses so +9967|Supplier#000009967|ulqn3UXU4UfbzeSRVoSd4k3Mc2V5t|5|15-289-544-5688|1814.23|unusual packages. bold requests wake quickly slyly ironic pinto beans. quic +9968|Supplier#000009968|q84VQRbUizJ|17|27-552-144-5491|8750.61|es unwind busily pending accounts. furiously ironic packages use furi +9969|Supplier#000009969|aGlh8upPi1lh8ZPbbZICU|12|22-391-956-5759|-477.42|sly even notornis. bold instructions sleep slyly. carefully regular instructions could have to use f +9970|Supplier#000009970|jAKeu0oDMMx7Pxwz|7|17-209-430-7778|3956.32|ckly final foxes: slow ideas use carefully. permanent ideas +9971|Supplier#000009971|pP5A68oYqRf9aHxF0XrjbVeHdwS|17|27-957-241-4796|7882.80|inal pinto beans. regular, final theodolites against the ironic requests h +9972|Supplier#000009972|Gf,aDlPGBtXHSXpdKR70TWzSela,Yw9hLl|3|13-842-720-1629|202.50|s. slyly stealthy requests are blithely. pending ins +9973|Supplier#000009973|8Tza,SSAjJe|5|15-416-747-5079|2331.95| ironic pinto beans are carefu +9974|Supplier#000009974|7wJ,J5DKcxSU4Kp1cQLpbcAvB5AsvKT|3|13-981-166-5707|9014.86|gular deposits wake regularly across the blithely ironi +9975|Supplier#000009975|Y2BWVNmh4TUgOEEUVnlwNdE|17|27-388-359-6916|7869.69|ic pinto beans across the furiously furious deposits maintain carefully across the ins +9976|Supplier#000009976|zLT6ZaLiz2Kgwh2I 6|0|10-132-638-2412|3.06|t the silent platelets. quickly final pinto beans w +9977|Supplier#000009977|8MBpF ,IjFsqejw2J,IoEZfTmlE7HdivNbYUBNS|18|28-169-166-1859|5370.62|otes. slyly unusual theodolites use fur +9978|Supplier#000009978|hoQmgkOOrsf0C0b8OF2dlhmQS3hwGqh|2|12-304-792-5255|7023.82|ithely among the bravely regular dependencies? carefully +9979|Supplier#000009979|Ote0EB9LmVAeCZHyK K|0|10-250-898-7175|6390.95|ts cajole furiously fluffy package +9980|Supplier#000009980|NcLNZ0sHf2k5GuCAhY3DsqigCjXrjBS|0|10-629-997-4692|6773.47|lly. regular dependencies according to the even, specia +9981|Supplier#000009981|NX1n8lXZ0ouphhFqAPKth58qURAu0XrXxxx|5|15-151-440-4215|666.96|iously final foxes wake slyly regular deposits. slyly ironic packages wake blithely dogged +9982|Supplier#000009982| t8wsPc8eVysqbkQaUd1cw6wC4YIbvPuWgZ|15|25-263-817-4575|9460.75| beans wake. ironic accounts against the blithely ironic sheaves kindle always fluffy +9983|Supplier#000009983|CQ1QB8x QKJqR8txuxa,6Cit a|18|28-691-438-9663|-159.67|uctions haggle. foxes haggle qui +9984|Supplier#000009984|rLkAcZoRIttWlBOXvI8qA|13|23-878-425-8047|9403.33|ffily unusual packages cajole against the fluffily final deposits. express, unusual packages ab +9985|Supplier#000009985|BZSKwKEjlX8JPr,rNVMzhjc5a|9|19-178-709-7161|2917.78|ly at the even deposits; slyly even packages detect furiously across the quickly ironic packa +9986|Supplier#000009986|miUqD2v,UJazIe|8|18-816-555-5359|-920.63| asymptotes nag slyly pinto beans. pending requests haggle slyly. pending deposits +9987|Supplier#000009987|uyeONdpYJFjU Xh6e1YVk,rbJlQczYul|12|22-168-108-7868|1950.70|ickly even attainments wake bli +9988|Supplier#000009988|7K7m3ynw9BpE0jaMNMe63D6RiU|0|10-413-557-3439|3430.57|s are furiously according to the final, sly foxe +9989|Supplier#000009989|OPo6ysXdxWAgTkgs,52rf6lO2ickEQ|0|10-115-204-7711|4536.26|ly fluffily regular forges. slyly express pinto beans are special orbits. blith +9990|Supplier#000009990|eo1f7McWNU9blo|7|17-606-745-1212|8969.64|ronic accounts. daringly pending pinto be +9991|Supplier#000009991|RnP1Z uvwftshFtf|13|23-451-948-8464|6785.10|. furiously pending accounts b +9992|Supplier#000009992|mngsMzPxdOOQGHAPFUH|5|15-467-813-3355|9242.91|even dinos wake among the slyly regular ideas. furiously ironic requests wake furiously quickl +9993|Supplier#000009993|vwhUkukD cuAjoCNRj,vw,jSFRY5zzfNLO|0|10-706-284-4813|1602.27|ctions hang slyly. furiously regular requests cajo +9994|Supplier#000009994|jx0yUkSCRhx6Iuvp19mTyXbrZw15ul|16|26-377-491-1391|-745.42|. ironic, ironic theodolites boost. even, final hockey players sle +9995|Supplier#000009995|Jooxc8G,ai win5mixYBKolLAODh0T|9|19-685-242-4442|9664.02|luffily final packages wake packages. special notornis sleep furious +9996|Supplier#000009996|a4eOd7SzN5RnrCwyAh5iey|10|20-898-443-4436|6209.67|s above the blithely even deposits play carefully special requests. blithely ironic depe +9997|Supplier#000009997|Wc0lgaT,CWQYMS|15|25-177-334-7328|7011.83|ve the furiously ironic platelets. evenly +9998|Supplier#000009998|1RTcQaCJzbx7GAjIc,tajct,8K|1|11-122-533-7674|2801.35|e regular excuses. blithely final pinto beans haggle? final requests haggle quickly quickl +9999|Supplier#000009999|mX37oAzqsBPhN1LWdzV p|9|19-773-990-5609|231.69|ounts cajole fluffily among the quickly ironic requests. special, regular +10000|Supplier#000010000|aTGLEusCiL4F PDBdv665XBJhPyCOB0i|19|29-578-432-2146|8968.42|ly regular foxes boost slyly. quickly special waters boost carefully ironi diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeWriteTableSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeWriteTableSuite.scala index ccf7bb5d5b2ab..99f946cd7dc86 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeWriteTableSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeWriteTableSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.delta.DeltaLog import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{DecimalType, LongType, StringType, StructField, StructType} import org.scalatest.BeforeAndAfterAll @@ -64,6 +65,7 @@ class GlutenClickHouseNativeWriteTableSuite // TODO: support default ANSI policy .set("spark.sql.storeAssignmentPolicy", "legacy") .set("spark.sql.warehouse.dir", getWarehouseDir) + .set("spark.gluten.sql.columnar.backend.ch.runtime_config.logger.level", "debug") .setMaster("local[1]") } @@ -91,7 +93,6 @@ class GlutenClickHouseNativeWriteTableSuite private val table_name_template = "hive_%s_test" private val table_name_vanilla_template = "hive_%s_test_written_by_vanilla" - private val formats = Array("orc", "parquet") override protected def afterAll(): Unit = { DeltaLog.clearCache() @@ -123,30 +124,9 @@ class GlutenClickHouseNativeWriteTableSuite import java.io.File - def writeIntoNewTableWithSql(table_name: String, table_create_sql: String)( - fields: Seq[String]): Unit = { - withDestinationTable(table_name, table_create_sql) { - checkNativeWrite( - s"insert overwrite $table_name select ${fields.mkString(",")}" + - s" from origin_table", - checkNative = true) - } - } - - def writeAndCheckRead( - table_name: String, - write: Seq[String] => Unit, - fields: Seq[String]): Unit = { - val originDF = spark.createDataFrame(genTestData()) - originDF.createOrReplaceTempView("origin_table") - - spark.sql(s"drop table IF EXISTS $table_name") - + def compareSource(original_table: String, table_name: String, fields: Seq[String]): Unit = { val rowsFromOriginTable = - spark.sql(s"select ${fields.mkString(",")} from origin_table").collect() - // write them to parquet table - write(fields) - + spark.sql(s"select ${fields.mkString(",")} from $original_table").collect() val dfFromWriteTable = spark.sql( s"select " + @@ -156,6 +136,17 @@ class GlutenClickHouseNativeWriteTableSuite s"from $table_name") checkAnswer(dfFromWriteTable, rowsFromOriginTable) } + def writeAndCheckRead( + original_table: String, + table_name: String, + fields: Seq[String], + checkNative: Boolean = true)(write: Seq[String] => Unit): Unit = + withDestinationTable(table_name) { + withNativeWriteCheck(checkNative) { + write(fields) + } + compareSource(original_table, table_name, fields) + } def recursiveListFiles(f: File): Array[File] = { val these = f.listFiles @@ -190,53 +181,94 @@ class GlutenClickHouseNativeWriteTableSuite ("date_field", "date") ) - def withDestinationTable(table: String, createTableSql: String)(f: => Unit): Unit = { - spark.sql(s"drop table IF EXISTS $table") - spark.sql(s"$createTableSql") - f - } - - def nativeWrite(f: String => Unit): Unit = { - withSQLConf(("spark.gluten.sql.native.writer.enabled", "true")) { - formats.foreach(f(_)) - } - } - def nativeWrite2( f: String => (String, String, String), - extraCheck: (String, String, String) => Unit = null): Unit = nativeWrite { + extraCheck: (String, String) => Unit = null, + checkNative: Boolean = true): Unit = nativeWrite { format => val (table_name, table_create_sql, insert_sql) = f(format) withDestinationTable(table_name, table_create_sql) { - checkNativeWrite(insert_sql, checkNative = true) - Option(extraCheck).foreach(_(table_name, table_create_sql, insert_sql)) + checkInsertQuery(insert_sql, checkNative) + Option(extraCheck).foreach(_(table_name, format)) } } - def nativeWriteWithOriginalView[A <: Product: TypeTag]( - data: Seq[A], - viewName: String, - pairs: (String, String)*)(f: String => Unit): Unit = { - val configs = pairs :+ ("spark.gluten.sql.native.writer.enabled", "true") - withSQLConf(configs: _*) { - withTempView(viewName) { - spark.createDataFrame(data).createOrReplaceTempView(viewName) - formats.foreach(f(_)) + def withSource[A <: Product: TypeTag](data: Seq[A], viewName: String, pairs: (String, String)*)( + block: => Unit): Unit = + withSource(spark.createDataFrame(data), viewName, pairs: _*)(block) + + private lazy val supplierSchema = StructType.apply( + Seq( + StructField.apply("s_suppkey", LongType, nullable = true), + StructField.apply("s_name", StringType, nullable = true), + StructField.apply("s_address", StringType, nullable = true), + StructField.apply("s_nationkey", LongType, nullable = true), + StructField.apply("s_phone", StringType, nullable = true), + StructField.apply("s_acctbal", DecimalType(15, 2), nullable = true), + StructField.apply("s_comment", StringType, nullable = true) + )) + + private def supplierDF = { + spark.read + .option("delimiter", "|") + .option("header", "false") + .schema(supplierSchema) + .csv(s"$rootPath/csv-data/supplier.csv") + .toDF() + } + + test("supplier: csv to parquet- insert overwrite local directory") { + withSource(supplierDF, "supplier") { + nativeWrite { + format => + val sql = + s"""insert overwrite local directory + |'$basePath/test_insert_into_${format}_supplier' + |stored as $format select * from supplier""".stripMargin + checkInsertQuery(sql, checkNative = true) + } + } + } + + test("supplier: csv to parquet- insert into one partition") { + val originViewName = "supplier" + lazy val create_columns = supplierSchema + .filterNot(f => f.name.equals("s_nationkey")) + .map(f => s"${f.name} ${f.dataType.catalogString}") + .mkString(",") + lazy val all_columns = supplierSchema + .filterNot(f => f.name.equals("s_nationkey")) + .map(f => s"${f.name}") + .mkString(",") + ", s_nationkey" + withSource(supplierDF, originViewName) { + nativeWrite2 { + format => + val table_name = s"supplier_$format" + val table_create_sql = + s"""create table if not exists $table_name + |($create_columns) + |partitioned by (s_nationkey bigint) stored as $format""".stripMargin + val insert_sql = + s"""insert into $table_name + |select $all_columns from $originViewName""".stripMargin + (table_name, table_create_sql, insert_sql) } } } test("test insert into dir") { - nativeWriteWithOriginalView(genTestData(), "origin_table") { - format => - Seq( - s"""insert overwrite local directory '$basePath/test_insert_into_${format}_dir1' - |stored as $format select ${fields_.keys.mkString(",")} - |from origin_table""".stripMargin, - s"""insert overwrite local directory '$basePath/test_insert_into_${format}_dir2' - |stored as $format select string_field, sum(int_field) as x - |from origin_table group by string_field""".stripMargin - ).foreach(checkNativeWrite(_, checkNative = true)) + withSource(genTestData(), "origin_table") { + nativeWrite { + format => + Seq( + s"""insert overwrite local directory '$basePath/test_insert_into_${format}_dir1' + |stored as $format select ${fields_.keys.mkString(",")} + |from origin_table""".stripMargin, + s"""insert overwrite local directory '$basePath/test_insert_into_${format}_dir2' + |stored as $format select string_field, sum(int_field) as x + |from origin_table group by string_field""".stripMargin + ).foreach(checkInsertQuery(_, checkNative = true)) + } } } @@ -253,54 +285,52 @@ class GlutenClickHouseNativeWriteTableSuite (table_name, table_create_sql, insert_sql) } - def nativeFormatWrite(format: String): Unit = { - val (table_name, table_create_sql, insert_sql) = destination(format) - withDestinationTable(table_name, table_create_sql) { - checkNativeWrite(insert_sql, checkNative = true) - var files = recursiveListFiles(new File(getWarehouseDir + "/" + table_name)) - .filter(_.getName.endsWith(s".$format")) - if (format == "orc") { - files = files.filter(_.getName.contains(".lz4")) + withSource(genTestData(), "origin_table", ("spark.sql.orc.compression.codec", "lz4")) { + nativeWrite2( + format => destination(format), + (table_name, format) => { + var files = recursiveListFiles(new File(getWarehouseDir + "/" + table_name)) + .filter(_.getName.endsWith(s".$format")) + if (format == "orc") { + files = files.filter(_.getName.contains(".lz4")) + } + assertResult(1)(files.length) + assert(files.head.getAbsolutePath.contains("another_date_field=2020-01-01")) } - assert(files.length == 1) - assert(files.head.getAbsolutePath.contains("another_date_field=2020-01-01")) - } + ) } - - nativeWriteWithOriginalView( - genTestData(), - "origin_table", - ("spark.sql.orc.compression.codec", "lz4"))(nativeFormatWrite) } test("test CTAS") { - nativeWriteWithOriginalView(genTestData(), "origin_table") { - format => - val table_name = table_name_template.format(format) - val table_create_sql = - s"create table $table_name using $format as select " + - fields_ - .map(f => s"${f._1}") - .mkString(",") + - " from origin_table" - val insert_sql = - s"create table $table_name as select " + - fields_ - .map(f => s"${f._1}") - .mkString(",") + - " from origin_table" - withDestinationTable(table_name, table_create_sql) { - spark.sql(s"drop table IF EXISTS $table_name") - - try { - // FIXME: using checkNativeWrite - spark.sql(insert_sql) - } catch { - case _: UnsupportedOperationException => // expected - case e: Exception => fail("should not throw exception", e) + withSource(genTestData(), "origin_table") { + nativeWrite { + format => + val table_name = table_name_template.format(format) + val ctas_support_sql = + s"create table $table_name using $format as select " + + fields_ + .map(f => s"${f._1}") + .mkString(",") + + " from origin_table" + val ctas_not_support_sql = + s"create table $table_name as select " + + fields_ + .map(f => s"${f._1}") + .mkString(",") + + " from origin_table" + withDestinationTable(table_name) { + checkInsertQuery(ctas_support_sql, checkNative = true) } - } - + withDestinationTable(table_name) { + try { + // hive format without format name not support + checkInsertQuery(ctas_not_support_sql, checkNative = false) + } catch { + case _: UnsupportedOperationException => // expected + case e: Exception => fail("should not throw exception", e) + } + } + } } } @@ -320,28 +350,29 @@ class GlutenClickHouseNativeWriteTableSuite (table_name, table_create_sql, insert_sql) } - def nativeFormatWrite(format: String): Unit = { - val (table_name, table_create_sql, insert_sql) = destination(format) - withDestinationTable(table_name, table_create_sql) { - checkNativeWrite(insert_sql, checkNative = true) - val files = recursiveListFiles(new File(getWarehouseDir + "/" + table_name)) - .filter(_.getName.startsWith("part")) - assert(files.length == 1) - assert(files.head.getAbsolutePath.contains("another_date_field=2020-01-01")) - } - } - - nativeWriteWithOriginalView( + withSource( genTestData(), "origin_table", ("spark.sql.hive.convertMetastoreParquet", "false"), - ("spark.sql.hive.convertMetastoreOrc", "false"))(nativeFormatWrite) + ("spark.sql.hive.convertMetastoreOrc", "false")) { + nativeWrite2( + format => destination(format), + (table_name, format) => { + val files = recursiveListFiles(new File(getWarehouseDir + "/" + table_name)) + .filter(_.getName.startsWith("part")) + assertResult(1)(files.length) + assert(files.head.getAbsolutePath.contains("another_date_field=2020-01-01")) + }, + isSparkVersionLE("3.3") + ) + } } test("test 1-col partitioned table") { - nativeWrite { - format => - { + val origin_table = "origin_table" + withSource(genTestData(), origin_table) { + nativeWrite2( + format => { val table_name = table_name_template.format(format) val table_create_sql = s"create table if not exists $table_name (" + @@ -351,38 +382,35 @@ class GlutenClickHouseNativeWriteTableSuite .mkString(",") + " ) partitioned by (date_field date) " + s"stored as $format" - - writeAndCheckRead( - table_name, - writeIntoNewTableWithSql(table_name, table_create_sql), - fields_.keys.toSeq) - } + val insert_sql = + s"""insert overwrite $table_name select ${fields_.keys.toSeq.mkString(",")} + |from $origin_table""".stripMargin + (table_name, table_create_sql, insert_sql) + }) } } - // even if disable native writer, this UT fail, spark bug??? - ignore("test 1-col partitioned table, partitioned by already ordered column") { - withSQLConf(("spark.gluten.sql.native.writer.enabled", "false")) { - val originDF = spark.createDataFrame(genTestData()) - originDF.createOrReplaceTempView("origin_table") - - for (format <- formats) { - val table_name = table_name_template.format(format) - val table_create_sql = - s"create table if not exists $table_name (" + - fields_ - .filterNot(e => e._1.equals("date_field")) - .map(f => s"${f._1} ${f._2}") - .mkString(",") + - " ) partitioned by (date_field date) " + - s"stored as $format" - - spark.sql(s"drop table IF EXISTS $table_name") - spark.sql(table_create_sql) - spark.sql( - s"insert overwrite $table_name select ${fields_.mkString(",")}" + - s" from origin_table order by date_field") - } + test("test 1-col partitioned table, partitioned by already ordered column") { + val origin_table = "origin_table" + def destination(format: String): (String, String, String) = { + val table_name = table_name_template.format(format) + val table_create_sql = + s"create table if not exists $table_name (" + + fields_ + .filterNot(e => e._1.equals("date_field")) + .map(f => s"${f._1} ${f._2}") + .mkString(",") + + " ) partitioned by (date_field date) " + + s"stored as $format" + val insert_sql = + s"""insert overwrite $table_name select ${fields_.keys.mkString(",")} + |from $origin_table order by date_field""".stripMargin + (table_name, table_create_sql, insert_sql) + } + def check(table_name: String, format: String): Unit = + compareSource(origin_table, table_name, fields_.keys.toSeq) + withSource(genTestData(), origin_table) { + nativeWrite2(destination, check) } } @@ -399,63 +427,68 @@ class GlutenClickHouseNativeWriteTableSuite ("date_field", "date"), ("byte_field", "byte") ) - nativeWrite { - format => - val table_name = table_name_template.format(format) - val table_create_sql = - s"create table if not exists $table_name (" + - fields - .filterNot(e => e._1.equals("date_field") || e._1.equals("byte_field")) - .map(f => s"${f._1} ${f._2}") - .mkString(",") + " ) partitioned by (date_field date, byte_field byte) " + - s"stored as $format" - - writeAndCheckRead( - table_name, - writeIntoNewTableWithSql(table_name, table_create_sql), - fields.keys.toSeq) + val origin_table = "origin_table" + def destination(format: String): (String, String, String) = { + val table_name = table_name_template.format(format) + val table_create_sql = + s"create table if not exists $table_name (" + + fields + .filterNot(e => e._1.equals("date_field") || e._1.equals("byte_field")) + .map(f => s"${f._1} ${f._2}") + .mkString(",") + " ) partitioned by (date_field date, byte_field byte) " + + s"stored as $format" + val insert_sql = + s"""insert overwrite $table_name select ${fields.keys.mkString(",")} + |from $origin_table order by date_field""".stripMargin + (table_name, table_create_sql, insert_sql) + } + def check(table_name: String, format: String): Unit = + compareSource(origin_table, table_name, fields.keys.toSeq) + withSource(genTestData(), origin_table) { + nativeWrite2(destination, check) } } ignore( "test hive parquet/orc table, all types of columns being partitioned except the date_field," + " ignore because takes too long") { - withSQLConf( - ("spark.gluten.sql.native.writer.enabled", "true"), - (GlutenConfig.GLUTEN_ENABLED.key, "true")) { - - val fields: ListMap[String, String] = ListMap( - ("date_field", "date"), - ("timestamp_field", "timestamp"), - ("string_field", "string"), - ("int_field", "int"), - ("long_field", "long"), - ("float_field", "float"), - ("double_field", "double"), - ("short_field", "short"), - ("byte_field", "byte"), - ("boolean_field", "boolean"), - ("decimal_field", "decimal(23,12)") - ) - for (format <- formats) { - val table_name = table_name_template.format(format) - for (field <- fields.filterNot(e => e._1.equals("date_field"))) { - spark.sql(s"drop table if exists $table_name") - val table_create_sql = - s"create table if not exists $table_name (" + - " date_field date" + " ) partitioned by (" + - field._1 + " " + field._2 + - ") " + - s"stored as $format" + val fields: ListMap[String, String] = ListMap( + ("date_field", "date"), + ("timestamp_field", "timestamp"), + ("string_field", "string"), + ("int_field", "int"), + ("long_field", "long"), + ("float_field", "float"), + ("double_field", "double"), + ("short_field", "short"), + ("byte_field", "byte"), + ("boolean_field", "boolean"), + ("decimal_field", "decimal(23,12)") + ) - writeAndCheckRead( - table_name, - writeIntoNewTableWithSql(table_name, table_create_sql), - List("date_field", field._1)) - } + val origin_table = "origin_table" + def destination(format: String, field: (String, String)): (String, String, String) = { + val table_name = table_name_template.format(format) + val table_create_sql = + s"create table if not exists $table_name (" + + " date_field date" + " ) partitioned by (" + + field._1 + " " + field._2 + ") " + + s"stored as $format" + val insert_sql = + s"""insert overwrite $table_name + |select ${List("date_field", field._1).mkString(",")} from $origin_table""".stripMargin + (table_name, table_create_sql, insert_sql) + } + def check(table_name: String, format: String, field: (String, String)): Unit = + compareSource(origin_table, table_name, List("date_field", field._1)) + + withSource(genTestData(), origin_table) { + for (field <- fields.filterNot(e => e._1.equals("date_field"))) { + nativeWrite2( + format => destination(format, field), + (table_name, format) => check(table_name, format, field)) } - } } @@ -474,23 +507,27 @@ class GlutenClickHouseNativeWriteTableSuite ("boolean_field", "boolean"), ("decimal_field", "decimal(23,12)") ) - nativeWrite { - format => - val table_name = table_name_template.format(format) - val table_create_sql = - s"create table if not exists $table_name (" + - " date_field date" + " ) partitioned by (" + - fields - .filterNot(e => e._1.equals("date_field")) - .map(f => s"${f._1} ${f._2}") - .mkString(",") + - ") " + - s"stored as $format" - - writeAndCheckRead( - table_name, - writeIntoNewTableWithSql(table_name, table_create_sql), - fields.keys.toSeq) + val origin_table = "origin_table" + def destination(format: String): (String, String, String) = { + val table_name = table_name_template.format(format) + val table_create_sql = + s"create table if not exists $table_name (" + + " date_field date" + " ) partitioned by (" + + fields + .filterNot(e => e._1.equals("date_field")) + .map(f => s"${f._1} ${f._2}") + .mkString(",") + + ") " + + s"stored as $format" + val insert_sql = + s"""insert overwrite $table_name select ${fields.keys.mkString(",")} + |from $origin_table order by date_field""".stripMargin + (table_name, table_create_sql, insert_sql) + } + def check(table_name: String, format: String): Unit = + compareSource(origin_table, table_name, fields.keys.toSeq) + withSource(genTestData(), origin_table) { + nativeWrite2(destination, check) } } @@ -498,53 +535,56 @@ class GlutenClickHouseNativeWriteTableSuite val fields: ListMap[String, String] = ListMap( ("sum(int_field)", "bigint") ) - nativeWrite { - format => - val table_name = table_name_template.format(format) - val table_create_sql = - s"create table if not exists $table_name (" + - fields - .map(f => s"${getColumnName(f._1)} ${f._2}") - .mkString(",") + - s" ) stored as $format" - - writeAndCheckRead( - table_name, - writeIntoNewTableWithSql(table_name, table_create_sql), - fields.keys.toSeq) + val origin_table = "origin_table" + def destination(format: String): (String, String, String) = { + val table_name = table_name_template.format(format) + val table_create_sql = + s"create table if not exists $table_name (" + + fields + .map(f => s"${getColumnName(f._1)} ${f._2}") + .mkString(",") + + s" ) stored as $format" + val insert_sql = + s"insert overwrite $table_name select ${fields.keys.toSeq.mkString(",")} from $origin_table" + (table_name, table_create_sql, insert_sql) + } + def check(table_name: String, format: String): Unit = + compareSource(origin_table, table_name, fields.keys.toSeq) + withSource(genTestData(), origin_table) { + nativeWrite2(destination, check) } } test("test 1-col partitioned + 1-col bucketed table") { - nativeWrite { - format => - // spark write does not support bucketed table - // https://issues.apache.org/jira/browse/SPARK-19256 - val table_name = table_name_template.format(format) - writeAndCheckRead( - table_name, - fields => { - spark - .table("origin_table") - .select(fields.head, fields.tail: _*) - .write - .format(format) - .partitionBy("date_field") - .bucketBy(2, "byte_field") - .saveAsTable(table_name) - }, - fields_.keys.toSeq - ) + val origin_table = "origin_table" + withSource(genTestData(), origin_table) { + nativeWrite { + format => + // spark write does not support bucketed table + // https://issues.apache.org/jira/browse/SPARK-19256 + val table_name = table_name_template.format(format) + writeAndCheckRead(origin_table, table_name, fields_.keys.toSeq, isSparkVersionLE("3.3")) { + fields => + spark + .table("origin_table") + .select(fields.head, fields.tail: _*) + .write + .format(format) + .partitionBy("date_field") + .bucketBy(2, "byte_field") + .saveAsTable(table_name) + } - assert( - new File(getWarehouseDir + "/" + table_name) - .listFiles() - .filter(_.isDirectory) - .filter(!_.getName.equals("date_field=__HIVE_DEFAULT_PARTITION__")) - .head - .listFiles() - .count(!_.isHidden) == 2 - ) // 2 bucket files + assertResult(2)( + new File(getWarehouseDir + "/" + table_name) + .listFiles() + .filter(_.isDirectory) + .filter(!_.getName.equals("date_field=__HIVE_DEFAULT_PARTITION__")) + .head + .listFiles() + .count(!_.isHidden) + ) // 2 bucket files + } } } @@ -562,48 +602,55 @@ class GlutenClickHouseNativeWriteTableSuite ("date_field", "date"), ("timestamp_field", "timestamp") ) - - for (format <- formats) { - val table_name = table_name_template.format(format) - withSQLConf(("spark.gluten.sql.native.writer.enabled", "true")) { - writeAndCheckRead( - table_name, - fields => { - spark - .table("origin_table") - .select(fields.head, fields.tail: _*) - .write - .format(format) - .bucketBy(10, fields.head, fields.tail: _*) - .saveAsTable(table_name) - }, - fields.keys.toSeq - ) + def excludeTimeFieldForORC(format: String): Seq[String] = { + if (format.equals("orc") && isSparkVersionGE("3.4")) { + // FIXME:https://github.com/apache/incubator-gluten/pull/6507 + fields.keys.filterNot(_.equals("timestamp_field")).toSeq + } else { + fields.keys.toSeq } - - val table_name_vanilla = table_name_vanilla_template.format(format) - withSQLConf(("spark.gluten.sql.native.writer.enabled", "true")) { - spark.sql(s"drop table IF EXISTS $table_name_vanilla") - spark - .table("origin_table") - .select(fields.keys.toSeq.head, fields.keys.toSeq.tail: _*) - .write - .format(format) - .bucketBy(10, fields.keys.toSeq.head, fields.keys.toSeq.tail: _*) - .saveAsTable(table_name_vanilla) - - val sigsOfNativeWriter = - getSignature( - format, - recursiveListFiles(new File(getWarehouseDir + "/" + table_name)) - .filter(_.getName.endsWith(s".$format"))).sorted - val sigsOfVanillaWriter = - getSignature( - format, - recursiveListFiles(new File(getWarehouseDir + "/" + table_name_vanilla)) - .filter(_.getName.endsWith(s".$format"))).sorted - - assert(sigsOfVanillaWriter.sameElements(sigsOfNativeWriter)) + } + val origin_table = "origin_table" + withSource(genTestData(), origin_table) { + nativeWrite { + format => + val table_name = table_name_template.format(format) + val testFields = excludeTimeFieldForORC(format) + writeAndCheckRead(origin_table, table_name, testFields, isSparkVersionLE("3.3")) { + fields => + spark + .table(origin_table) + .select(fields.head, fields.tail: _*) + .write + .format(format) + .bucketBy(10, fields.head, fields.tail: _*) + .saveAsTable(table_name) + } + val table_name_vanilla = table_name_vanilla_template.format(format) + spark.sql(s"drop table IF EXISTS $table_name_vanilla") + withSQLConf(("spark.gluten.sql.native.writer.enabled", "false")) { + withNativeWriteCheck(checkNative = false) { + spark + .table("origin_table") + .select(testFields.head, testFields.tail: _*) + .write + .format(format) + .bucketBy(10, testFields.head, testFields.tail: _*) + .saveAsTable(table_name_vanilla) + } + } + val sigsOfNativeWriter = + getSignature( + format, + recursiveListFiles(new File(getWarehouseDir + "/" + table_name)) + .filter(_.getName.endsWith(s".$format"))).sorted + val sigsOfVanillaWriter = + getSignature( + format, + recursiveListFiles(new File(getWarehouseDir + "/" + table_name_vanilla)) + .filter(_.getName.endsWith(s".$format"))).sorted + + assertResult(sigsOfVanillaWriter)(sigsOfNativeWriter) } } } @@ -624,49 +671,49 @@ class GlutenClickHouseNativeWriteTableSuite ("map", "map") ) - for (format <- formats) { - val table_name = table_name_template.format(format) - withSQLConf(("spark.gluten.sql.native.writer.enabled", "true")) { - writeAndCheckRead( - table_name, - fields => { - spark - .table("origin_table") - .select(fields.head, fields.tail: _*) - .write - .format(format) - .partitionBy("date_field") - .bucketBy(10, "byte_field", "string_field") - .saveAsTable(table_name) - }, - fields.keys.toSeq - ) - } + val origin_table = "origin_table" + withSource(genTestData(), origin_table) { + nativeWrite { + format => + val table_name = table_name_template.format(format) + writeAndCheckRead(origin_table, table_name, fields.keys.toSeq, isSparkVersionLE("3.3")) { + fields => + spark + .table("origin_table") + .select(fields.head, fields.tail: _*) + .write + .format(format) + .partitionBy("date_field") + .bucketBy(10, "byte_field", "string_field") + .saveAsTable(table_name) + } - val table_name_vanilla = table_name_vanilla_template.format(format) - withSQLConf(("spark.gluten.sql.native.writer.enabled", "true")) { - spark.sql(s"drop table IF EXISTS $table_name_vanilla") - spark - .table("origin_table") - .select(fields.keys.toSeq.head, fields.keys.toSeq.tail: _*) - .write - .format(format) - .partitionBy("date_field") - .bucketBy(10, "byte_field", "string_field") - .saveAsTable(table_name_vanilla) - - val sigsOfNativeWriter = - getSignature( - format, - recursiveListFiles(new File(getWarehouseDir + "/" + table_name)) - .filter(_.getName.endsWith(s".$format"))).sorted - val sigsOfVanillaWriter = - getSignature( - format, - recursiveListFiles(new File(getWarehouseDir + "/" + table_name_vanilla)) - .filter(_.getName.endsWith(s".$format"))).sorted - - assert(sigsOfVanillaWriter.sameElements(sigsOfNativeWriter)) + val table_name_vanilla = table_name_vanilla_template.format(format) + spark.sql(s"drop table IF EXISTS $table_name_vanilla") + withSQLConf(("spark.gluten.sql.native.writer.enabled", "false")) { + withNativeWriteCheck(checkNative = false) { + spark + .table("origin_table") + .select(fields.keys.toSeq.head, fields.keys.toSeq.tail: _*) + .write + .format(format) + .partitionBy("date_field") + .bucketBy(10, "byte_field", "string_field") + .saveAsTable(table_name_vanilla) + } + val sigsOfNativeWriter = + getSignature( + format, + recursiveListFiles(new File(getWarehouseDir + "/" + table_name)) + .filter(_.getName.endsWith(s".$format"))).sorted + val sigsOfVanillaWriter = + getSignature( + format, + recursiveListFiles(new File(getWarehouseDir + "/" + table_name_vanilla)) + .filter(_.getName.endsWith(s".$format"))).sorted + + assertResult(sigsOfVanillaWriter)(sigsOfNativeWriter) + } } } } @@ -677,18 +724,19 @@ class GlutenClickHouseNativeWriteTableSuite val table_name = table_name_template.format(format) spark.sql(s"drop table IF EXISTS $table_name") - // 8096 row per block, so there will be blocks containing all 0 in p, all 1 in p - spark - .range(30000) - .selectExpr("id", "id % 2 as p") - .orderBy("p") - .write - .format(format) - .partitionBy("p") - .saveAsTable(table_name) - + withNativeWriteCheck(checkNative = true) { + // 8096 row per block, so there will be blocks containing all 0 in p, all 1 in p + spark + .range(30000) + .selectExpr("id", "id % 2 as p") + .orderBy("p") + .write + .format(format) + .partitionBy("p") + .saveAsTable(table_name) + } val ret = spark.sql(s"select sum(id) from $table_name").collect().apply(0).apply(0) - assert(ret == 449985000) + assertResult(449985000)(ret) } } @@ -697,13 +745,15 @@ class GlutenClickHouseNativeWriteTableSuite format => val table_name = table_name_template.format(format) spark.sql(s"drop table IF EXISTS $table_name") - spark - .range(200) - .selectExpr("id", " cast((id + rand()) as decimal(23,12)) as p") - .write - .format(format) - .partitionBy("p") - .saveAsTable(table_name) + withNativeWriteCheck(checkNative = true) { + spark + .range(200) + .selectExpr("id", " cast((id + rand()) as decimal(23,12)) as p") + .write + .format(format) + .partitionBy("p") + .saveAsTable(table_name) + } val ret = spark.sql(s"select max(p) from $table_name").collect().apply(0).apply(0) } } @@ -730,15 +780,15 @@ class GlutenClickHouseNativeWriteTableSuite format => val table_name = table_name_template.format(format) spark.sql(s"drop table IF EXISTS $table_name") - - spark - .range(10000000) - .selectExpr("id", "cast('2020-01-01' as date) as p") - .write - .format(format) - .bucketBy(2, "p") - .saveAsTable(table_name) - + withNativeWriteCheck(checkNative = isSparkVersionLE("3.3")) { + spark + .range(10000000) + .selectExpr("id", "cast('2020-01-01' as date) as p") + .write + .format(format) + .bucketBy(2, "p") + .saveAsTable(table_name) + } assertResult(10000000)(spark.table(table_name).count()) } } @@ -748,15 +798,15 @@ class GlutenClickHouseNativeWriteTableSuite format => val table_name = table_name_template.format(format) spark.sql(s"drop table IF EXISTS $table_name") - - spark - .range(30000) - .selectExpr("id", "cast(null as string) as p") - .write - .format(format) - .partitionBy("p") - .saveAsTable(table_name) - + withNativeWriteCheck(checkNative = true) { + spark + .range(30000) + .selectExpr("id", "cast(null as string) as p") + .write + .format(format) + .partitionBy("p") + .saveAsTable(table_name) + } assertResult(30000)(spark.table(table_name).count()) } } @@ -766,15 +816,15 @@ class GlutenClickHouseNativeWriteTableSuite format => val table_name = table_name_template.format(format) spark.sql(s"drop table IF EXISTS $table_name") - - spark - .range(30000) - .selectExpr("id", "cast(null as string) as p") - .write - .format(format) - .bucketBy(2, "p") - .saveAsTable(table_name) - + withNativeWriteCheck(checkNative = isSparkVersionLE("3.3")) { + spark + .range(30000) + .selectExpr("id", "cast(null as string) as p") + .write + .format(format) + .bucketBy(2, "p") + .saveAsTable(table_name) + } assertResult(30000)(spark.table(table_name).count()) } } @@ -789,7 +839,7 @@ class GlutenClickHouseNativeWriteTableSuite s"insert into $table_name select id, cast(id as string) from range(10) where id > 100" ) }, - (table_name, _, _) => { + (table_name, _) => { assertResult(0)(spark.table(table_name).count()) } ) @@ -802,12 +852,12 @@ class GlutenClickHouseNativeWriteTableSuite withDestinationTable( table_name, s"create table $table_name (id int, str string) stored as $format") { - checkNativeWrite( + checkInsertQuery( s"insert overwrite table $table_name " + "select id, cast(id as string) from range(10) union all " + "select 10, '10' from range(10)", checkNative = true) - checkNativeWrite( + checkInsertQuery( s"insert overwrite table $table_name " + "select id, cast(id as string) from range(10) union all " + "select 10, cast(id as string) from range(10)", @@ -829,25 +879,62 @@ class GlutenClickHouseNativeWriteTableSuite "select id, cast(id as string), concat('aaa', cast(id as string)) from range(10)" ) }, - (table_name, _, _) => + (table_name, _) => + // https://github.com/apache/spark/pull/38151 add read-side char padding cause fallback. compareResultsAgainstVanillaSpark( s"select * from $table_name", compareResult = true, - _ => {}) + _ => {}, + isSparkVersionLE("3.3") + ) ) } test("GLUTEN-4316: fix crash on dynamic partition inserting") { - nativeWrite2 { - format => - val tbl = "t_" + format - val sql1 = - s"create table $tbl(a int, b map, c struct) " + - s"partitioned by (day string) stored as $format" - val sql2 = s"insert overwrite $tbl partition (day) " + - s"select id as a, str_to_map(concat('t1:','a','&t2:','b'),'&',':'), " + - s"struct('1', null) as c, '2024-01-08' as day from range(10)" - (tbl, sql1, sql2) - } + nativeWrite2( + { + format => + val table_name = "t_" + format + val create_sql = + s"""create table $table_name( + | a int, + | b map, + | c struct + | ) partitioned by (day string) + | stored as $format""".stripMargin + + // FIXME: + // Spark analyzer(>=3.4) will resolve map type to + // map_from_arrays(transform(map_keys(map('t1','a','t2','b')), v->v), + // transform(map_values(map('t1','a','t2','b')), v->v)) + // which cause core dump. see https://github.com/apache/incubator-gluten/issues/6561 + // for details. + val insert_sql = + if (isSparkVersionLE("3.3")) { + s"""insert overwrite $table_name partition (day) + |select id as a, + | str_to_map(concat('t1:','a','&t2:','b'),'&',':'), + | struct('1', null) as c, + | '2024-01-08' as day + |from range(10)""".stripMargin + } else { + s"""insert overwrite $table_name partition (day) + |select id as a, + | map('t1', 'a', 't2', 'b'), + | struct('1', null) as c, + | '2024-01-08' as day + |from range(10)""".stripMargin + } + (table_name, create_sql, insert_sql) + }, + (table_name, _) => + if (isSparkVersionGE("3.4")) { + // FIXME: Don't Know Why Failed + compareResultsAgainstVanillaSpark( + s"select * from $table_name", + compareResult = true, + _ => {}) + } + ) } } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseMetricsUTUtils.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseMetricsUTUtils.scala index ee0ad8039afc1..3253e04bb36cc 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseMetricsUTUtils.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseMetricsUTUtils.scala @@ -45,9 +45,8 @@ object GlutenClickHouseMetricsUTUtils { SubstraitPlanPrinterUtil.jsonToSubstraitPlan( substraitPlanJsonStr.replaceAll("basePath", basePath.substring(1))) - val transKernel = new CHNativeExpressionEvaluator() val mockMemoryAllocator = CHNativeMemoryAllocators.contextInstanceForUT() - val resIter = transKernel.createKernelWithBatchIterator( + val resIter = CHNativeExpressionEvaluator.createKernelWithBatchIterator( mockMemoryAllocator.getNativeInstanceId, substraitPlan.toByteArray, new Array[Array[Byte]](0), diff --git a/backends-clickhouse/src/test/scala/org/apache/spark/gluten/NativeWriteChecker.scala b/backends-clickhouse/src/test/scala/org/apache/spark/gluten/NativeWriteChecker.scala index 79616d52d0bc4..590d221f0e3a0 100644 --- a/backends-clickhouse/src/test/scala/org/apache/spark/gluten/NativeWriteChecker.scala +++ b/backends-clickhouse/src/test/scala/org/apache/spark/gluten/NativeWriteChecker.scala @@ -18,35 +18,69 @@ package org.apache.spark.gluten import org.apache.gluten.execution.GlutenClickHouseWholeStageTransformerSuite -import org.apache.spark.sql.execution.QueryExecution +import org.apache.spark.sql.{Dataset, Row} +import org.apache.spark.sql.execution.{ColumnarWriteFilesExec, QueryExecution} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.datasources.FakeRowAdaptor import org.apache.spark.sql.util.QueryExecutionListener -trait NativeWriteChecker extends GlutenClickHouseWholeStageTransformerSuite { +trait NativeWriteChecker + extends GlutenClickHouseWholeStageTransformerSuite + with AdaptiveSparkPlanHelper { - def checkNativeWrite(sqlStr: String, checkNative: Boolean): Unit = { + private val formats: Seq[String] = Seq("orc", "parquet") + + def withNativeWriteCheck(checkNative: Boolean)(block: => Unit): Unit = { var nativeUsed = false val queryListener = new QueryExecutionListener { - override def onFailure(f: String, qe: QueryExecution, e: Exception): Unit = {} + override def onFailure(f: String, qe: QueryExecution, e: Exception): Unit = { + fail("query failed", e) + } override def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = { if (!nativeUsed) { + val executedPlan = stripAQEPlan(qe.executedPlan) nativeUsed = if (isSparkVersionGE("3.4")) { - false + executedPlan.find(_.isInstanceOf[ColumnarWriteFilesExec]).isDefined } else { - qe.executedPlan.find(_.isInstanceOf[FakeRowAdaptor]).isDefined + executedPlan.find(_.isInstanceOf[FakeRowAdaptor]).isDefined } } } } - try { spark.listenerManager.register(queryListener) - spark.sql(sqlStr) + block spark.sparkContext.listenerBus.waitUntilEmpty() assertResult(checkNative)(nativeUsed) } finally { spark.listenerManager.unregister(queryListener) } } + def checkInsertQuery(sqlStr: String, checkNative: Boolean): Unit = + withNativeWriteCheck(checkNative) { + spark.sql(sqlStr) + } + + def withDestinationTable(table: String, createTableSql: String = "select 1")(f: => Unit): Unit = { + spark.sql(s"drop table IF EXISTS $table") + spark.sql(s"$createTableSql") + f + } + + def nativeWrite(f: String => Unit): Unit = { + withSQLConf(("spark.gluten.sql.native.writer.enabled", "true")) { + formats.foreach(f(_)) + } + } + + def withSource(df: Dataset[Row], viewName: String, pairs: (String, String)*)( + block: => Unit): Unit = { + withSQLConf(pairs: _*) { + withTempView(viewName) { + df.createOrReplaceTempView(viewName) + block + } + } + } } diff --git a/cpp-ch/local-engine/Common/BlockTypeUtils.cpp b/cpp-ch/local-engine/Common/BlockTypeUtils.cpp new file mode 100644 index 0000000000000..f6dcd0959225b --- /dev/null +++ b/cpp-ch/local-engine/Common/BlockTypeUtils.cpp @@ -0,0 +1,56 @@ +/* + * 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. + */ + +#include "BlockTypeUtils.h" + +#include +#include + +namespace local_engine +{ +DB::NamesAndTypesList blockToNameAndTypeList(const DB::Block & header) +{ + DB::NamesAndTypesList types; + for (const auto & name : header.getNames()) + { + const auto * column = header.findByName(name); + types.push_back(DB::NameAndTypePair(column->name, column->type)); + } + return types; +} + +DB::DataTypePtr wrapNullableType(bool nullable, DB::DataTypePtr nested_type) +{ + if (nullable && !nested_type->isNullable()) + { + if (nested_type->isLowCardinalityNullable()) + return nested_type; + else if (!nested_type->lowCardinality()) + return std::make_shared(nested_type); + else + return std::make_shared( + std::make_shared(dynamic_cast(*nested_type).getDictionaryType())); + } + + + if (nullable && !nested_type->isNullable()) + return std::make_shared(nested_type); + else + return nested_type; +} + +} \ No newline at end of file diff --git a/cpp-ch/local-engine/Common/BlockTypeUtils.h b/cpp-ch/local-engine/Common/BlockTypeUtils.h new file mode 100644 index 0000000000000..af01b528d6e9c --- /dev/null +++ b/cpp-ch/local-engine/Common/BlockTypeUtils.h @@ -0,0 +1,89 @@ +/* + * 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. + */ + +#pragma once + +#include +#include +#include +#include +#include + +namespace local_engine +{ +inline DB::DataTypePtr BIGINT() +{ + return std::make_shared(); +} +inline DB::DataTypePtr INT() +{ + return std::make_shared(); +} +inline DB::DataTypePtr INT16() +{ + return std::make_shared(); +} +inline DB::DataTypePtr INT8() +{ + return std::make_shared(); +} +inline DB::DataTypePtr UBIGINT() +{ + return std::make_shared(); +} +inline DB::DataTypePtr UINT() +{ + return std::make_shared(); +} +inline DB::DataTypePtr UINT16() +{ + return std::make_shared(); +} +inline DB::DataTypePtr UINT8() +{ + return std::make_shared(); +} + +inline DB::DataTypePtr DOUBLE() +{ + return std::make_shared(); +} + +inline DB::DataTypePtr STRING() +{ + return std::make_shared(); +} + +inline DB::DataTypePtr DATE() +{ + return std::make_shared(); +} + +inline DB::Block makeBlockHeader(const DB::ColumnsWithTypeAndName & data_) +{ + return DB::Block(data_); +} + +DB::NamesAndTypesList blockToNameAndTypeList(const DB::Block & header); +DB::DataTypePtr wrapNullableType(bool nullable, DB::DataTypePtr nested_type); + +inline DB::DataTypePtr wrapNullableType(const substrait::Type_Nullability nullable, const DB::DataTypePtr & nested_type) +{ + return wrapNullableType(nullable == substrait::Type_Nullability_NULLABILITY_NULLABLE, nested_type); +} + +} diff --git a/cpp-ch/local-engine/Common/CHUtil.cpp b/cpp-ch/local-engine/Common/CHUtil.cpp index b74c18dd14af1..1be9c09a8c266 100644 --- a/cpp-ch/local-engine/Common/CHUtil.cpp +++ b/cpp-ch/local-engine/Common/CHUtil.cpp @@ -487,7 +487,7 @@ String QueryPipelineUtil::explainPipeline(DB::QueryPipeline & pipeline) using namespace DB; -std::map BackendInitializerUtil::getBackendConfMap(const std::string_view plan) +std::map BackendInitializerUtil::getBackendConfMap(std::string_view plan) { std::map ch_backend_conf; if (plan.empty()) @@ -496,8 +496,8 @@ std::map BackendInitializerUtil::getBackendConfMap(con /// Parse backend configs from plan extensions do { - auto plan_ptr = std::make_unique(); - auto success = plan_ptr->ParseFromString(plan); + substrait::Plan sPlan; + auto success = sPlan.ParseFromString(plan); if (!success) break; @@ -506,15 +506,15 @@ std::map BackendInitializerUtil::getBackendConfMap(con namespace pb_util = google::protobuf::util; pb_util::JsonOptions options; std::string json; - auto s = pb_util::MessageToJsonString(*plan_ptr, &json, options); + auto s = pb_util::MessageToJsonString(sPlan, &json, options); if (!s.ok()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not convert Substrait Plan to Json"); LOG_DEBUG(&Poco::Logger::get("CHUtil"), "Update Config Map Plan:\n{}", json); } - if (!plan_ptr->has_advanced_extensions() || !plan_ptr->advanced_extensions().has_enhancement()) + if (!sPlan.has_advanced_extensions() || !sPlan.advanced_extensions().has_enhancement()) break; - const auto & enhancement = plan_ptr->advanced_extensions().enhancement(); + const auto & enhancement = sPlan.advanced_extensions().enhancement(); if (!enhancement.Is()) break; @@ -935,8 +935,7 @@ void BackendInitializerUtil::initCompiledExpressionCache(DB::Context::Configurat #endif } - -void BackendInitializerUtil::init(const std::string & plan) +void BackendInitializerUtil::init(const std::string_view plan) { std::map backend_conf_map = getBackendConfMap(plan); DB::Context::ConfigurationPtr config = initConfig(backend_conf_map); diff --git a/cpp-ch/local-engine/Common/CHUtil.h b/cpp-ch/local-engine/Common/CHUtil.h index 7be3f86dc2303..98139fb49a5b3 100644 --- a/cpp-ch/local-engine/Common/CHUtil.h +++ b/cpp-ch/local-engine/Common/CHUtil.h @@ -42,6 +42,8 @@ namespace local_engine static const String MERGETREE_INSERT_WITHOUT_LOCAL_STORAGE = "mergetree.insert_without_local_storage"; static const String MERGETREE_MERGE_AFTER_INSERT = "mergetree.merge_after_insert"; static const std::string DECIMAL_OPERATIONS_ALLOW_PREC_LOSS = "spark.sql.decimalOperations.allowPrecisionLoss"; +static const std::string SPARK_TASK_WRITE_TMEP_DIR = "gluten.write.temp.dir"; +static const std::string SPARK_TASK_WRITE_FILENAME = "gluten.write.file.name"; static const std::unordered_set BOOL_VALUE_SETTINGS{ MERGETREE_MERGE_AFTER_INSERT, MERGETREE_INSERT_WITHOUT_LOCAL_STORAGE, DECIMAL_OPERATIONS_ALLOW_PREC_LOSS}; @@ -151,7 +153,7 @@ class BackendInitializerUtil /// Initialize two kinds of resources /// 1. global level resources like global_context/shared_context, notice that they can only be initialized once in process lifetime /// 2. session level resources like settings/configs, they can be initialized multiple times following the lifetime of executor/driver - static void init(const std::string & plan); + static void init(const std::string_view plan); static void updateConfig(const DB::ContextMutablePtr &, const std::string_view); @@ -210,7 +212,7 @@ class BackendInitializerUtil static std::vector wrapDiskPathConfig(const String & path_prefix, const String & path_suffix, Poco::Util::AbstractConfiguration & config); - static std::map getBackendConfMap(const std::string_view plan); + static std::map getBackendConfMap(std::string_view plan); inline static std::once_flag init_flag; inline static Poco::Logger * logger; diff --git a/cpp-ch/local-engine/Parser/FunctionParser.cpp b/cpp-ch/local-engine/Parser/FunctionParser.cpp index 206fb6c50ec97..513470d0c250b 100644 --- a/cpp-ch/local-engine/Parser/FunctionParser.cpp +++ b/cpp-ch/local-engine/Parser/FunctionParser.cpp @@ -22,6 +22,7 @@ #include #include #include +#include #include namespace DB diff --git a/cpp-ch/local-engine/Parser/JoinRelParser.cpp b/cpp-ch/local-engine/Parser/JoinRelParser.cpp index 09a5152174fdf..460311e289b67 100644 --- a/cpp-ch/local-engine/Parser/JoinRelParser.cpp +++ b/cpp-ch/local-engine/Parser/JoinRelParser.cpp @@ -34,7 +34,6 @@ #include #include -#include #include diff --git a/cpp-ch/local-engine/Parser/ProjectRelParser.h b/cpp-ch/local-engine/Parser/ProjectRelParser.h index 48a16d774d887..328acfc72fd5f 100644 --- a/cpp-ch/local-engine/Parser/ProjectRelParser.h +++ b/cpp-ch/local-engine/Parser/ProjectRelParser.h @@ -57,8 +57,8 @@ class ProjectRelParser : public RelParser { if (rel.has_generate()) return rel.generate().input(); - else - return rel.project().input(); + + return rel.project().input(); } }; } diff --git a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp index 50431abd9fdba..151e24c6da456 100644 --- a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp +++ b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp @@ -55,7 +55,9 @@ #include #include #include +#include #include +#include #include #include #include @@ -71,11 +73,13 @@ #include #include #include +#include #include #include #include #include #include +#include #include #include #include @@ -280,7 +284,7 @@ QueryPlanStepPtr SerializedPlanParser::parseReadRealWithLocalFile(const substrai if (rel.has_local_files()) local_files = rel.local_files(); else - local_files = parseLocalFiles(split_infos.at(nextSplitInfoIndex())); + local_files = BinaryToMessage(split_infos.at(nextSplitInfoIndex())); auto source = std::make_shared(context, header, local_files); auto source_pipe = Pipe(source); auto source_step = std::make_unique(context, std::move(source_pipe), "substrait local files"); @@ -346,121 +350,99 @@ IQueryPlanStep * SerializedPlanParser::addRollbackFilterHeaderStep(QueryPlanPtr return step_ptr; } -DataTypePtr wrapNullableType(substrait::Type_Nullability nullable, DataTypePtr nested_type) +void adjustOutput(const DB::QueryPlanPtr & query_plan, const substrait::PlanRel & root_rel) { - return wrapNullableType(nullable == substrait::Type_Nullability_NULLABILITY_NULLABLE, nested_type); -} - -DataTypePtr wrapNullableType(bool nullable, DataTypePtr nested_type) -{ - if (nullable && !nested_type->isNullable()) + if (root_rel.root().names_size()) { - if (nested_type->isLowCardinalityNullable()) - { - return nested_type; - } - else - { - if (!nested_type->lowCardinality()) - return std::make_shared(nested_type); - else - return std::make_shared( - std::make_shared( - dynamic_cast(*nested_type).getDictionaryType())); - } + ActionsDAGPtr actions_dag = std::make_shared(blockToNameAndTypeList(query_plan->getCurrentDataStream().header)); + NamesWithAliases aliases; + auto cols = query_plan->getCurrentDataStream().header.getNamesAndTypesList(); + if (cols.getNames().size() != static_cast(root_rel.root().names_size())) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Missmatch result columns size."); + for (int i = 0; i < static_cast(cols.getNames().size()); i++) + aliases.emplace_back(NameWithAlias(cols.getNames()[i], root_rel.root().names(i))); + actions_dag->project(aliases); + auto expression_step = std::make_unique(query_plan->getCurrentDataStream(), actions_dag); + expression_step->setStepDescription("Rename Output"); + query_plan->addStep(std::move(expression_step)); } - - if (nullable && !nested_type->isNullable()) - return std::make_shared(nested_type); - else - return nested_type; -} - -QueryPlanPtr SerializedPlanParser::parse(const substrait::Plan & plan) -{ - logDebugMessage(plan, "substrait plan"); - parseExtensions(plan.extensions()); - if (plan.relations_size() == 1) + // fixes: issue-1874, to keep the nullability as expected. + const auto & output_schema = root_rel.root().output_schema(); + if (output_schema.types_size()) { - auto root_rel = plan.relations().at(0); - if (!root_rel.has_root()) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "must have root rel!"); - } - std::list rel_stack; - auto query_plan = parseOp(root_rel.root().input(), rel_stack); - if (root_rel.root().names_size()) - { - ActionsDAGPtr actions_dag = std::make_shared(blockToNameAndTypeList(query_plan->getCurrentDataStream().header)); - NamesWithAliases aliases; - auto cols = query_plan->getCurrentDataStream().header.getNamesAndTypesList(); - if (cols.getNames().size() != static_cast(root_rel.root().names_size())) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Missmatch result columns size."); - } - for (int i = 0; i < static_cast(cols.getNames().size()); i++) - { - aliases.emplace_back(NameWithAlias(cols.getNames()[i], root_rel.root().names(i))); - } - actions_dag->project(aliases); - auto expression_step = std::make_unique(query_plan->getCurrentDataStream(), actions_dag); - expression_step->setStepDescription("Rename Output"); - query_plan->addStep(std::move(expression_step)); - } - - // fixes: issue-1874, to keep the nullability as expected. - const auto & output_schema = root_rel.root().output_schema(); - if (output_schema.types_size()) + auto original_header = query_plan->getCurrentDataStream().header; + const auto & original_cols = original_header.getColumnsWithTypeAndName(); + if (static_cast(output_schema.types_size()) != original_cols.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Mismatch output schema"); + bool need_final_project = false; + ColumnsWithTypeAndName final_cols; + for (int i = 0; i < output_schema.types_size(); ++i) { - auto original_header = query_plan->getCurrentDataStream().header; - const auto & original_cols = original_header.getColumnsWithTypeAndName(); - if (static_cast(output_schema.types_size()) != original_cols.size()) + const auto & col = original_cols[i]; + auto type = TypeParser::parseType(output_schema.types(i)); + // At present, we only check nullable mismatch. + // intermediate aggregate data is special, no check here. + if (type->isNullable() != col.type->isNullable() && !typeid_cast(col.type.get())) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Mismatch output schema"); - } - bool need_final_project = false; - ColumnsWithTypeAndName final_cols; - for (int i = 0; i < output_schema.types_size(); ++i) - { - const auto & col = original_cols[i]; - auto type = TypeParser::parseType(output_schema.types(i)); - // At present, we only check nullable mismatch. - // intermediate aggregate data is special, no check here. - if (type->isNullable() != col.type->isNullable() && !typeid_cast(col.type.get())) + if (type->isNullable()) { - if (type->isNullable()) - { - auto wrapped = wrapNullableType(true, col.type); - final_cols.emplace_back(type->createColumn(), wrapped, col.name); - need_final_project = !wrapped->equals(*col.type); - } - else - { - final_cols.emplace_back(type->createColumn(), removeNullable(col.type), col.name); - need_final_project = true; - } + auto wrapped = wrapNullableType(true, col.type); + final_cols.emplace_back(type->createColumn(), wrapped, col.name); + need_final_project = !wrapped->equals(*col.type); } else { - final_cols.push_back(col); + final_cols.emplace_back(type->createColumn(), removeNullable(col.type), col.name); + need_final_project = true; } } - if (need_final_project) + else { - ActionsDAGPtr final_project - = ActionsDAG::makeConvertingActions(original_cols, final_cols, ActionsDAG::MatchColumnsMode::Position); - QueryPlanStepPtr final_project_step = std::make_unique(query_plan->getCurrentDataStream(), final_project); - final_project_step->setStepDescription("Project for output schema"); - query_plan->addStep(std::move(final_project_step)); + final_cols.push_back(col); } } - return query_plan; + if (need_final_project) + { + ActionsDAGPtr final_project + = ActionsDAG::makeConvertingActions(original_cols, final_cols, ActionsDAG::MatchColumnsMode::Position); + QueryPlanStepPtr final_project_step = std::make_unique(query_plan->getCurrentDataStream(), final_project); + final_project_step->setStepDescription("Project for output schema"); + query_plan->addStep(std::move(final_project_step)); + } } - else - { +} + +QueryPlanPtr SerializedPlanParser::parse(const substrait::Plan & plan) +{ + logDebugMessage(plan, "substrait plan"); + parseExtensions(plan.extensions()); + if (plan.relations_size() != 1) throw Exception(ErrorCodes::BAD_ARGUMENTS, "too many relations found"); + + const substrait::PlanRel & root_rel = plan.relations().at(0); + if (!root_rel.has_root()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "must have root rel!"); + + const bool writePipeline = root_rel.root().input().has_write(); + const substrait::Rel & first_read_rel = writePipeline ? root_rel.root().input().write().input() : root_rel.root().input(); + + std::list rel_stack; + auto query_plan = parseOp(first_read_rel, rel_stack); + if (!writePipeline) + adjustOutput(query_plan, root_rel); + +#ifndef NDEBUG + PlanUtil::checkOuputType(*query_plan); +#endif + + if (auto * logger = &Poco::Logger::get("SerializedPlanParser"); logger->debug()) + { + auto out = PlanUtil::explainPlan(*query_plan); + LOG_DEBUG(logger, "clickhouse plan:\n{}", out); } + + return query_plan; } QueryPlanPtr SerializedPlanParser::parseOp(const substrait::Rel & rel, std::list & rel_stack) @@ -513,7 +495,7 @@ QueryPlanPtr SerializedPlanParser::parseOp(const substrait::Rel & rel, std::list if (read.has_extension_table()) extension_table = read.extension_table(); else - extension_table = parseExtensionTable(split_infos.at(nextSplitInfoIndex())); + extension_table = BinaryToMessage(split_infos.at(nextSplitInfoIndex())); MergeTreeRelParser mergeTreeParser(this, context); query_plan = mergeTreeParser.parseReadRel(std::make_unique(), read, extension_table); @@ -554,17 +536,6 @@ QueryPlanPtr SerializedPlanParser::parseOp(const substrait::Rel & rel, std::list return query_plan; } -NamesAndTypesList SerializedPlanParser::blockToNameAndTypeList(const Block & header) -{ - NamesAndTypesList types; - for (const auto & name : header.getNames()) - { - const auto * column = header.findByName(name); - types.push_back(NameAndTypePair(column->name, column->type)); - } - return types; -} - std::optional SerializedPlanParser::getFunctionSignatureName(UInt32 function_ref) const { auto it = function_mapping.find(std::to_string(function_ref)); @@ -784,16 +755,13 @@ const ActionsDAG::Node * SerializedPlanParser::parseFunctionWithDAG( } void SerializedPlanParser::parseFunctionArguments( - ActionsDAGPtr & actions_dag, - ActionsDAG::NodeRawConstPtrs & parsed_args, - const substrait::Expression_ScalarFunction & scalar_function) + ActionsDAGPtr & actions_dag, ActionsDAG::NodeRawConstPtrs & parsed_args, const substrait::Expression_ScalarFunction & scalar_function) { auto function_signature = function_mapping.at(std::to_string(scalar_function.function_reference())); const auto & args = scalar_function.arguments(); parsed_args.reserve(args.size()); for (const auto & arg : args) parsed_args.emplace_back(parseExpression(actions_dag, arg.value())); - } // Convert signed integer index into unsigned integer index @@ -1340,104 +1308,58 @@ const ActionsDAG::Node * SerializedPlanParser::parseExpression(ActionsDAGPtr act } } -substrait::ReadRel::ExtensionTable SerializedPlanParser::parseExtensionTable(const std::string & split_info) +DB::QueryPipelineBuilderPtr SerializedPlanParser::buildQueryPipeline(DB::QueryPlan & query_plan) { - substrait::ReadRel::ExtensionTable extension_table; - google::protobuf::io::CodedInputStream coded_in( - reinterpret_cast(split_info.data()), static_cast(split_info.size())); - coded_in.SetRecursionLimit(100000); - - auto ok = extension_table.ParseFromCodedStream(&coded_in); - if (!ok) - throw Exception(ErrorCodes::CANNOT_PARSE_PROTOBUF_SCHEMA, "Parse substrait::ReadRel::ExtensionTable from string failed"); - logDebugMessage(extension_table, "extension_table"); - return extension_table; -} - -substrait::ReadRel::LocalFiles SerializedPlanParser::parseLocalFiles(const std::string & split_info) -{ - substrait::ReadRel::LocalFiles local_files; - google::protobuf::io::CodedInputStream coded_in( - reinterpret_cast(split_info.data()), static_cast(split_info.size())); - coded_in.SetRecursionLimit(100000); - - auto ok = local_files.ParseFromCodedStream(&coded_in); - if (!ok) - throw Exception(ErrorCodes::CANNOT_PARSE_PROTOBUF_SCHEMA, "Parse substrait::ReadRel::LocalFiles from string failed"); - logDebugMessage(local_files, "local_files"); - return local_files; -} - -std::unique_ptr SerializedPlanParser::createExecutor(DB::QueryPlanPtr query_plan) -{ - Stopwatch stopwatch; - auto * logger = &Poco::Logger::get("SerializedPlanParser"); const Settings & settings = context->getSettingsRef(); - QueryPriorities priorities; - auto query_status = std::make_shared( + const auto query_status = std::make_shared( context, "", context->getClientInfo(), - priorities.insert(static_cast(settings.priority)), + priorities.insert(settings.priority), CurrentThread::getGroup(), IAST::QueryKind::Select, settings, 0); - - QueryPlanOptimizationSettings optimization_settings{.optimize_plan = settings.query_plan_enable_optimizations}; - auto pipeline_builder = query_plan->buildQueryPipeline( + const QueryPlanOptimizationSettings optimization_settings{.optimize_plan = settings.query_plan_enable_optimizations}; + return query_plan.buildQueryPipeline( optimization_settings, BuildQueryPipelineSettings{ .actions_settings = ExpressionActionsSettings{.can_compile_expressions = true, .min_count_to_compile_expression = 3, .compile_expressions = CompileExpressions::yes}, .process_list_element = query_status}); - QueryPipeline pipeline = QueryPipelineBuilder::getPipeline(std::move(*pipeline_builder)); - LOG_INFO(logger, "build pipeline {} ms", stopwatch.elapsedMicroseconds() / 1000.0); - - LOG_DEBUG( - logger, "clickhouse plan [optimization={}]:\n{}", settings.query_plan_enable_optimizations, PlanUtil::explainPlan(*query_plan)); - LOG_DEBUG(logger, "clickhouse pipeline:\n{}", QueryPipelineUtil::explainPipeline(pipeline)); - - return std::make_unique( - context, std::move(query_plan), std::move(pipeline), query_plan->getCurrentDataStream().header.cloneEmpty()); } -QueryPlanPtr SerializedPlanParser::parse(const std::string_view plan) +std::unique_ptr SerializedPlanParser::createExecutor(const std::string_view plan) { - substrait::Plan s_plan; - /// https://stackoverflow.com/questions/52028583/getting-error-parsing-protobuf-data - /// Parsing may fail when the number of recursive layers is large. - /// Here, set a limit large enough to avoid this problem. - /// Once this problem occurs, it is difficult to troubleshoot, because the pb of c++ will not provide any valid information - google::protobuf::io::CodedInputStream coded_in(reinterpret_cast(plan.data()), static_cast(plan.size())); - coded_in.SetRecursionLimit(100000); + const auto s_plan = BinaryToMessage(plan); + return createExecutor(parse(s_plan), s_plan); +} - if (!s_plan.ParseFromCodedStream(&coded_in)) - throw Exception(ErrorCodes::CANNOT_PARSE_PROTOBUF_SCHEMA, "Parse substrait::Plan from string failed"); +std::unique_ptr SerializedPlanParser::createExecutor(DB::QueryPlanPtr query_plan, const substrait::Plan & s_plan) +{ + Stopwatch stopwatch; - auto res = parse(s_plan); + const Settings & settings = context->getSettingsRef(); + auto builder = buildQueryPipeline(*query_plan); -#ifndef NDEBUG - PlanUtil::checkOuputType(*res); -#endif + /// + assert(s_plan.relations_size() == 1); + const substrait::PlanRel & root_rel = s_plan.relations().at(0); + assert(root_rel.has_root()); + if (root_rel.root().input().has_write()) + addSinkTransfrom(context, root_rel.root().input().write(), builder); + /// + QueryPipeline pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); auto * logger = &Poco::Logger::get("SerializedPlanParser"); - if (logger->debug()) - { - auto out = PlanUtil::explainPlan(*res); - LOG_DEBUG(logger, "clickhouse plan:\n{}", out); - } - return res; -} + LOG_INFO(logger, "build pipeline {} ms", stopwatch.elapsedMicroseconds() / 1000.0); + LOG_DEBUG( + logger, "clickhouse plan [optimization={}]:\n{}", settings.query_plan_enable_optimizations, PlanUtil::explainPlan(*query_plan)); + LOG_DEBUG(logger, "clickhouse pipeline:\n{}", QueryPipelineUtil::explainPipeline(pipeline)); -QueryPlanPtr SerializedPlanParser::parseJson(const std::string_view & json_plan) -{ - substrait::Plan plan; - auto s = google::protobuf::util::JsonStringToMessage(json_plan, &plan); - if (!s.ok()) - throw Exception(ErrorCodes::CANNOT_PARSE_PROTOBUF_SCHEMA, "Parse substrait::Plan from json string failed: {}", s.ToString()); - return parse(plan); + bool dump_pipeline = context->getConfigRef().getBool("dump_pipeline", false); + return std::make_unique(std::move(query_plan), std::move(pipeline), dump_pipeline); } SerializedPlanParser::SerializedPlanParser(const ContextPtr & context_) : context(context_) @@ -1689,7 +1611,7 @@ SharedContextHolder SerializedPlanParser::shared_context; LocalExecutor::~LocalExecutor() { - if (context->getConfigRef().getBool("dump_pipeline", false)) + if (dump_pipeline) LOG_INFO(&Poco::Logger::get("LocalExecutor"), "Dump pipeline:\n{}", dumpPipeline()); if (spark_buffer) @@ -1763,11 +1685,11 @@ Block & LocalExecutor::getHeader() return header; } -LocalExecutor::LocalExecutor(const ContextPtr & context_, QueryPlanPtr query_plan, QueryPipeline && pipeline, const Block & header_) +LocalExecutor::LocalExecutor(QueryPlanPtr query_plan, QueryPipeline && pipeline, bool dump_pipeline_) : query_pipeline(std::move(pipeline)) , executor(std::make_unique(query_pipeline)) - , header(header_) - , context(context_) + , header(query_plan->getCurrentDataStream().header.cloneEmpty()) + , dump_pipeline(dump_pipeline_) , ch_column_to_spark_row(std::make_unique()) , current_query_plan(std::move(query_plan)) { diff --git a/cpp-ch/local-engine/Parser/SerializedPlanParser.h b/cpp-ch/local-engine/Parser/SerializedPlanParser.h index fbc22a41dd050..a7d77fde84c1b 100644 --- a/cpp-ch/local-engine/Parser/SerializedPlanParser.h +++ b/cpp-ch/local-engine/Parser/SerializedPlanParser.h @@ -35,9 +35,6 @@ namespace local_engine { -DataTypePtr wrapNullableType(substrait::Type_Nullability nullable, DataTypePtr nested_type); -DataTypePtr wrapNullableType(bool nullable, DataTypePtr nested_type); - std::string join(const ActionsDAG::NodeRawConstPtrs & v, char c); class SerializedPlanParser; @@ -83,20 +80,16 @@ class SerializedPlanParser friend class MergeTreeRelParser; friend class ProjectRelParser; - std::unique_ptr createExecutor(DB::QueryPlanPtr query_plan); - - DB::QueryPlanPtr parse(const std::string_view plan); - DB::QueryPlanPtr parse(const substrait::Plan & plan); + std::unique_ptr createExecutor(DB::QueryPlanPtr query_plan, const substrait::Plan & s_plan); public: explicit SerializedPlanParser(const ContextPtr & context); - /// UT only - DB::QueryPlanPtr parseJson(const std::string_view & json_plan); - std::unique_ptr createExecutor(const substrait::Plan & plan) { return createExecutor(parse((plan))); } + /// visible for UT + DB::QueryPlanPtr parse(const substrait::Plan & plan); + std::unique_ptr createExecutor(const substrait::Plan & plan) { return createExecutor(parse(plan), plan); } + DB::QueryPipelineBuilderPtr buildQueryPipeline(DB::QueryPlan & query_plan); /// - - template std::unique_ptr createExecutor(const std::string_view plan); DB::QueryPlanStepPtr parseReadRealWithLocalFile(const substrait::ReadRel & rel); @@ -105,9 +98,6 @@ class SerializedPlanParser static bool isReadRelFromJava(const substrait::ReadRel & rel); static bool isReadFromMergeTree(const substrait::ReadRel & rel); - static substrait::ReadRel::LocalFiles parseLocalFiles(const std::string & split_info); - static substrait::ReadRel::ExtensionTable parseExtensionTable(const std::string & split_info); - void addInputIter(jobject iter, bool materialize_input) { input_iters.emplace_back(iter); @@ -147,7 +137,6 @@ class SerializedPlanParser std::vector extra_plan_holder; private: - static DB::NamesAndTypesList blockToNameAndTypeList(const DB::Block & header); DB::QueryPlanPtr parseOp(const substrait::Rel & rel, std::list & rel_stack); void collectJoinKeys(const substrait::Expression & condition, std::vector> & join_keys, int32_t right_key_start); @@ -227,12 +216,6 @@ class SerializedPlanParser const ActionsDAG::Node * addColumn(DB::ActionsDAGPtr actions_dag, const DataTypePtr & type, const Field & field); }; -template -std::unique_ptr SerializedPlanParser::createExecutor(const std::string_view plan) -{ - return createExecutor(JsonPlan ? parseJson(plan) : parse(plan)); -} - struct SparkBuffer { char * address; @@ -242,7 +225,7 @@ struct SparkBuffer class LocalExecutor : public BlockIterator { public: - LocalExecutor(const ContextPtr & context_, QueryPlanPtr query_plan, QueryPipeline && pipeline, const Block & header_); + LocalExecutor(QueryPlanPtr query_plan, QueryPipeline && pipeline, bool dump_pipeline_ = false); ~LocalExecutor(); SparkRowInfoPtr next(); @@ -266,7 +249,7 @@ class LocalExecutor : public BlockIterator QueryPipeline query_pipeline; std::unique_ptr executor; Block header; - ContextPtr context; + bool dump_pipeline; std::unique_ptr ch_column_to_spark_row; std::unique_ptr spark_buffer; QueryPlanPtr current_query_plan; diff --git a/cpp-ch/local-engine/Parser/SubstraitParserUtils.h b/cpp-ch/local-engine/Parser/SubstraitParserUtils.h new file mode 100644 index 0000000000000..f247a3bddc095 --- /dev/null +++ b/cpp-ch/local-engine/Parser/SubstraitParserUtils.h @@ -0,0 +1,69 @@ +/* + * 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. + */ +#pragma once + +#include +#include +#include + +namespace DB::ErrorCodes +{ +extern const int CANNOT_PARSE_PROTOBUF_SCHEMA; +} + +namespace local_engine +{ + +template +Message JsonStringToMessage(std::string_view json) +{ + Message message; + auto status = google::protobuf::util::JsonStringToMessage(json, &message); + if (!status.ok()) + { + std::string errmsg(status.message()); + throw DB::Exception(DB::ErrorCodes::CANNOT_PARSE_PROTOBUF_SCHEMA, "Parse failed due to {}", errmsg); + } + return message; +} + +template +std::string JsonStringToBinary(const std::string_view json) +{ + Message message = JsonStringToMessage(json); + std::string binary; + message.SerializeToString(&binary); + return binary; +} + +template +Message BinaryToMessage(const std::string_view binary) +{ + Message message; + /// https://stackoverflow.com/questions/52028583/getting-error-parsing-protobuf-data + /// Parsing may fail when the number of recursive layers is large. + /// Here, set a limit large enough to avoid this problem. + /// Once this problem occurs, it is difficult to troubleshoot, because the pb of c++ will not provide any valid information + google::protobuf::io::CodedInputStream coded_in(reinterpret_cast(binary.data()), static_cast(binary.size())); + coded_in.SetRecursionLimit(100000); + + if (!message.ParseFromCodedStream(&coded_in)) + throw DB::Exception(DB::ErrorCodes::CANNOT_PARSE_PROTOBUF_SCHEMA, "Parse failed"); + return message; +} + +} // namespace local_engine diff --git a/cpp-ch/local-engine/Parser/TypeParser.cpp b/cpp-ch/local-engine/Parser/TypeParser.cpp index 40360ea77302f..39d52131e4e58 100644 --- a/cpp-ch/local-engine/Parser/TypeParser.cpp +++ b/cpp-ch/local-engine/Parser/TypeParser.cpp @@ -193,9 +193,9 @@ DB::DataTypePtr TypeParser::parseType(const substrait::Type & substrait_type, st struct_field_types[i] = parseType(types[i]); const auto & names = substrait_type.struct_().names(); - for (int i = 0; i < names.size(); ++i) - if (!names[i].empty()) - struct_field_names.push_back(names[i]); + for (const auto & name : names) + if (!name.empty()) + struct_field_names.push_back(name); } if (!struct_field_names.empty()) @@ -240,9 +240,7 @@ DB::DataTypePtr TypeParser::parseType(const substrait::Type & substrait_type, st } -DB::Block TypeParser::buildBlockFromNamedStruct( - const substrait::NamedStruct & struct_, - const std::string & low_card_cols) +DB::Block TypeParser::buildBlockFromNamedStruct(const substrait::NamedStruct & struct_, const std::string & low_card_cols) { std::unordered_set low_card_columns; Poco::StringTokenizer tokenizer(low_card_cols, ","); @@ -262,9 +260,7 @@ DB::Block TypeParser::buildBlockFromNamedStruct( auto ch_type = parseType(substrait_type, &field_names); if (low_card_columns.contains(name)) - { ch_type = std::make_shared(ch_type); - } // This is a partial aggregate data column. // It's type is special, must be a struct type contains all arguments types. diff --git a/cpp-ch/local-engine/Parser/TypeParser.h b/cpp-ch/local-engine/Parser/TypeParser.h index 666effff45d37..57a12de0489d4 100644 --- a/cpp-ch/local-engine/Parser/TypeParser.h +++ b/cpp-ch/local-engine/Parser/TypeParser.h @@ -43,8 +43,7 @@ namespace local_engine } // low_card_cols is in format of "cola,colb". Currently does not nested column to be LowCardinality. - static DB::Block buildBlockFromNamedStruct(const substrait::NamedStruct& struct_, - const std::string& low_card_cols = ""); + static DB::Block buildBlockFromNamedStruct(const substrait::NamedStruct& struct_, const std::string& low_card_cols = ""); /// Build block from substrait NamedStruct without DFS rules, different from buildBlockFromNamedStruct static DB::Block buildBlockFromNamedStructWithoutDFS(const substrait::NamedStruct& struct_); diff --git a/cpp-ch/local-engine/Parser/WriteRelParser.cpp b/cpp-ch/local-engine/Parser/WriteRelParser.cpp new file mode 100644 index 0000000000000..b32b7bc6337b4 --- /dev/null +++ b/cpp-ch/local-engine/Parser/WriteRelParser.cpp @@ -0,0 +1,221 @@ +/* + * 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. + */ + +#include "WriteRelParser.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +using namespace local_engine; + +DB::ProcessorPtr makeSink( + const DB::ContextPtr & context, + const DB::Names & partition_by, + const DB::Block & input_header, + const DB::Block & output_header, + const std::string & base_path, + const std::string & filename, + const std::string & format_hint, + const std::shared_ptr & stats) +{ + if (partition_by.empty()) + { + auto file_sink = std::make_shared(context, base_path, "", filename, format_hint, input_header); + file_sink->setStats(stats); + return file_sink; + } + + auto file_sink = std::make_shared( + context, partition_by, input_header, output_header, base_path, filename, format_hint); + file_sink->setStats(stats); + return file_sink; +} + +bool need_fix_tuple(const DB::DataTypePtr& input, const DB::DataTypePtr& output) +{ + const auto orgial = typeid_cast(input.get()); + const auto output_type = typeid_cast(output.get()); + return orgial != nullptr && output_type != nullptr && !orgial->equals(*output_type); +} + +DB::ExpressionActionsPtr create_rename_action(const DB::Block & input, const DB::Block & output) +{ + DB::NamesWithAliases aliases; + for (auto ouput_name = output.begin(), input_iter = input.begin(); ouput_name != output.end(); ++ouput_name, ++input_iter) + aliases.emplace_back(DB::NameWithAlias(input_iter->name, ouput_name->name)); + + const auto actions_dag = std::make_shared(blockToNameAndTypeList(input)); + actions_dag->project(aliases); + return std::make_shared(actions_dag); +} + +DB::ExpressionActionsPtr create_project_action(const DB::Block & input, const DB::Block & output) +{ + DB::ColumnsWithTypeAndName final_cols; + std::ranges::transform( + output, std::back_inserter(final_cols), [](const DB::ColumnWithTypeAndName& out_ocl) { + const auto out_type = out_ocl.type; + return DB::ColumnWithTypeAndName(out_type->createColumn(), out_type, out_ocl.name); + }); + assert(final_cols.size() == output.columns()); + + const auto & original_cols = input.getColumnsWithTypeAndName(); + ActionsDAGPtr final_project = ActionsDAG::makeConvertingActions(original_cols, final_cols, ActionsDAG::MatchColumnsMode::Position); + return std::make_shared(final_project); +} + +void adjust_output(const DB::QueryPipelineBuilderPtr & builder, const DB::Block& output) +{ + const auto input = builder->getHeader(); + if (input.columns() != output.columns()) + { + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, + "Missmatch result columns size, input size is {}, but output size is {}", input.columns(), output.columns()); + } + + auto mismatch_pair = + std::mismatch(input.begin(), input.end(), output.begin(), + [](const DB::ColumnWithTypeAndName& lhs, const DB::ColumnWithTypeAndName& rhs) {return lhs.name == rhs.name;}); + bool name_is_diffient = mismatch_pair.first != input.end(); + + mismatch_pair = std::mismatch(input.begin(), input.end(), output.begin(), + [](const DB::ColumnWithTypeAndName& lhs, const DB::ColumnWithTypeAndName& rhs) + { + return lhs.type->equals(*rhs.type); + }); + bool type_is_diffient = mismatch_pair.first != input.end(); + + DB::ExpressionActionsPtr convert_action; + + if (type_is_diffient) + convert_action = create_project_action(input, output); + + if(name_is_diffient && !convert_action) + convert_action = create_rename_action(input, output); + + if (!convert_action) + return; + + builder->addSimpleTransform( + [&](const DB::Block & cur_header, const DB::QueryPipelineBuilder::StreamType stream_type) -> DB::ProcessorPtr + { + if (stream_type != DB::QueryPipelineBuilder::StreamType::Main) + return nullptr; + return std::make_shared(cur_header, convert_action); + }); +} + +namespace local_engine +{ + +void addSinkTransfrom(const DB::ContextPtr & context, const substrait::WriteRel & write_rel, const DB::QueryPipelineBuilderPtr & builder) +{ + const DB::Settings & settings = context->getSettingsRef(); + + DB::Field field_tmp_dir; + if (!settings.tryGet(SPARK_TASK_WRITE_TMEP_DIR, field_tmp_dir)) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Write Pipeline need inject temp directory."); + const auto & tmp_dir = field_tmp_dir.get(); + + DB::Field field_filename; + if (!settings.tryGet(SPARK_TASK_WRITE_FILENAME, field_filename)) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Write Pipeline need inject file name."); + const auto & filename = field_filename.get(); + + assert(write_rel.has_named_table()); + const substrait::NamedObjectWrite & named_table = write_rel.named_table(); + google::protobuf::StringValue optimization; + named_table.advanced_extension().optimization().UnpackTo(&optimization); + auto config = local_engine::parse_write_parameter(optimization.value()); + + //TODO : set compression codec according to format + assert(config["isSnappy"] == "1"); + assert(config.contains("format")); + + assert(write_rel.has_table_schema()); + const substrait::NamedStruct & table_schema = write_rel.table_schema(); + auto blockHeader = TypeParser::buildBlockFromNamedStruct(table_schema); + const auto partitionCols = collect_partition_cols(blockHeader, table_schema); + + auto stats = std::make_shared(blockHeader); + + adjust_output(builder, blockHeader); + + builder->addSimpleTransform( + [&](const Block & cur_header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr + { + if (stream_type != QueryPipelineBuilder::StreamType::Main) + return nullptr; + return makeSink(context, partitionCols, cur_header, blockHeader, tmp_dir, filename, config["format"], stats); + }); + builder->addSimpleTransform( + [&](const Block &, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr + { + if (stream_type != QueryPipelineBuilder::StreamType::Main) + return nullptr; + return stats; + }); +} + +std::map parse_write_parameter(const std::string & input) +{ + std::map reuslt; + const std::string prefix = "WriteParameters:"; + const size_t prefix_pos = input.find(prefix); + if (prefix_pos == std::string::npos) + return reuslt; + + const size_t start_pos = prefix_pos + prefix.length(); + const size_t end_pos = input.find('\n', start_pos); + + if (end_pos == std::string::npos) + return reuslt; + + for (const Poco::StringTokenizer tok(input.substr(start_pos, end_pos - start_pos), ";", Poco::StringTokenizer::TOK_TRIM); + const auto & parameter : tok) + { + const size_t pos = parameter.find('='); + if (pos == std::string::npos) + continue; + reuslt[parameter.substr(0, pos)] = parameter.substr(pos + 1); + } + return reuslt; +} + +DB::Names collect_partition_cols(const DB::Block & header, const substrait::NamedStruct & struct_) +{ + DB::Names result; + assert(struct_.column_types_size() == header.columns()); + assert(struct_.column_types_size() == struct_.struct_().types_size()); + + auto name_iter = header.begin(); + auto type_iter = struct_.column_types().begin(); + for (; name_iter != header.end(); ++name_iter, ++type_iter) + if (*type_iter == ::substrait::NamedStruct::PARTITION_COL) + result.push_back(name_iter->name); + return result; +} + +} \ No newline at end of file diff --git a/cpp-ch/local-engine/Parser/WriteRelParser.h b/cpp-ch/local-engine/Parser/WriteRelParser.h new file mode 100644 index 0000000000000..9d896e7ca53a8 --- /dev/null +++ b/cpp-ch/local-engine/Parser/WriteRelParser.h @@ -0,0 +1,46 @@ +/* + * 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. + */ +#pragma once + +#include +#include +#include +#include +#include + +namespace substrait +{ +class WriteRel; +class NamedStruct; +} + +namespace DB +{ +class QueryPipelineBuilder; +using QueryPipelineBuilderPtr = std::unique_ptr; +} + +namespace local_engine +{ + +void addSinkTransfrom(const DB::ContextPtr & context, const substrait::WriteRel & write_rel, const DB::QueryPipelineBuilderPtr & builder); + +/// Visible for UTs +std::map parse_write_parameter(const std::string & input); +DB::Names collect_partition_cols(const DB::Block & header, const substrait::NamedStruct & struct_); + +} diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/arithmetic.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/arithmetic.cpp index 7d8c3f948c631..b621798c3b308 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/arithmetic.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/arithmetic.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include namespace DB::ErrorCodes diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayPosition.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayPosition.cpp index d3eed7c67568d..528a80c075a64 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayPosition.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayPosition.cpp @@ -14,10 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#include +#include #include +#include +#include #include -#include namespace DB { diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/slice.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/slice.cpp index 2dca0cee182e2..46f00ce7cf552 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/slice.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/slice.cpp @@ -14,10 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#include -#include #include #include +#include +#include +#include namespace DB { diff --git a/cpp-ch/local-engine/Storages/Output/FileWriterWrappers.cpp b/cpp-ch/local-engine/Storages/Output/FileWriterWrappers.cpp index d0574e7ff2693..fc4b3a72f75b8 100644 --- a/cpp-ch/local-engine/Storages/Output/FileWriterWrappers.cpp +++ b/cpp-ch/local-engine/Storages/Output/FileWriterWrappers.cpp @@ -16,15 +16,15 @@ */ #include "FileWriterWrappers.h" #include -#include -#include -#include -#include namespace local_engine { -NormalFileWriter::NormalFileWriter(OutputFormatFilePtr file_, DB::ContextPtr context_) : FileWriterWrapper(file_), context(context_) +const std::string SubstraitFileSink::NO_PARTITION_ID{"__NO_PARTITION_ID__"}; +const std::string SubstraitPartitionedFileSink::DEFAULT_PARTITION_NAME{"__HIVE_DEFAULT_PARTITION__"}; + +NormalFileWriter::NormalFileWriter(const OutputFormatFilePtr & file_, const DB::ContextPtr & context_) + : FileWriterWrapper(file_), context(context_) { } @@ -54,8 +54,8 @@ void NormalFileWriter::close() writer->finish(); } -FileWriterWrapper * -createFileWriterWrapper(const std::string & file_uri, const std::vector & preferred_column_names, const std::string & format_hint) +OutputFormatFilePtr create_output_format_file( + const DB::ContextPtr & context, const std::string & file_uri, const DB::Names & preferred_column_names, const std::string & format_hint) { // the passed in file_uri is exactly what is expected to see in the output folder // e.g /xxx/中文/timestamp_field=2023-07-13 03%3A00%3A17.622/abc.parquet @@ -63,10 +63,14 @@ createFileWriterWrapper(const std::string & file_uri, const std::vector createFileWriterWrapper( + const DB::ContextPtr & context, const std::string & file_uri, const DB::Names & preferred_column_names, const std::string & format_hint) +{ + return std::make_unique(create_output_format_file(context, file_uri, preferred_column_names, format_hint), context); } } diff --git a/cpp-ch/local-engine/Storages/Output/FileWriterWrappers.h b/cpp-ch/local-engine/Storages/Output/FileWriterWrappers.h index 5c8061c5e5193..57cb47e41a559 100644 --- a/cpp-ch/local-engine/Storages/Output/FileWriterWrappers.h +++ b/cpp-ch/local-engine/Storages/Output/FileWriterWrappers.h @@ -16,19 +16,22 @@ */ #pragma once -#include #include -#include #include #include #include +#include +#include #include #include -#include +#include +#include #include -#include -#include +#include #include +#include +#include +#include namespace local_engine { @@ -36,7 +39,7 @@ namespace local_engine class FileWriterWrapper { public: - explicit FileWriterWrapper(OutputFormatFilePtr file_) : file(file_) { } + explicit FileWriterWrapper(const OutputFormatFilePtr & file_) : file(file_) { } virtual ~FileWriterWrapper() = default; virtual void consume(DB::Block & block) = 0; virtual void close() = 0; @@ -52,7 +55,7 @@ class NormalFileWriter : public FileWriterWrapper public: //TODO: EmptyFileReader and ConstColumnsFileReader ? //TODO: to support complex types - NormalFileWriter(OutputFormatFilePtr file_, DB::ContextPtr context_); + NormalFileWriter(const OutputFormatFilePtr & file_, const DB::ContextPtr & context_); ~NormalFileWriter() override = default; void consume(DB::Block & block) override; void close() override; @@ -65,6 +68,224 @@ class NormalFileWriter : public FileWriterWrapper std::unique_ptr writer; }; -FileWriterWrapper * -createFileWriterWrapper(const std::string & file_uri, const std::vector & preferred_column_names, const std::string & format_hint); +std::unique_ptr createFileWriterWrapper( + const DB::ContextPtr & context, + const std::string & file_uri, + const DB::Names & preferred_column_names, + const std::string & format_hint); + +OutputFormatFilePtr create_output_format_file( + const DB::ContextPtr & context, + const std::string & file_uri, + const DB::Names & preferred_column_names, + const std::string & format_hint); + +class WriteStats : public DB::ISimpleTransform +{ + bool all_chunks_processed_ = false; /// flag to determine if we have already processed all chunks + Arena partition_keys_arena_; + std::string filename_; + + absl::flat_hash_map fiel_to_count_; + + static Block statsHeader() + { + return makeBlockHeader({{STRING(), "filename"}, {STRING(), "partition_id"}, {BIGINT(), "record_count"}}); + } + + Chunk final_result() const + { + ///TODO: improve performance + auto file_col = STRING()->createColumn(); + auto partition_col = STRING()->createColumn(); + auto countCol = BIGINT()->createColumn(); + UInt64 num_rows = 0; + for (const auto & [relative_path, rows] : fiel_to_count_) + { + if (rows == 0) + continue; + file_col->insertData(filename_.c_str(), filename_.size()); + partition_col->insertData(relative_path.data, relative_path.size); + countCol->insert(rows); + num_rows++; + } + + const DB::Columns res_columns{std::move(file_col), std::move(partition_col), std::move(countCol)}; + return DB::Chunk(res_columns, num_rows); + } + +public: + explicit WriteStats(const Block & input_header_) : ISimpleTransform(input_header_, statsHeader(), true) { } + + Status prepare() override + { + if (input.isFinished() && !output.isFinished() && !has_input && !all_chunks_processed_) + { + all_chunks_processed_ = true; + /// return Ready to call transform() for generating filling rows after latest chunk was processed + return Status::Ready; + } + + return ISimpleTransform::prepare(); + } + + String getName() const override { return "WriteStats"; } + void transform(Chunk & chunk) override + { + if (all_chunks_processed_) + chunk = final_result(); + else + chunk = {}; + } + + void addFilePath(const String & patition_id, const String & filename) + { + assert(!filename.empty()); + + if (filename_.empty()) + filename_ = filename; + + assert(filename_ == filename); + + if (patition_id.empty()) + return; + fiel_to_count_.emplace(copyStringInArena(partition_keys_arena_, patition_id), 0); + } + + void collectStats(const String & file_path, size_t rows) + { + if (const auto it = fiel_to_count_.find(file_path); it != fiel_to_count_.end()) + { + it->second += rows; + return; + } + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "File path {} not found in the stats map", file_path); + } +}; + +class SubstraitFileSink final : public SinkToStorage +{ + const std::string partition_id_; + const std::string relative_path_; + OutputFormatFile::OutputFormatPtr output_format_; + std::shared_ptr stats_{nullptr}; + + static std::string makeFilename(const std::string & base_path, const std::string & partition_id, const std::string & relative) + { + if (partition_id.empty()) + return fmt::format("{}/{}", base_path, relative); + return fmt::format("{}/{}/{}", base_path, partition_id, relative); + } + +public: + /// visible for UTs + static const std::string NO_PARTITION_ID; + + explicit SubstraitFileSink( + const DB::ContextPtr & context, + const std::string & base_path, + const std::string & partition_id, + const std::string & relative, + const std::string & format_hint, + const Block & header) + : SinkToStorage(header) + , partition_id_(partition_id.empty() ? NO_PARTITION_ID : partition_id) + , relative_path_(relative) + , output_format_(create_output_format_file(context, makeFilename(base_path, partition_id, relative), header.getNames(), format_hint) + ->createOutputFormat(header)) + { + } + String getName() const override { return "SubstraitFileSink"; } + + ///TODO: remove this function + void setStats(const std::shared_ptr & stats) + { + stats_ = stats; + stats_->addFilePath(partition_id_, relative_path_); + } + +protected: + void consume(Chunk & chunk) override + { + const size_t row_count = chunk.getNumRows(); + output_format_->output->write(materializeBlock(getHeader().cloneWithColumns(chunk.detachColumns()))); + + if (stats_) + stats_->collectStats(partition_id_, row_count); + } + void onFinish() override + { + output_format_->output->finalize(); + output_format_->output->flush(); + output_format_->write_buffer->finalize(); + } +}; + +class SubstraitPartitionedFileSink final : public DB::PartitionedSink +{ + static const std::string DEFAULT_PARTITION_NAME; + +public: + /// visible for UTs + static ASTPtr make_partition_expression(const DB::Names & partition_columns) + { + /// Parse the following expression into ASTs + /// cancat('/col_name=', 'toString(col_name)') + bool add_slash = false; + ASTs arguments; + for (const auto & column : partition_columns) + { + // partition_column= + std::string key = add_slash ? fmt::format("/{}=", column) : fmt::format("{}=", column); + add_slash = true; + arguments.emplace_back(std::make_shared(key)); + + // ifNull(toString(partition_column), DEFAULT_PARTITION_NAME) + // FIXME if toString(partition_column) is empty + auto column_ast = std::make_shared(column); + ASTs if_null_args{makeASTFunction("toString", ASTs{column_ast}), std::make_shared(DEFAULT_PARTITION_NAME)}; + arguments.emplace_back(makeASTFunction("ifNull", std::move(if_null_args))); + } + return DB::makeASTFunction("concat", std::move(arguments)); + } + +private: + const std::string base_path_; + const std::string filenmame_; + ContextPtr context_; + const Block sample_block_; + const std::string format_hint_; + std::shared_ptr stats_{nullptr}; + +public: + SubstraitPartitionedFileSink( + const ContextPtr & context, + const Names & partition_by, + const Block & input_header, + const Block & sample_block, + const std::string & base_path, + const std::string & filename, + const std::string & format_hint) + : PartitionedSink(make_partition_expression(partition_by), context, input_header) + , base_path_(base_path) + , filenmame_(filename) + , context_(context) + , sample_block_(sample_block) + , format_hint_(format_hint) + { + } + SinkPtr createSinkForPartition(const String & partition_id) override + { + assert(stats_); + const auto partition_path = fmt::format("{}/{}", partition_id, filenmame_); + PartitionedSink::validatePartitionKey(partition_path, true); + auto file_sink = std::make_shared(context_, base_path_, partition_id, filenmame_, format_hint_, sample_block_); + file_sink->setStats(stats_); + return file_sink; + } + String getName() const override { return "SubstraitPartitionedFileSink"; } + + ///TODO: remove this function + void setStats(const std::shared_ptr & stats) { stats_ = stats; } +}; } diff --git a/cpp-ch/local-engine/Storages/Output/ParquetOutputFormatFile.cpp b/cpp-ch/local-engine/Storages/Output/ParquetOutputFormatFile.cpp index fc67c4260511a..6ef8b45246755 100644 --- a/cpp-ch/local-engine/Storages/Output/ParquetOutputFormatFile.cpp +++ b/cpp-ch/local-engine/Storages/Output/ParquetOutputFormatFile.cpp @@ -34,7 +34,7 @@ namespace local_engine ParquetOutputFormatFile::ParquetOutputFormatFile( DB::ContextPtr context_, const std::string & file_uri_, - WriteBufferBuilderPtr write_buffer_builder_, + const WriteBufferBuilderPtr & write_buffer_builder_, const std::vector & preferred_column_names_) : OutputFormatFile(context_, file_uri_, write_buffer_builder_, preferred_column_names_) { diff --git a/cpp-ch/local-engine/Storages/Output/ParquetOutputFormatFile.h b/cpp-ch/local-engine/Storages/Output/ParquetOutputFormatFile.h index 1b0c5abf5e603..13b731600938c 100644 --- a/cpp-ch/local-engine/Storages/Output/ParquetOutputFormatFile.h +++ b/cpp-ch/local-engine/Storages/Output/ParquetOutputFormatFile.h @@ -32,7 +32,7 @@ class ParquetOutputFormatFile : public OutputFormatFile explicit ParquetOutputFormatFile( DB::ContextPtr context_, const std::string & file_uri_, - WriteBufferBuilderPtr write_buffer_builder_, + const WriteBufferBuilderPtr & write_buffer_builder_, const std::vector & preferred_column_names_); ~ParquetOutputFormatFile() override = default; diff --git a/cpp-ch/local-engine/Storages/Output/WriteBufferBuilder.cpp b/cpp-ch/local-engine/Storages/Output/WriteBufferBuilder.cpp index 3dc205c39cbd7..ea93480b7683b 100644 --- a/cpp-ch/local-engine/Storages/Output/WriteBufferBuilder.cpp +++ b/cpp-ch/local-engine/Storages/Output/WriteBufferBuilder.cpp @@ -39,7 +39,7 @@ namespace local_engine class LocalFileWriteBufferBuilder : public WriteBufferBuilder { public: - explicit LocalFileWriteBufferBuilder(DB::ContextPtr context_) : WriteBufferBuilder(context_) { } + explicit LocalFileWriteBufferBuilder(const DB::ContextPtr & context_) : WriteBufferBuilder(context_) { } ~LocalFileWriteBufferBuilder() override = default; std::unique_ptr build(const std::string & file_uri_) override @@ -61,7 +61,7 @@ class LocalFileWriteBufferBuilder : public WriteBufferBuilder class HDFSFileWriteBufferBuilder : public WriteBufferBuilder { public: - explicit HDFSFileWriteBufferBuilder(DB::ContextPtr context_) : WriteBufferBuilder(context_) { } + explicit HDFSFileWriteBufferBuilder(const DB::ContextPtr & context_) : WriteBufferBuilder(context_) { } ~HDFSFileWriteBufferBuilder() override = default; std::unique_ptr build(const std::string & file_uri_) override @@ -81,8 +81,7 @@ class HDFSFileWriteBufferBuilder : public WriteBufferBuilder auto first = new_file_uri.find('/', new_file_uri.find("//") + 2); auto last = new_file_uri.find_last_of('/'); auto dir = new_file_uri.substr(first, last - first); - int err = hdfsCreateDirectory(fs.get(), dir.c_str()); - if (err) + if (hdfsCreateDirectory(fs.get(), dir.c_str())) throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Cannot create dir for {} because {}", dir, std::string(hdfsGetLastError())); DB::WriteSettings write_settings; @@ -106,7 +105,7 @@ WriteBufferBuilderFactory & WriteBufferBuilderFactory::instance() return instance; } -WriteBufferBuilderPtr WriteBufferBuilderFactory::createBuilder(const String & schema, DB::ContextPtr context) +WriteBufferBuilderPtr WriteBufferBuilderFactory::createBuilder(const String & schema, const DB::ContextPtr & context) { auto it = builders.find(schema); if (it == builders.end()) @@ -114,7 +113,7 @@ WriteBufferBuilderPtr WriteBufferBuilderFactory::createBuilder(const String & sc return it->second(context); } -void WriteBufferBuilderFactory::registerBuilder(const String & schema, NewBuilder newer) +void WriteBufferBuilderFactory::registerBuilder(const String & schema, const NewBuilder & newer) { auto it = builders.find(schema); if (it != builders.end()) diff --git a/cpp-ch/local-engine/Storages/Output/WriteBufferBuilder.h b/cpp-ch/local-engine/Storages/Output/WriteBufferBuilder.h index 2d59e69a0a068..a187aa5cc2f6d 100644 --- a/cpp-ch/local-engine/Storages/Output/WriteBufferBuilder.h +++ b/cpp-ch/local-engine/Storages/Output/WriteBufferBuilder.h @@ -27,7 +27,7 @@ namespace local_engine class WriteBufferBuilder { public: - explicit WriteBufferBuilder(DB::ContextPtr context_) : context(context_) { } + explicit WriteBufferBuilder(const DB::ContextPtr & context_) : context(context_) { } virtual ~WriteBufferBuilder() = default; /// build a new write buffer virtual std::unique_ptr build(const std::string & file_uri_) = 0; @@ -43,9 +43,9 @@ class WriteBufferBuilderFactory : public boost::noncopyable public: using NewBuilder = std::function; static WriteBufferBuilderFactory & instance(); - WriteBufferBuilderPtr createBuilder(const String & schema, DB::ContextPtr context); + WriteBufferBuilderPtr createBuilder(const String & schema, const DB::ContextPtr & context); - void registerBuilder(const String & schema, NewBuilder newer); + void registerBuilder(const String & schema, const NewBuilder & newer); private: std::map builders; diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.h b/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.h index d9c485ed1015a..e67259ce59cc4 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.h +++ b/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.h @@ -20,14 +20,21 @@ #include #include + #include #include + #include + #include + #include + #include #include + + namespace DB { namespace ErrorCodes diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/JSONFormatFile.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/JSONFormatFile.cpp index d51ed08c6e2d7..b3e47681a8897 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/JSONFormatFile.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/JSONFormatFile.cpp @@ -16,6 +16,7 @@ */ #include "JSONFormatFile.h" + #include #include #include diff --git a/cpp-ch/local-engine/jni/jni_common.h b/cpp-ch/local-engine/jni/jni_common.h index c1e0fbead5357..e072d124ca72c 100644 --- a/cpp-ch/local-engine/jni/jni_common.h +++ b/cpp-ch/local-engine/jni/jni_common.h @@ -204,11 +204,11 @@ class SafeNativeArray { SafeNativeArray& operator=(const SafeNativeArray&) = delete; SafeNativeArray& operator=(SafeNativeArray&&) = delete; - const NativeArrayType elems() const { + NativeArrayType elems() const { return reinterpret_cast(nativeArray_); } - const jsize length() const { + jsize length() const { return env_->GetArrayLength(javaArray_); } diff --git a/cpp-ch/local-engine/local_engine_jni.cpp b/cpp-ch/local-engine/local_engine_jni.cpp index 627e6154cdcf7..17d087bb82ff4 100644 --- a/cpp-ch/local-engine/local_engine_jni.cpp +++ b/cpp-ch/local-engine/local_engine_jni.cpp @@ -27,6 +27,7 @@ #include #include #include +#include #include #include #include @@ -77,16 +78,13 @@ static std::string jstring2string(JNIEnv * env, jstring jStr) if (!jStr) return ""; - jclass string_class = env->GetObjectClass(jStr); - jmethodID get_bytes = env->GetMethodID(string_class, "getBytes", "(Ljava/lang/String;)[B"); - jbyteArray string_jbytes + const jclass string_class = env->GetObjectClass(jStr); + const jmethodID get_bytes = env->GetMethodID(string_class, "getBytes", "(Ljava/lang/String;)[B"); + const auto string_jbytes = static_cast(local_engine::safeCallObjectMethod(env, jStr, get_bytes, env->NewStringUTF("UTF-8"))); - size_t length = static_cast(env->GetArrayLength(string_jbytes)); - jbyte * p_bytes = env->GetByteArrayElements(string_jbytes, nullptr); - - std::string ret = std::string(reinterpret_cast(p_bytes), length); - env->ReleaseByteArrayElements(string_jbytes, p_bytes, JNI_ABORT); + const auto string_jbytes_a = local_engine::getByteArrayElementsSafe(env, string_jbytes); + std::string ret{reinterpret_cast(string_jbytes_a.elems()), static_cast(string_jbytes_a.length())}; env->DeleteLocalRef(string_jbytes); env->DeleteLocalRef(string_class); @@ -203,7 +201,7 @@ JNIEXPORT void JNI_OnUnload(JavaVM * vm, void * /*reserved*/) env->DeleteGlobalRef(local_engine::ReservationListenerWrapper::reservation_listener_class); } -JNIEXPORT void Java_org_apache_gluten_vectorized_ExpressionEvaluatorJniWrapper_nativeInitNative(JNIEnv * env, jobject, jbyteArray conf_plan) +JNIEXPORT void Java_org_apache_gluten_vectorized_ExpressionEvaluatorJniWrapper_nativeInitNative(JNIEnv * env, jclass, jbyteArray conf_plan) { LOCAL_ENGINE_JNI_METHOD_START const auto conf_plan_a = local_engine::getByteArrayElementsSafe(env, conf_plan); @@ -212,16 +210,33 @@ JNIEXPORT void Java_org_apache_gluten_vectorized_ExpressionEvaluatorJniWrapper_n LOCAL_ENGINE_JNI_METHOD_END(env, ) } -JNIEXPORT void Java_org_apache_gluten_vectorized_ExpressionEvaluatorJniWrapper_nativeFinalizeNative(JNIEnv * env) +JNIEXPORT void Java_org_apache_gluten_vectorized_ExpressionEvaluatorJniWrapper_nativeFinalizeNative(JNIEnv * env, jclass) { LOCAL_ENGINE_JNI_METHOD_START local_engine::BackendFinalizerUtil::finalizeSessionally(); LOCAL_ENGINE_JNI_METHOD_END(env, ) } +JNIEXPORT void Java_org_apache_gluten_vectorized_ExpressionEvaluatorJniWrapper_injectWriteFilesTempPath( + JNIEnv * env, jclass, jlong allocator_id, jbyteArray temp_path, jbyteArray filename) +{ + LOCAL_ENGINE_JNI_METHOD_START + const auto query_context = local_engine::getAllocator(allocator_id)->query_context; + + const auto path_array = local_engine::getByteArrayElementsSafe(env, temp_path); + const std::string c_path{reinterpret_cast(path_array.elems()), static_cast(path_array.length())}; + query_context->setSetting(local_engine::SPARK_TASK_WRITE_TMEP_DIR, c_path); + + const auto filename_array = local_engine::getByteArrayElementsSafe(env, filename); + const std::string c_filename{reinterpret_cast(filename_array.elems()), static_cast(filename_array.length())}; + query_context->setSetting(local_engine::SPARK_TASK_WRITE_FILENAME, c_filename); + + LOCAL_ENGINE_JNI_METHOD_END(env, ) +} + JNIEXPORT jlong Java_org_apache_gluten_vectorized_ExpressionEvaluatorJniWrapper_nativeCreateKernelWithIterator( JNIEnv * env, - jobject /*obj*/, + jclass , jlong allocator_id, jbyteArray plan, jobjectArray split_infos, @@ -257,8 +272,7 @@ JNIEXPORT jlong Java_org_apache_gluten_vectorized_ExpressionEvaluatorJniWrapper_ const auto plan_a = local_engine::getByteArrayElementsSafe(env, plan); const std::string::size_type plan_size = plan_a.length(); - local_engine::LocalExecutor * executor - = parser.createExecutor({reinterpret_cast(plan_a.elems()), plan_size}).release(); + local_engine::LocalExecutor * executor = parser.createExecutor({reinterpret_cast(plan_a.elems()), plan_size}).release(); LOG_INFO(&Poco::Logger::get("jni"), "Construct LocalExecutor {}", reinterpret_cast(executor)); executor->setMetric(parser.getMetric()); executor->setExtraPlanHolder(parser.extra_plan_holder); @@ -281,14 +295,14 @@ JNIEXPORT jlong Java_org_apache_gluten_vectorized_BatchIterator_nativeCHNext(JNI LOCAL_ENGINE_JNI_METHOD_START local_engine::LocalExecutor * executor = reinterpret_cast(executor_address); DB::Block * column_batch = executor->nextColumnar(); - return reinterpret_cast(column_batch); + return reinterpret_cast(column_batch); LOCAL_ENGINE_JNI_METHOD_END(env, -1) } JNIEXPORT void Java_org_apache_gluten_vectorized_BatchIterator_nativeCancel(JNIEnv * env, jobject /*obj*/, jlong executor_address) { LOCAL_ENGINE_JNI_METHOD_START - auto *executor = reinterpret_cast(executor_address); + auto * executor = reinterpret_cast(executor_address); executor->cancel(); LOG_INFO(&Poco::Logger::get("jni"), "Cancel LocalExecutor {}", reinterpret_cast(executor)); LOCAL_ENGINE_JNI_METHOD_END(env, ) @@ -297,7 +311,7 @@ JNIEXPORT void Java_org_apache_gluten_vectorized_BatchIterator_nativeCancel(JNIE JNIEXPORT void Java_org_apache_gluten_vectorized_BatchIterator_nativeClose(JNIEnv * env, jobject /*obj*/, jlong executor_address) { LOCAL_ENGINE_JNI_METHOD_START - auto *executor = reinterpret_cast(executor_address); + auto * executor = reinterpret_cast(executor_address); LOG_INFO(&Poco::Logger::get("jni"), "Finalize LocalExecutor {}", reinterpret_cast(executor)); delete executor; LOCAL_ENGINE_JNI_METHOD_END(env, ) @@ -805,8 +819,7 @@ JNIEXPORT jlong Java_org_apache_gluten_vectorized_CHBlockConverterJniWrapper_con JNIEXPORT void Java_org_apache_gluten_vectorized_CHBlockConverterJniWrapper_freeBlock(JNIEnv * env, jclass, jlong block_address) { LOCAL_ENGINE_JNI_METHOD_START - local_engine::SparkRowToCHColumn converter; - converter.freeBlock(reinterpret_cast(block_address)); + local_engine::SparkRowToCHColumn::freeBlock(reinterpret_cast(block_address)); LOCAL_ENGINE_JNI_METHOD_END(env, ) } @@ -858,8 +871,8 @@ JNIEXPORT jlong Java_org_apache_spark_sql_execution_datasources_CHDatasourceJniW JNIEnv * env, jobject, jstring file_uri_, jobjectArray names_, jstring format_hint_) { LOCAL_ENGINE_JNI_METHOD_START - int num_columns = env->GetArrayLength(names_); - std::vector names; + const int num_columns = env->GetArrayLength(names_); + DB::Names names; names.reserve(num_columns); for (int i = 0; i < num_columns; i++) { @@ -867,10 +880,11 @@ JNIEXPORT jlong Java_org_apache_spark_sql_execution_datasources_CHDatasourceJniW names.emplace_back(jstring2string(env, name)); env->DeleteLocalRef(name); } - auto file_uri = jstring2string(env, file_uri_); - auto format_hint = jstring2string(env, format_hint_); + const auto file_uri = jstring2string(env, file_uri_); + const auto format_hint = jstring2string(env, format_hint_); // for HiveFileFormat, the file url may not end with .parquet, so we pass in the format as a hint - auto * writer = local_engine::createFileWriterWrapper(file_uri, names, format_hint); + const auto context = DB::Context::createCopy(local_engine::SerializedPlanParser::global_context); + auto * writer = local_engine::createFileWriterWrapper(context, file_uri, names, format_hint).release(); return reinterpret_cast(writer); LOCAL_ENGINE_JNI_METHOD_END(env, 0) } @@ -901,23 +915,12 @@ JNIEXPORT jlong Java_org_apache_spark_sql_execution_datasources_CHDatasourceJniW const auto bucket_dir = jstring2string(env, bucket_dir_); const auto plan_a = local_engine::getByteArrayElementsSafe(env, plan_); - - /// https://stackoverflow.com/questions/52028583/getting-error-parsing-protobuf-data - /// Parsing may fail when the number of recursive layers is large. - /// Here, set a limit large enough to avoid this problem. - /// Once this problem occurs, it is difficult to troubleshoot, because the pb of c++ will not provide any valid information - google::protobuf::io::CodedInputStream coded_in(plan_a.elems(), plan_a.length()); - coded_in.SetRecursionLimit(100000); - - substrait::Plan plan_ptr; - if (!plan_ptr.ParseFromCodedStream(&coded_in)) - throw DB::Exception(DB::ErrorCodes::CANNOT_PARSE_PROTOBUF_SCHEMA, "Parse substrait::Plan from string failed"); + auto plan_ptr = local_engine::BinaryToMessage( + {reinterpret_cast(plan_a.elems()), static_cast(plan_a.length())}); const auto split_info_a = local_engine::getByteArrayElementsSafe(env, split_info_); - const std::string::size_type split_info_size = split_info_a.length(); - std::string split_info_str{reinterpret_cast(split_info_a.elems()), split_info_size}; - - substrait::ReadRel::ExtensionTable extension_table = local_engine::SerializedPlanParser::parseExtensionTable(split_info_str); + auto extension_table = local_engine::BinaryToMessage( + {reinterpret_cast(split_info_a.elems()), static_cast(split_info_a.length())}); auto merge_tree_table = local_engine::MergeTreeRelParser::parseMergeTreeTable(extension_table); auto uuid = uuid_str + "_" + task_id; @@ -933,24 +936,12 @@ JNIEXPORT jstring Java_org_apache_spark_sql_execution_datasources_CHDatasourceJn { LOCAL_ENGINE_JNI_METHOD_START const auto plan_a = local_engine::getByteArrayElementsSafe(env, plan_); - const std::string::size_type plan_size = plan_a.length(); - - substrait::Plan plan_ptr; - if (!plan_ptr.ParseFromString({reinterpret_cast(plan_a.elems()), plan_size})) - throw Exception(DB::ErrorCodes::CANNOT_PARSE_PROTOBUF_SCHEMA, "Parse substrait::Plan from string failed"); + auto plan_ptr = local_engine::BinaryToMessage( + {reinterpret_cast(plan_a.elems()), static_cast(plan_a.length())}); const auto read_a = local_engine::getByteArrayElementsSafe(env, read_); - /// https://stackoverflow.com/questions/52028583/getting-error-parsing-protobuf-data - /// Parsing may fail when the number of recursive layers is large. - /// Here, set a limit large enough to avoid this problem. - /// Once this problem occurs, it is difficult to troubleshoot, because the pb of c++ will not provide any valid information - google::protobuf::io::CodedInputStream coded_in(read_a.elems(), read_a.length()); - coded_in.SetRecursionLimit(100000); - - substrait::Rel read_ptr; - if (!read_ptr.ParseFromCodedStream(&coded_in)) - throw Exception(DB::ErrorCodes::CANNOT_PARSE_PROTOBUF_SCHEMA, "Parse substrait::Expression from string failed"); - + auto read_ptr = local_engine::BinaryToMessage( + {reinterpret_cast(read_a.elems()), static_cast(read_a.length())}); local_engine::SerializedPlanParser parser(local_engine::SerializedPlanParser::global_context); parser.parseExtensions(plan_ptr.extensions()); @@ -1026,23 +1017,13 @@ JNIEXPORT jstring Java_org_apache_spark_sql_execution_datasources_CHDatasourceJn const auto bucket_dir = jstring2string(env, bucket_dir_); const auto plan_a = local_engine::getByteArrayElementsSafe(env, plan_); - - /// https://stackoverflow.com/questions/52028583/getting-error-parsing-protobuf-data - /// Parsing may fail when the number of recursive layers is large. - /// Here, set a limit large enough to avoid this problem. - /// Once this problem occurs, it is difficult to troubleshoot, because the pb of c++ will not provide any valid information - google::protobuf::io::CodedInputStream coded_in(plan_a.elems(), plan_a.length()); - coded_in.SetRecursionLimit(100000); - - substrait::Plan plan_ptr; - if (!plan_ptr.ParseFromCodedStream(&coded_in)) - throw DB::Exception(DB::ErrorCodes::CANNOT_PARSE_PROTOBUF_SCHEMA, "Parse substrait::Plan from string failed"); + auto plan_ptr = local_engine::BinaryToMessage( + {reinterpret_cast(plan_a.elems()), static_cast(plan_a.length())}); const auto split_info_a = local_engine::getByteArrayElementsSafe(env, split_info_); - const std::string::size_type split_info_size = split_info_a.length(); - std::string split_info_str{reinterpret_cast(split_info_a.elems()), split_info_size}; + auto extension_table = local_engine::BinaryToMessage( + {reinterpret_cast(split_info_a.elems()), static_cast(split_info_a.length())}); - substrait::ReadRel::ExtensionTable extension_table = local_engine::SerializedPlanParser::parseExtensionTable(split_info_str); google::protobuf::StringValue table; table.ParseFromString(extension_table.detail().value()); auto merge_tree_table = local_engine::parseMergeTreeTableString(table.value()); @@ -1251,14 +1232,13 @@ JNIEXPORT jlong Java_org_apache_gluten_vectorized_SimpleExpressionEval_createNativeInstance(JNIEnv * env, jclass, jobject input, jbyteArray plan) { LOCAL_ENGINE_JNI_METHOD_START - auto context = DB::Context::createCopy(local_engine::SerializedPlanParser::global_context); + const auto context = DB::Context::createCopy(local_engine::SerializedPlanParser::global_context); local_engine::SerializedPlanParser parser(context); - jobject iter = env->NewGlobalRef(input); + const jobject iter = env->NewGlobalRef(input); parser.addInputIter(iter, false); const auto plan_a = local_engine::getByteArrayElementsSafe(env, plan); const std::string::size_type plan_size = plan_a.length(); - local_engine::LocalExecutor * executor - = parser.createExecutor({reinterpret_cast(plan_a.elems()), plan_size}).release(); + local_engine::LocalExecutor * executor = parser.createExecutor({reinterpret_cast(plan_a.elems()), plan_size}).release(); return reinterpret_cast(executor); LOCAL_ENGINE_JNI_METHOD_END(env, -1) } diff --git a/cpp-ch/local-engine/tests/CMakeLists.txt b/cpp-ch/local-engine/tests/CMakeLists.txt index be02bf6234d28..f9dd36c1288dd 100644 --- a/cpp-ch/local-engine/tests/CMakeLists.txt +++ b/cpp-ch/local-engine/tests/CMakeLists.txt @@ -12,6 +12,11 @@ # 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. + +file(REAL_PATH "${CMAKE_CURRENT_SOURCE_DIR}" GTEST_REAL_PATH) +file(REAL_PATH "${GTEST_REAL_PATH}/../../.." GLUTEN_REAL_PATH) # used in + # testConfig.h.in + configure_file(${CMAKE_CURRENT_SOURCE_DIR}/testConfig.h.in ${CMAKE_CURRENT_SOURCE_DIR}/testConfig.h) diff --git a/cpp-ch/local-engine/tests/benchmark_local_engine.cpp b/cpp-ch/local-engine/tests/benchmark_local_engine.cpp index 43cdab8a41fa4..cf9ecf37dd30d 100644 --- a/cpp-ch/local-engine/tests/benchmark_local_engine.cpp +++ b/cpp-ch/local-engine/tests/benchmark_local_engine.cpp @@ -234,7 +234,7 @@ DB::ContextMutablePtr global_context; std::ifstream t(path); std::string str((std::istreambuf_iterator(t)), std::istreambuf_iterator()); std::cout << "the plan from: " << path << std::endl; - auto local_executor = parser.createExecutor(str); + auto local_executor = parser.createExecutor(str); state.ResumeTiming(); while (local_executor->hasNext()) [[maybe_unused]] auto * x = local_executor->nextColumnar(); @@ -585,8 +585,7 @@ DB::ContextMutablePtr global_context; readIntBinary(x, buf); readIntBinary(y, buf); readIntBinary(z, buf); - std::cout << std::to_string(x) + " " << std::to_string(y) + " " << std::to_string(z) + " " - << "\n"; + std::cout << std::to_string(x) + " " << std::to_string(y) + " " << std::to_string(z) + " " << "\n"; data_buf.seek(x, SEEK_SET); assert(!data_buf.eof()); std::string data; diff --git a/cpp-ch/local-engine/tests/gluten_test_util.cpp b/cpp-ch/local-engine/tests/gluten_test_util.cpp index 0448092b960dd..1f1bd99836967 100644 --- a/cpp-ch/local-engine/tests/gluten_test_util.cpp +++ b/cpp-ch/local-engine/tests/gluten_test_util.cpp @@ -28,6 +28,7 @@ #include #include #include +#include #include namespace fs = std::filesystem; diff --git a/cpp-ch/local-engine/tests/gluten_test_util.h b/cpp-ch/local-engine/tests/gluten_test_util.h index 338d53be788a8..34e05b8b188b7 100644 --- a/cpp-ch/local-engine/tests/gluten_test_util.h +++ b/cpp-ch/local-engine/tests/gluten_test_util.h @@ -18,6 +18,7 @@ #pragma once #include +#include #include #include #include @@ -25,7 +26,6 @@ #include #include #include -#include #include using BlockRowType = DB::ColumnsWithTypeAndName; @@ -65,23 +65,6 @@ AnotherRowType readParquetSchema(const std::string & file); DB::ActionsDAGPtr parseFilter(const std::string & filter, const AnotherRowType & name_and_types); -namespace pb_util -{ -template -std::string JsonStringToBinary(const std::string_view & json) -{ - Message message; - std::string binary; - auto s = google::protobuf::util::JsonStringToMessage(json, &message); - if (!s.ok()) - { - const std::string err_msg{s.message()}; - throw std::runtime_error(err_msg); - } - message.SerializeToString(&binary); - return binary; -} -} } inline std::string replaceLocalFilesWildcards(const String & haystack, const String & replaced) @@ -90,54 +73,6 @@ inline std::string replaceLocalFilesWildcards(const String & haystack, const Str return boost::replace_all_copy(haystack, _WILDCARD_, replaced); } -inline DB::DataTypePtr BIGINT() -{ - return std::make_shared(); -} -inline DB::DataTypePtr INT() -{ - return std::make_shared(); -} -inline DB::DataTypePtr INT16() -{ - return std::make_shared(); -} -inline DB::DataTypePtr INT8() -{ - return std::make_shared(); -} -inline DB::DataTypePtr UBIGINT() -{ - return std::make_shared(); -} -inline DB::DataTypePtr UINT() -{ - return std::make_shared(); -} -inline DB::DataTypePtr UINT16() -{ - return std::make_shared(); -} -inline DB::DataTypePtr UINT8() -{ - return std::make_shared(); -} - -inline DB::DataTypePtr DOUBLE() -{ - return std::make_shared(); -} - -inline DB::DataTypePtr STRING() -{ - return std::make_shared(); -} - -inline DB::DataTypePtr DATE() -{ - return std::make_shared(); -} - inline BlockFieldType toBlockFieldType(const AnotherFieldType & type) { return BlockFieldType(type.type, type.name); @@ -148,6 +83,17 @@ inline AnotherFieldType toAnotherFieldType(const parquet::ColumnDescriptor & typ return {type.name(), local_engine::test::toDataType(type)}; } +inline AnotherRowType toAnotherRowType(const DB::Block & header) +{ + AnotherRowType types; + for (const auto & name : header.getNames()) + { + const auto * column = header.findByName(name); + types.push_back(DB::NameAndTypePair(column->name, column->type)); + } + return types; +} + inline BlockRowType toBlockRowType(const AnotherRowType & type, const bool reverse = false) { BlockRowType result; diff --git a/cpp-ch/local-engine/tests/gtest_clickhouse_54881.cpp b/cpp-ch/local-engine/tests/gtest_clickhouse_pr_verify.cpp similarity index 74% rename from cpp-ch/local-engine/tests/gtest_clickhouse_54881.cpp rename to cpp-ch/local-engine/tests/gtest_clickhouse_pr_verify.cpp index 69966ef93c0b1..6352a819927c6 100644 --- a/cpp-ch/local-engine/tests/gtest_clickhouse_54881.cpp +++ b/cpp-ch/local-engine/tests/gtest_clickhouse_pr_verify.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include @@ -42,11 +43,13 @@ TEST(Clickhouse, PR54881) = replaceLocalFilesWildcards(split_template, GLUTEN_DATA_DIR("/utils/extern-local-engine/tests/data/54881.snappy.parquet")); SerializedPlanParser parser(context1); - parser.addSplitInfo(test::pb_util::JsonStringToBinary(split)); + parser.addSplitInfo(local_engine::JsonStringToBinary(split)); - const auto local_executor = parser.createExecutor( + const auto plan = local_engine::JsonStringToMessage( {reinterpret_cast(gresource_embedded_pr_54881_jsonData), gresource_embedded_pr_54881_jsonSize}); + auto local_executor = parser.createExecutor(plan); + EXPECT_TRUE(local_executor->hasNext()); const Block & block = *local_executor->nextColumnar(); @@ -81,3 +84,16 @@ TEST(Clickhouse, PR54881) EXPECT_FALSE(local_executor->hasNext()); } + +// Plan for https://github.com/ClickHouse/ClickHouse/pull/65234 +INCBIN(resource_embedded_pr_65234_json, SOURCE_DIR "/utils/extern-local-engine/tests/json/clickhouse_pr_65234.json"); + +TEST(Clickhouse, PR65234) +{ + const std::string split = R"({"items":[{"uriFile":"file:///foo","length":"84633","parquet":{},"schema":{},"metadataColumns":[{}]}]})"; + SerializedPlanParser parser(SerializedPlanParser::global_context); + parser.addSplitInfo(local_engine::JsonStringToBinary(split)); + const auto plan = local_engine::JsonStringToMessage( + {reinterpret_cast(gresource_embedded_pr_65234_jsonData), gresource_embedded_pr_65234_jsonSize}); + auto query_plan = parser.parse(plan); +} \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/gtest_local_engine.cpp b/cpp-ch/local-engine/tests/gtest_local_engine.cpp index 962bf9def52e3..50e527afc8d5b 100644 --- a/cpp-ch/local-engine/tests/gtest_local_engine.cpp +++ b/cpp-ch/local-engine/tests/gtest_local_engine.cpp @@ -26,6 +26,7 @@ #include #include #include +#include #include #include #include @@ -95,7 +96,7 @@ void registerOutputFormatParquet(DB::FormatFactory & factory); int main(int argc, char ** argv) { - BackendInitializerUtil::init(test::pb_util::JsonStringToBinary( + BackendInitializerUtil::init(local_engine::JsonStringToBinary( {reinterpret_cast(gresource_embedded_config_jsonData), gresource_embedded_config_jsonSize})); auto & factory = FormatFactory::instance(); diff --git a/cpp-ch/local-engine/tests/gtest_parquet_columnindex.cpp b/cpp-ch/local-engine/tests/gtest_parquet_columnindex.cpp index 76c0d028a58e2..fbd7fbc63c270 100644 --- a/cpp-ch/local-engine/tests/gtest_parquet_columnindex.cpp +++ b/cpp-ch/local-engine/tests/gtest_parquet_columnindex.cpp @@ -15,13 +15,12 @@ * limitations under the License. */ -#include - #include "config.h" #if USE_PARQUET #include #include +#include #include #include #include @@ -29,6 +28,7 @@ #include #include +#include #include #include #include @@ -470,7 +470,9 @@ TEST(ColumnIndex, FilteringWithAllNullPages) } TEST(ColumnIndex, FilteringWithNotFoundColumnName) { + using namespace test_utils; + using namespace local_engine; const local_engine::ColumnIndexStore column_index_store = buildTestColumnIndexStore(); { @@ -1040,6 +1042,7 @@ TEST_P(TestBuildPageReadStates, BuildPageReadStates) TEST(ColumnIndex, VectorizedParquetRecordReader) { + using namespace local_engine; //TODO: move test parquet to s3 and download to CI machine. const std::string filename = "/home/chang/test/tpch/parquet/Index/60001/part-00000-76ef9b89-f292-495f-9d0d-98325f3d8956-c000.snappy.parquet"; diff --git a/cpp-ch/local-engine/tests/gtest_parquet_read.cpp b/cpp-ch/local-engine/tests/gtest_parquet_read.cpp index 9623ffa98d281..e113438344596 100644 --- a/cpp-ch/local-engine/tests/gtest_parquet_read.cpp +++ b/cpp-ch/local-engine/tests/gtest_parquet_read.cpp @@ -15,15 +15,13 @@ * limitations under the License. */ -#include - - #include "config.h" #if USE_PARQUET #include #include +#include #include #include #include @@ -41,6 +39,7 @@ #include #include #include +#include #include #include #include @@ -426,7 +425,7 @@ TEST(ParquetRead, LowLevelRead) TEST(ParquetRead, VectorizedColumnReader) { const std::string sample(local_engine::test::data_file("sample.parquet")); - Block blockHeader({{DOUBLE(), "b"}, {BIGINT(), "a"}}); + Block blockHeader({{local_engine::DOUBLE(), "b"}, {local_engine::BIGINT(), "a"}}); ReadBufferFromFile in(sample); const FormatSettings format_settings{}; auto arrow_file = local_engine::test::asArrowFileForParquet(in, format_settings); diff --git a/cpp-ch/local-engine/tests/gtest_parser.cpp b/cpp-ch/local-engine/tests/gtest_parser.cpp index aaaa3679ab94e..135f81a9149e7 100644 --- a/cpp-ch/local-engine/tests/gtest_parser.cpp +++ b/cpp-ch/local-engine/tests/gtest_parser.cpp @@ -16,141 +16,86 @@ */ #include #include +#include #include #include +#include +#include +#include +#include #include +#include +#include using namespace local_engine; using namespace DB; -// Plan for https://github.com/ClickHouse/ClickHouse/pull/65234 -INCBIN(resource_embedded_pr_65234_json, SOURCE_DIR "/utils/extern-local-engine/tests/json/clickhouse_pr_65234.json"); -TEST(SerializedPlanParser, PR65234) +INCBIN(resource_embedded_readcsv_json, SOURCE_DIR "/utils/extern-local-engine/tests/json/read_student_option_schema.csv.json"); +TEST(LocalExecutor, ReadCSV) { - const std::string split - = R"({"items":[{"uriFile":"file:///home/chang/SourceCode/rebase_gluten/backends-clickhouse/target/scala-2.12/test-classes/tests-working-home/tpch-data/supplier/part-00000-16caa751-9774-470c-bd37-5c84c53373c8-c000.snappy.parquet","length":"84633","parquet":{},"schema":{},"metadataColumns":[{}]}]})"; + const std::string split_template + = R"({"items":[{"uriFile":"{replace_local_files}","length":"56","text":{"fieldDelimiter":",","maxBlockSize":"8192","header":"1"},"schema":{"names":["id","name","language"],"struct":{"types":[{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}}]}},"metadataColumns":[{}]}]})"; + const std::string split = replaceLocalFilesWildcards( + split_template, GLUTEN_SOURCE_DIR("/backends-velox/src/test/resources/datasource/csv/student_option_schema.csv")); SerializedPlanParser parser(SerializedPlanParser::global_context); - parser.addSplitInfo(test::pb_util::JsonStringToBinary(split)); - auto query_plan - = parser.parseJson({reinterpret_cast(gresource_embedded_pr_65234_jsonData), gresource_embedded_pr_65234_jsonSize}); -} - -#include -#include -#include -#include -#include - -Chunk testChunk() -{ - auto nameCol = STRING()->createColumn(); - nameCol->insert("one"); - nameCol->insert("two"); - nameCol->insert("three"); - - auto valueCol = UINT()->createColumn(); - valueCol->insert(1); - valueCol->insert(2); - valueCol->insert(3); - MutableColumns x; - x.push_back(std::move(nameCol)); - x.push_back(std::move(valueCol)); - return {std::move(x), 3}; -} - -TEST(LocalExecutor, StorageObjectStorageSink) -{ - /// 0. Create ObjectStorage for HDFS - auto settings = SerializedPlanParser::global_context->getSettingsRef(); - const std::string query - = R"(CREATE TABLE hdfs_engine_xxxx (name String, value UInt32) ENGINE=HDFS('hdfs://localhost:8020/clickhouse/test2', 'Parquet'))"; - DB::ParserCreateQuery parser; - std::string error_message; - const char * pos = query.data(); - auto ast = DB::tryParseQuery( - parser, - pos, - pos + query.size(), - error_message, - /* hilite = */ false, - "QUERY TEST", - /* allow_multi_statements = */ false, - 0, - settings.max_parser_depth, - settings.max_parser_backtracks, - true); - auto & create = ast->as(); - auto arg = create.storage->children[0]; - const auto * func = arg->as(); - EXPECT_TRUE(func && func->name == "HDFS"); - - DB::StorageHDFSConfiguration config; - StorageObjectStorage::Configuration::initialize(config, arg->children[0]->children, SerializedPlanParser::global_context, false); - - const std::shared_ptr object_storage - = std::dynamic_pointer_cast(config.createObjectStorage(SerializedPlanParser::global_context, false)); - EXPECT_TRUE(object_storage != nullptr); - - RelativePathsWithMetadata files_with_metadata; - object_storage->listObjects("/clickhouse", files_with_metadata, 0); - - /// 1. Create ObjectStorageSink - DB::StorageObjectStorageSink sink{ - object_storage, config.clone(), {}, {{STRING(), "name"}, {UINT(), "value"}}, SerializedPlanParser::global_context, ""}; - - /// 2. Create Chunk - /// 3. comsume - sink.consume(testChunk()); - sink.onFinish(); -} + parser.addSplitInfo(local_engine::JsonStringToBinary(split)); + auto plan = local_engine::JsonStringToMessage( + {reinterpret_cast(gresource_embedded_readcsv_jsonData), gresource_embedded_readcsv_jsonSize}); -namespace DB -{ -SinkToStoragePtr createFilelinkSink( - const StorageMetadataPtr & metadata_snapshot, - const String & table_name_for_log, - const String & path, - CompressionMethod compression_method, - const std::optional & format_settings, - const String & format_name, - const ContextPtr & context, - int flags); + auto query_plan = parser.parse(plan); + const auto pipeline = parser.buildQueryPipeline(*query_plan); + LocalExecutor local_executor{std::move(query_plan), QueryPipelineBuilder::getPipeline(std::move(*pipeline))}; + EXPECT_TRUE(local_executor.hasNext()); + const Block & x = *local_executor.nextColumnar(); + EXPECT_EQ(4, x.rows()); } -INCBIN(resource_embedded_readcsv_json, SOURCE_DIR "/utils/extern-local-engine/tests/json/read_student_option_schema.csv.json"); -TEST(LocalExecutor, StorageFileSink) +size_t count(const substrait::Type_Struct & type) { - const std::string split - = R"({"items":[{"uriFile":"file:///home/chang/SourceCode/rebase_gluten/backends-velox/src/test/resources/datasource/csv/student_option_schema.csv","length":"56","text":{"fieldDelimiter":",","maxBlockSize":"8192","header":"1"},"schema":{"names":["id","name","language"],"struct":{"types":[{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}}]}},"metadataColumns":[{}]}]})"; - SerializedPlanParser parser(SerializedPlanParser::global_context); - parser.addSplitInfo(test::pb_util::JsonStringToBinary(split)); - auto local_executor = parser.createExecutor( - {reinterpret_cast(gresource_embedded_readcsv_jsonData), gresource_embedded_readcsv_jsonSize}); - - while (local_executor->hasNext()) + size_t ret = 0; + for (const auto & t : type.types()) { - const Block & x = *local_executor->nextColumnar(); - EXPECT_EQ(4, x.rows()); + if (t.has_struct_()) + ret += 1 + count(t.struct_()); + else + ret++; } + return ret; +} - StorageInMemoryMetadata metadata; - metadata.setColumns(ColumnsDescription::fromNamesAndTypes({{"name", STRING()}, {"value", UINT()}})); - StorageMetadataPtr metadata_ptr = std::make_shared(metadata); - - /* - auto sink = createFilelinkSink( - metadata_ptr, - "test_table", - "/tmp/test_table.parquet", - CompressionMethod::None, - {}, - "Parquet", - SerializedPlanParser::global_context, - 0); - - sink->consume(testChunk()); - sink->onFinish(); - */ +TEST(TypeParser, SchemaTest) +{ + const std::string scheam_str = R"({ + "names": [ + "count#16#Partial#count", + "anonymousfield0" + ], + "struct": { + "types": [ + { + "struct": { + "types": [ + { + "i64": { + "nullability": "NULLABILITY_REQUIRED" + } + } + ], + "nullability": "NULLABILITY_REQUIRED", + "names": [ + "anonymousField0" + ] + } + } + ] + } +})"; + + const auto schema = local_engine::JsonStringToMessage(scheam_str); + EXPECT_EQ(schema.names_size(), count(schema.struct_())); + const auto block = TypeParser::buildBlockFromNamedStruct(schema); + EXPECT_EQ(1, block.columns()); + debug::headBlock(block); } \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp b/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp new file mode 100644 index 0000000000000..e56184a1417cf --- /dev/null +++ b/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp @@ -0,0 +1,253 @@ +/* + * 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. + */ + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +using namespace local_engine; +using namespace DB; + +Chunk testChunk() +{ + auto nameCol = STRING()->createColumn(); + nameCol->insert("one"); + nameCol->insert("two"); + nameCol->insert("three"); + + auto valueCol = UINT()->createColumn(); + valueCol->insert(1); + valueCol->insert(2); + valueCol->insert(3); + MutableColumns x; + x.push_back(std::move(nameCol)); + x.push_back(std::move(valueCol)); + return {std::move(x), 3}; +} + +TEST(LocalExecutor, StorageObjectStorageSink) +{ + /// 0. Create ObjectStorage for HDFS + auto settings = SerializedPlanParser::global_context->getSettingsRef(); + const std::string query + = R"(CREATE TABLE hdfs_engine_xxxx (name String, value UInt32) ENGINE=HDFS('hdfs://localhost:8020/clickhouse/test2', 'Parquet'))"; + DB::ParserCreateQuery parser; + std::string error_message; + const char * pos = query.data(); + auto ast = DB::tryParseQuery( + parser, + pos, + pos + query.size(), + error_message, + /* hilite = */ false, + "QUERY TEST", + /* allow_multi_statements = */ false, + 0, + settings.max_parser_depth, + settings.max_parser_backtracks, + true); + auto & create = ast->as(); + auto arg = create.storage->children[0]; + const auto * func = arg->as(); + EXPECT_TRUE(func && func->name == "HDFS"); + + DB::StorageHDFSConfiguration config; + StorageObjectStorage::Configuration::initialize(config, arg->children[0]->children, SerializedPlanParser::global_context, false); + + const std::shared_ptr object_storage + = std::dynamic_pointer_cast(config.createObjectStorage(SerializedPlanParser::global_context, false)); + EXPECT_TRUE(object_storage != nullptr); + + RelativePathsWithMetadata files_with_metadata; + object_storage->listObjects("/clickhouse", files_with_metadata, 0); + + /// 1. Create ObjectStorageSink + DB::StorageObjectStorageSink sink{ + object_storage, config.clone(), {}, {{STRING(), "name"}, {UINT(), "value"}}, SerializedPlanParser::global_context, ""}; + + /// 2. Create Chunk + auto chunk = testChunk(); + /// 3. comsume + sink.consume(chunk); + sink.onFinish(); +} + + +INCBIN(resource_embedded_write_json, SOURCE_DIR "/utils/extern-local-engine/tests/json/native_write_plan.json"); +TEST(WritePipeline, SubstraitFileSink) +{ + const auto tmpdir = std::string{"file:///tmp/test_table/test"}; + const auto filename = std::string{"data.parquet"}; + const std::string split_template + = R"({"items":[{"uriFile":"{replace_local_files}","length":"1399183","text":{"fieldDelimiter":"|","maxBlockSize":"8192"},"schema":{"names":["s_suppkey","s_name","s_address","s_nationkey","s_phone","s_acctbal","s_comment"],"struct":{"types":[{"i64":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"i64":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"decimal":{"scale":2,"precision":15,"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}}]}},"metadataColumns":[{}]}]})"; + const std::string split + = replaceLocalFilesWildcards(split_template, GLUTEN_SOURCE_DIR("/backends-clickhouse/src/test/resources/csv-data/supplier.csv")); + + const auto context = DB::Context::createCopy(SerializedPlanParser::global_context); + context->setSetting(local_engine::SPARK_TASK_WRITE_TMEP_DIR, tmpdir); + context->setSetting(local_engine::SPARK_TASK_WRITE_FILENAME, filename); + SerializedPlanParser parser(context); + parser.addSplitInfo(local_engine::JsonStringToBinary(split)); + + const auto plan = local_engine::JsonStringToMessage( + {reinterpret_cast(gresource_embedded_write_jsonData), gresource_embedded_write_jsonSize}); + + EXPECT_EQ(1, plan.relations_size()); + const substrait::PlanRel & root_rel = plan.relations().at(0); + EXPECT_TRUE(root_rel.has_root()); + EXPECT_TRUE(root_rel.root().input().has_write()); + + const substrait::WriteRel & write_rel = root_rel.root().input().write(); + EXPECT_TRUE(write_rel.has_named_table()); + + const substrait::NamedObjectWrite & named_table = write_rel.named_table(); + + google::protobuf::StringValue optimization; + named_table.advanced_extension().optimization().UnpackTo(&optimization); + auto config = local_engine::parse_write_parameter(optimization.value()); + EXPECT_EQ(2, config.size()); + EXPECT_EQ("parquet", config["format"]); + EXPECT_EQ("1", config["isSnappy"]); + + + EXPECT_TRUE(write_rel.has_table_schema()); + const substrait::NamedStruct & table_schema = write_rel.table_schema(); + auto block = TypeParser::buildBlockFromNamedStruct(table_schema); + auto names = block.getNames(); + DB::Names expected{"s_suppkey", "s_name", "s_address", "s_nationkey", "s_phone", "s_acctbal", "s_comment111"}; + EXPECT_EQ(expected, names); + + auto partitionCols = collect_partition_cols(block, table_schema); + DB::Names expected_partition_cols; + EXPECT_EQ(expected_partition_cols, partitionCols); + + + auto local_executor = parser.createExecutor(plan); + EXPECT_TRUE(local_executor->hasNext()); + const Block & x = *local_executor->nextColumnar(); + debug::headBlock(x); + EXPECT_EQ(1, x.rows()); + const auto & col_a = *(x.getColumns()[0]); + EXPECT_EQ(filename, col_a.getDataAt(0)); + const auto & col_b = *(x.getColumns()[1]); + EXPECT_EQ(SubstraitFileSink::NO_PARTITION_ID, col_b.getDataAt(0)); + const auto & col_c = *(x.getColumns()[2]); + EXPECT_EQ(10000, col_c.getInt(0)); +} + +INCBIN(resource_embedded_write_one_partition_json, SOURCE_DIR "/utils/extern-local-engine/tests/json/native_write_one_partition.json"); + +TEST(WritePipeline, SubstraitPartitionedFileSink) +{ + const std::string split_template + = R"({"items":[{"uriFile":"{replace_local_files}","length":"1399183","text":{"fieldDelimiter":"|","maxBlockSize":"8192"},"schema":{"names":["s_suppkey","s_name","s_address","s_nationkey","s_phone","s_acctbal","s_comment"],"struct":{"types":[{"i64":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"i64":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"decimal":{"scale":2,"precision":15,"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}}]}},"metadataColumns":[{}]}]})"; + const std::string split + = replaceLocalFilesWildcards(split_template, GLUTEN_SOURCE_DIR("/backends-clickhouse/src/test/resources/csv-data/supplier.csv")); + + const auto context = DB::Context::createCopy(SerializedPlanParser::global_context); + context->setSetting(local_engine::SPARK_TASK_WRITE_TMEP_DIR, std::string{"file:///tmp/test_table/test_partition"}); + context->setSetting(local_engine::SPARK_TASK_WRITE_FILENAME, std::string{"data.parquet"}); + SerializedPlanParser parser(context); + parser.addSplitInfo(local_engine::JsonStringToBinary(split)); + + const auto plan = local_engine::JsonStringToMessage( + {reinterpret_cast(gresource_embedded_write_one_partition_jsonData), gresource_embedded_write_one_partition_jsonSize}); + + EXPECT_EQ(1, plan.relations_size()); + const substrait::PlanRel & root_rel = plan.relations().at(0); + EXPECT_TRUE(root_rel.has_root()); + EXPECT_TRUE(root_rel.root().input().has_write()); + + const substrait::WriteRel & write_rel = root_rel.root().input().write(); + EXPECT_TRUE(write_rel.has_named_table()); + + const substrait::NamedObjectWrite & named_table = write_rel.named_table(); + + google::protobuf::StringValue optimization; + named_table.advanced_extension().optimization().UnpackTo(&optimization); + auto config = local_engine::parse_write_parameter(optimization.value()); + EXPECT_EQ(2, config.size()); + EXPECT_EQ("parquet", config["format"]); + EXPECT_EQ("1", config["isSnappy"]); + + + EXPECT_TRUE(write_rel.has_table_schema()); + const substrait::NamedStruct & table_schema = write_rel.table_schema(); + auto block = TypeParser::buildBlockFromNamedStruct(table_schema); + auto names = block.getNames(); + DB::Names expected{"s_suppkey", "s_name", "s_address", "s_phone", "s_acctbal", "s_comment", "s_nationkey"}; + EXPECT_EQ(expected, names); + + auto partitionCols = local_engine::collect_partition_cols(block, table_schema); + DB::Names expected_partition_cols{"s_nationkey"}; + EXPECT_EQ(expected_partition_cols, partitionCols); + + auto local_executor = parser.createExecutor(plan); + EXPECT_TRUE(local_executor->hasNext()); + const Block & x = *local_executor->nextColumnar(); + debug::headBlock(x, 25); + EXPECT_EQ(25, x.rows()); + // const auto & col_b = *(x.getColumns()[1]); + // EXPECT_EQ(16, col_b.getInt(0)); +} + +TEST(WritePipeline, ComputePartitionedExpression) +{ + const auto context = DB::Context::createCopy(SerializedPlanParser::global_context); + + auto partition_by = SubstraitPartitionedFileSink::make_partition_expression({"s_nationkey", "name"}); + + ASTs arguments(1, partition_by); + ASTPtr partition_by_string = makeASTFunction("toString", std::move(arguments)); + + Block sample_block{{STRING(), "name"}, {UINT(), "s_nationkey"}}; + auto syntax_result = TreeRewriter(context).analyze(partition_by_string, sample_block.getNamesAndTypesList()); + auto partition_by_expr = ExpressionAnalyzer(partition_by_string, syntax_result, context).getActions(false); + + + auto partition_by_column_name = partition_by_string->getColumnName(); + + Chunk chunk = testChunk(); + const auto & columns = chunk.getColumns(); + Block block_with_partition_by_expr = sample_block.cloneWithoutColumns(); + block_with_partition_by_expr.setColumns(columns); + partition_by_expr->execute(block_with_partition_by_expr); + + size_t chunk_rows = chunk.getNumRows(); + EXPECT_EQ(3, chunk_rows); + + const auto * partition_by_result_column = block_with_partition_by_expr.getByName(partition_by_column_name).column.get(); + EXPECT_EQ("s_nationkey=1/name=one", partition_by_result_column->getDataAt(0)); + EXPECT_EQ("s_nationkey=2/name=two", partition_by_result_column->getDataAt(1)); + EXPECT_EQ("s_nationkey=3/name=three", partition_by_result_column->getDataAt(2)); +} \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/json/native_write_one_partition.json b/cpp-ch/local-engine/tests/json/native_write_one_partition.json new file mode 100644 index 0000000000000..45b3f60e41fa1 --- /dev/null +++ b/cpp-ch/local-engine/tests/json/native_write_one_partition.json @@ -0,0 +1,283 @@ +{ + "relations": [ + { + "root": { + "input": { + "write": { + "namedTable": { + "advancedExtension": { + "optimization": { + "@type": "type.googleapis.com/google.protobuf.StringValue", + "value": "WriteParameters:isSnappy=1;format=parquet\n" + }, + "enhancement": { + "@type": "type.googleapis.com/substrait.Type", + "struct": { + "types": [ + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "decimal": { + "scale": 2, + "precision": 15, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + } + ], + "nullability": "NULLABILITY_REQUIRED" + } + } + } + }, + "tableSchema": { + "names": [ + "s_suppkey", + "s_name", + "s_address", + "s_phone", + "s_acctbal", + "s_comment", + "s_nationkey" + ], + "struct": { + "types": [ + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "decimal": { + "scale": 2, + "precision": 15, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + } + ] + }, + "columnTypes": [ + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "PARTITION_COL" + ] + }, + "input": { + "project": { + "common": { + "emit": { + "outputMapping": [ + 7, + 8, + 9, + 10, + 11, + 12, + 13 + ] + } + }, + "input": { + "read": { + "common": { + "direct": {} + }, + "baseSchema": { + "names": [ + "s_suppkey", + "s_name", + "s_address", + "s_nationkey", + "s_phone", + "s_acctbal", + "s_comment" + ], + "struct": { + "types": [ + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "decimal": { + "scale": 2, + "precision": 15, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + } + ] + }, + "columnTypes": [ + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL" + ] + }, + "advancedExtension": { + "optimization": { + "@type": "type.googleapis.com/google.protobuf.StringValue", + "value": "isMergeTree=0\n" + } + } + } + }, + "expressions": [ + { + "selection": { + "directReference": { + "structField": {} + } + } + }, + { + "selection": { + "directReference": { + "structField": { + "field": 1 + } + } + } + }, + { + "selection": { + "directReference": { + "structField": { + "field": 2 + } + } + } + }, + { + "selection": { + "directReference": { + "structField": { + "field": 4 + } + } + } + }, + { + "selection": { + "directReference": { + "structField": { + "field": 5 + } + } + } + }, + { + "selection": { + "directReference": { + "structField": { + "field": 6 + } + } + } + }, + { + "selection": { + "directReference": { + "structField": { + "field": 3 + } + } + } + } + ] + } + } + } + }, + "outputSchema": { + "nullability": "NULLABILITY_REQUIRED" + } + } + } + ] +} \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/json/native_write_plan.json b/cpp-ch/local-engine/tests/json/native_write_plan.json new file mode 100644 index 0000000000000..8d5ffce7b1636 --- /dev/null +++ b/cpp-ch/local-engine/tests/json/native_write_plan.json @@ -0,0 +1,203 @@ +{ + "relations": [ + { + "root": { + "input": { + "write": { + "namedTable": { + "advancedExtension": { + "optimization": { + "@type": "type.googleapis.com/google.protobuf.StringValue", + "value": "WriteParameters:isSnappy=1;format=parquet\n" + }, + "enhancement": { + "@type": "type.googleapis.com/substrait.Type", + "struct": { + "types": [ + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "decimal": { + "scale": 2, + "precision": 15, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + } + ], + "nullability": "NULLABILITY_REQUIRED" + } + } + } + }, + "tableSchema": { + "names": [ + "s_suppkey", + "s_name", + "s_address", + "s_nationkey", + "s_phone", + "s_acctbal", + "s_comment111" + ], + "struct": { + "types": [ + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "decimal": { + "scale": 2, + "precision": 15, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + } + ] + }, + "columnTypes": [ + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL" + ] + }, + "input": { + "read": { + "common": { + "direct": {} + }, + "baseSchema": { + "names": [ + "s_suppkey", + "s_name", + "s_address", + "s_nationkey", + "s_phone", + "s_acctbal", + "s_comment" + ], + "struct": { + "types": [ + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "decimal": { + "scale": 2, + "precision": 15, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + } + ] + }, + "columnTypes": [ + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL" + ] + }, + "advancedExtension": { + "optimization": { + "@type": "type.googleapis.com/google.protobuf.StringValue", + "value": "isMergeTree=0\n" + } + } + } + } + } + }, + "outputSchema": { + "nullability": "NULLABILITY_REQUIRED" + } + } + } + ] +} \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/json/native_write_plan_1_spark33.json b/cpp-ch/local-engine/tests/json/native_write_plan_1_spark33.json new file mode 100644 index 0000000000000..e053c99352647 --- /dev/null +++ b/cpp-ch/local-engine/tests/json/native_write_plan_1_spark33.json @@ -0,0 +1,116 @@ +{ + "extensions": [ + { + "extensionFunction": { + "name": "sum:req_i32" + } + } + ], + "relations": [ + { + "root": { + "input": { + "aggregate": { + "common": { + "direct": {} + }, + "input": { + "read": { + "baseSchema": { + "names": [ + "string_field#0", + "int_field#1" + ], + "struct": { + "types": [ + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i32": { + "nullability": "NULLABILITY_NULLABLE" + } + } + ] + } + }, + "localFiles": { + "items": [ + { + "uriFile": "iterator:0" + } + ] + } + } + }, + "groupings": [ + { + "groupingExpressions": [ + { + "selection": { + "directReference": { + "structField": {} + } + } + } + ] + } + ], + "measures": [ + { + "measure": { + "phase": "AGGREGATION_PHASE_INITIAL_TO_INTERMEDIATE", + "outputType": { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + "arguments": [ + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 1 + } + } + } + } + } + ] + } + } + ], + "advancedExtension": { + "optimization": { + "@type": "type.googleapis.com/google.protobuf.StringValue", + "value": "has_required_child_distribution_expressions=false\n" + } + } + } + }, + "names": [ + "string_field#0", + "sum#31" + ], + "outputSchema": { + "types": [ + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + } + ], + "nullability": "NULLABILITY_REQUIRED" + } + } + } + ] +} \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/json/native_write_plan_1_spark35.json b/cpp-ch/local-engine/tests/json/native_write_plan_1_spark35.json new file mode 100644 index 0000000000000..31592e5c75df3 --- /dev/null +++ b/cpp-ch/local-engine/tests/json/native_write_plan_1_spark35.json @@ -0,0 +1,246 @@ +{ + "relations": [ + { + "root": { + "input": { + "write": { + "namedTable": { + "advancedExtension": { + "optimization": { + "@type": "type.googleapis.com/google.protobuf.StringValue", + "value": "WriteParameters:isSnappy=1;format=orc\n" + }, + "enhancement": { + "@type": "type.googleapis.com/substrait.Type", + "struct": { + "types": [ + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i32": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp32": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i16": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i8": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "bool": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "decimal": { + "scale": 18, + "precision": 38, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + } + ], + "nullability": "NULLABILITY_REQUIRED" + } + } + } + }, + "tableSchema": { + "names": [ + "string_field", + "int_field", + "long_field", + "float_field", + "double_field", + "short_field", + "byte_field", + "boolean_field", + "decimal_field", + "date_field" + ], + "struct": { + "types": [ + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i32": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp32": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i16": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i8": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "bool": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "decimal": { + "scale": 18, + "precision": 38, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + } + ] + }, + "columnTypes": [ + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL" + ] + }, + "input": { + "read": { + "baseSchema": { + "names": [ + "string_field#0", + "int_field#1", + "long_field#2", + "float_field#3", + "double_field#4", + "short_field#5", + "byte_field#6", + "boolean_field#7", + "decimal_field#8", + "date_field#9" + ], + "struct": { + "types": [ + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i32": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp32": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i16": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i8": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "bool": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "decimal": { + "scale": 18, + "precision": 38, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + } + ] + } + }, + "localFiles": { + "items": [ + { + "uriFile": "iterator:0" + } + ] + } + } + } + } + }, + "outputSchema": { + "nullability": "NULLABILITY_REQUIRED" + } + } + } + ] +} \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/testConfig.h.in b/cpp-ch/local-engine/tests/testConfig.h.in index 75157c0126fd1..8dd3c2cb68d9c 100644 --- a/cpp-ch/local-engine/tests/testConfig.h.in +++ b/cpp-ch/local-engine/tests/testConfig.h.in @@ -4,3 +4,5 @@ #define PARQUET_DATA(file) "file://@PARQUET_DATA_DIR@"#file #define MERGETREE_DATA(file) "@MERGETREE_DATA_DIR@"#file +#define GLUTEN_SOURCE_DIR_ "file://@GLUTEN_REAL_PATH@" +#define GLUTEN_SOURCE_DIR(file) GLUTEN_SOURCE_DIR_ file \ No newline at end of file From a4cafeee129fc8bc06733994f1bbebf0d008a85b Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Fri, 26 Jul 2024 10:02:05 +0800 Subject: [PATCH 03/61] [VL] Gluten-it: --data-gen-strategy=once to skip data-gen when it already exists (#6587) --- .github/workflows/velox_be.yml.deprecated | 12 ++++---- .github/workflows/velox_docker.yml | 18 ++++++------ .../integration/command/DataGenMixin.java | 24 +++++++++++---- .../integration/action/DataGenOnly.scala | 29 +++++++++++++++++-- 4 files changed, 60 insertions(+), 23 deletions(-) diff --git a/.github/workflows/velox_be.yml.deprecated b/.github/workflows/velox_be.yml.deprecated index d095af64d7b07..6ff5ec743c140 100644 --- a/.github/workflows/velox_be.yml.deprecated +++ b/.github/workflows/velox_be.yml.deprecated @@ -529,9 +529,9 @@ jobs: $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh 'cd /opt/gluten/tools/gluten-it && \ mvn clean install -Pspark-3.2 \ && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries \ - --local --preset=velox --benchmark-type=h --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 --skip-data-gen --random-kill-tasks \ + --local --preset=velox --benchmark-type=h --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 --data-gen-strategy=skip --random-kill-tasks \ && GLUTEN_IT_JVM_ARGS=-Xmx50G sbin/gluten-it.sh queries \ - --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=50g -s=30.0 --threads=32 --iterations=1 --skip-data-gen --random-kill-tasks' + --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=50g -s=30.0 --threads=32 --iterations=1 --data-gen-strategy=skip --random-kill-tasks' - name: Exit docker container if: ${{ always() }} run: | @@ -580,7 +580,7 @@ jobs: mvn clean install -Pspark-3.2 \ && GLUTEN_IT_JVM_ARGS=-Xmx50G sbin/gluten-it.sh parameterized \ --local --preset=velox --benchmark-type=ds --error-on-memleak --queries=q67,q95 -s=30.0 --threads=12 --shuffle-partitions=72 --iterations=1 \ - --skip-data-gen -m=OffHeapExecutionMemory \ + --data-gen-strategy=skip -m=OffHeapExecutionMemory \ -d=ISOLATION:OFF,spark.gluten.memory.isolation=false \ -d=OFFHEAP_SIZE:5g,spark.memory.offHeap.size=5g \ -d=OFFHEAP_SIZE:3g,spark.memory.offHeap.size=3g \ @@ -592,7 +592,7 @@ jobs: mvn clean install -Pspark-3.2 \ && GLUTEN_IT_JVM_ARGS=-Xmx50G sbin/gluten-it.sh parameterized \ --local --preset=velox --benchmark-type=ds --error-on-memleak --queries=q67,q95 -s=30.0 --threads=12 --shuffle-partitions=72 --iterations=1 \ - --skip-data-gen -m=OffHeapExecutionMemory \ + --data-gen-strategy=skip -m=OffHeapExecutionMemory \ -d=ISOLATION:ON,spark.gluten.memory.isolation=true,spark.memory.storageFraction=0.1 \ -d=OFFHEAP_SIZE:5g,spark.memory.offHeap.size=5g \ -d=OFFHEAP_SIZE:3g,spark.memory.offHeap.size=3g \ @@ -603,7 +603,7 @@ jobs: $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh 'cd /opt/gluten/tools/gluten-it && \ GLUTEN_IT_JVM_ARGS=-Xmx50G sbin/gluten-it.sh parameterized \ --local --preset=velox --benchmark-type=ds --error-on-memleak --queries=q23a,q23b -s=30.0 --threads=12 --shuffle-partitions=72 --iterations=1 \ - --skip-data-gen -m=OffHeapExecutionMemory \ + --data-gen-strategy=skip -m=OffHeapExecutionMemory \ -d=ISOLATION:OFF,spark.gluten.memory.isolation=false \ -d=ISOLATION:ON,spark.gluten.memory.isolation=true,spark.memory.storageFraction=0.1 \ -d=OFFHEAP_SIZE:2g,spark.memory.offHeap.size=2g \ @@ -615,7 +615,7 @@ jobs: $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh 'cd /opt/gluten/tools/gluten-it && \ GLUTEN_IT_JVM_ARGS=-Xmx50G sbin/gluten-it.sh parameterized \ --local --preset=velox --benchmark-type=ds --error-on-memleak --queries=q97 -s=30.0 --threads=12 --shuffle-partitions=72 --iterations=1 \ - --skip-data-gen -m=OffHeapExecutionMemory \ + --data-gen-strategy=skip -m=OffHeapExecutionMemory \ -d=ISOLATION:OFF,spark.gluten.memory.isolation=false \ -d=ISOLATION:ON,spark.gluten.memory.isolation=true,spark.memory.storageFraction=0.1 \ -d=OFFHEAP_SIZE:2g,spark.memory.offHeap.size=2g \ diff --git a/.github/workflows/velox_docker.yml b/.github/workflows/velox_docker.yml index 1e88e034e5858..47dd7a9190b84 100644 --- a/.github/workflows/velox_docker.yml +++ b/.github/workflows/velox_docker.yml @@ -296,7 +296,7 @@ jobs: cd tools/gluten-it \ && GLUTEN_IT_JVM_ARGS=-Xmx3G sbin/gluten-it.sh parameterized \ --local --preset=velox --benchmark-type=ds --error-on-memleak --queries=q67,q95 -s=30.0 --threads=12 --shuffle-partitions=72 --iterations=1 \ - --skip-data-gen -m=OffHeapExecutionMemory \ + --data-gen-strategy=skip -m=OffHeapExecutionMemory \ -d=ISOLATION:OFF,spark.gluten.memory.isolation=false \ -d=OFFHEAP_SIZE:6g,spark.memory.offHeap.size=6g \ -d=OFFHEAP_SIZE:4g,spark.memory.offHeap.size=4g \ @@ -308,7 +308,7 @@ jobs: cd tools/gluten-it \ && GLUTEN_IT_JVM_ARGS=-Xmx3G sbin/gluten-it.sh parameterized \ --local --preset=velox --benchmark-type=ds --error-on-memleak --queries=q67 -s=30.0 --threads=12 --shuffle-partitions=72 --iterations=1 \ - --skip-data-gen -m=OffHeapExecutionMemory \ + --data-gen-strategy=skip -m=OffHeapExecutionMemory \ -d=ISOLATION:ON,spark.gluten.memory.isolation=true,spark.memory.storageFraction=0.1 \ -d=OFFHEAP_SIZE:6g,spark.memory.offHeap.size=6g \ -d=OFFHEAP_SIZE:4g,spark.memory.offHeap.size=4g \ @@ -319,7 +319,7 @@ jobs: cd tools/gluten-it \ && GLUTEN_IT_JVM_ARGS=-Xmx3G sbin/gluten-it.sh parameterized \ --local --preset=velox --benchmark-type=ds --error-on-memleak --queries=q95 -s=30.0 --threads=12 --shuffle-partitions=72 --iterations=1 \ - --skip-data-gen -m=OffHeapExecutionMemory \ + --data-gen-strategy=skip -m=OffHeapExecutionMemory \ -d=ISOLATION:ON,spark.gluten.memory.isolation=true,spark.memory.storageFraction=0.1 \ -d=OFFHEAP_SIZE:6g,spark.memory.offHeap.size=6g \ -d=OFFHEAP_SIZE:4g,spark.memory.offHeap.size=4g \ @@ -330,7 +330,7 @@ jobs: cd tools/gluten-it \ && GLUTEN_IT_JVM_ARGS=-Xmx3G sbin/gluten-it.sh parameterized \ --local --preset=velox --benchmark-type=ds --error-on-memleak --queries=q23a,q23b -s=30.0 --threads=12 --shuffle-partitions=72 --iterations=1 \ - --skip-data-gen -m=OffHeapExecutionMemory \ + --data-gen-strategy=skip -m=OffHeapExecutionMemory \ -d=ISOLATION:OFF,spark.gluten.memory.isolation=false \ -d=OFFHEAP_SIZE:2g,spark.memory.offHeap.size=2g \ -d=FLUSH_MODE:DISABLED,spark.gluten.sql.columnar.backend.velox.flushablePartialAggregation=false,spark.gluten.sql.columnar.backend.velox.maxPartialAggregationMemoryRatio=1.0,spark.gluten.sql.columnar.backend.velox.maxExtendedPartialAggregationMemoryRatio=1.0,spark.gluten.sql.columnar.backend.velox.abandonPartialAggregationMinPct=100,spark.gluten.sql.columnar.backend.velox.abandonPartialAggregationMinRows=0 \ @@ -341,7 +341,7 @@ jobs: cd tools/gluten-it \ && GLUTEN_IT_JVM_ARGS=-Xmx3G sbin/gluten-it.sh parameterized \ --local --preset=velox --benchmark-type=ds --error-on-memleak --queries=q23a,q23b -s=30.0 --threads=12 --shuffle-partitions=72 --iterations=1 \ - --skip-data-gen -m=OffHeapExecutionMemory \ + --data-gen-strategy=skip -m=OffHeapExecutionMemory \ -d=ISOLATION:ON,spark.gluten.memory.isolation=true,spark.memory.storageFraction=0.1 \ -d=OFFHEAP_SIZE:2g,spark.memory.offHeap.size=2g \ -d=FLUSH_MODE:DISABLED,spark.gluten.sql.columnar.backend.velox.flushablePartialAggregation=false,spark.gluten.sql.columnar.backend.velox.maxPartialAggregationMemoryRatio=1.0,spark.gluten.sql.columnar.backend.velox.maxExtendedPartialAggregationMemoryRatio=1.0,spark.gluten.sql.columnar.backend.velox.abandonPartialAggregationMinPct=100,spark.gluten.sql.columnar.backend.velox.abandonPartialAggregationMinRows=0 \ @@ -352,7 +352,7 @@ jobs: cd tools/gluten-it \ && GLUTEN_IT_JVM_ARGS=-Xmx3G sbin/gluten-it.sh parameterized \ --local --preset=velox --benchmark-type=ds --error-on-memleak --queries=q97 -s=30.0 --threads=12 --shuffle-partitions=72 --iterations=1 \ - --skip-data-gen -m=OffHeapExecutionMemory \ + --data-gen-strategy=skip -m=OffHeapExecutionMemory \ -d=ISOLATION:OFF,spark.gluten.memory.isolation=false \ -d=ISOLATION:ON,spark.gluten.memory.isolation=true,spark.memory.storageFraction=0.1 \ -d=OFFHEAP_SIZE:2g,spark.memory.offHeap.size=2g \ @@ -408,7 +408,7 @@ jobs: cd tools/gluten-it \ && GLUTEN_IT_JVM_ARGS=-Xmx6G sbin/gluten-it.sh queries \ --local --preset=velox --benchmark-type=ds --error-on-memleak -s=30.0 --off-heap-size=8g --threads=12 --shuffle-partitions=72 --iterations=1 \ - --skip-data-gen --random-kill-tasks --no-session-reuse + --data-gen-strategy=skip --random-kill-tasks --no-session-reuse # run-tpc-test-ubuntu-sf30: # needs: build-native-lib-centos-7 @@ -457,10 +457,10 @@ jobs: # cd tools/gluten-it \ # && GLUTEN_IT_JVM_ARGS=-Xmx6G sbin/gluten-it.sh queries-compare \ # --local --preset=velox --benchmark-type=h --error-on-memleak -s=30.0 --off-heap-size=8g --threads=12 --shuffle-partitions=72 --iterations=1 \ - # --skip-data-gen --shard=${{ matrix.shard }} \ + # --data-gen-strategy=skip --shard=${{ matrix.shard }} \ # && GLUTEN_IT_JVM_ARGS=-Xmx6G sbin/gluten-it.sh queries-compare \ # --local --preset=velox --benchmark-type=ds --error-on-memleak -s=30.0 --off-heap-size=8g --threads=12 --shuffle-partitions=72 --iterations=1 \ - # --skip-data-gen --shard=${{ matrix.shard }} + # --data-gen-strategy=skip --shard=${{ matrix.shard }} run-tpc-test-centos8-uniffle: needs: build-native-lib-centos-7 diff --git a/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/command/DataGenMixin.java b/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/command/DataGenMixin.java index 0682f5601a926..3854d078e261c 100644 --- a/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/command/DataGenMixin.java +++ b/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/command/DataGenMixin.java @@ -17,23 +17,35 @@ package org.apache.gluten.integration.command; import org.apache.gluten.integration.action.Action; +import org.apache.gluten.integration.action.DataGenOnly; import picocli.CommandLine; public class DataGenMixin { + @CommandLine.Option(names = {"--data-gen-strategy"}, description = "The strategy of data generation, accepted values: skip, once, always", defaultValue = "always") + private String dataGenStrategy; + @CommandLine.Option(names = {"-s", "--scale"}, description = "The scale factor of sample TPC-H dataset", defaultValue = "0.1") private double scale; @CommandLine.Option(names = {"--gen-partitioned-data"}, description = "Generate data with partitions", defaultValue = "false") private boolean genPartitionedData; - @CommandLine.Option(names = {"--skip-data-gen"}, description = "Skip data generation", defaultValue = "false") - private boolean skipDataGen; - public Action[] makeActions() { - if (skipDataGen) { - return new Action[0]; + final DataGenOnly.Strategy strategy; + switch (dataGenStrategy) { + case "skip": + strategy = DataGenOnly.Skip$.MODULE$; + break; + case "once": + strategy = DataGenOnly.Once$.MODULE$; + break; + case "always": + strategy = DataGenOnly.Always$.MODULE$; + break; + default: + throw new IllegalArgumentException("Unexpected data-gen strategy: " + dataGenStrategy); } - return new Action[]{new org.apache.gluten.integration.action.DataGenOnly(scale, genPartitionedData)}; + return new Action[]{new org.apache.gluten.integration.action.DataGenOnly(strategy, scale, genPartitionedData)}; } public double getScale() { diff --git a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/DataGenOnly.scala b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/DataGenOnly.scala index bc43834610a4b..dc54e9737703a 100644 --- a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/DataGenOnly.scala +++ b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/DataGenOnly.scala @@ -20,11 +20,36 @@ import org.apache.gluten.integration.Suite import java.io.File -case class DataGenOnly(scale: Double, genPartitionedData: Boolean) extends Action { +case class DataGenOnly(strategy: DataGenOnly.Strategy, scale: Double, genPartitionedData: Boolean) + extends Action { override def execute(suite: Suite): Boolean = { + strategy match { + case DataGenOnly.Skip => + // Do nothing + case DataGenOnly.Once => + val dataPath = suite.dataWritePath(scale, genPartitionedData) + val alreadyExists = new File(dataPath).exists() + if (alreadyExists) { + println(s"Data already exists at $dataPath, skipping generating it.") + } else { + gen(suite) + } + case DataGenOnly.Always => + gen(suite) + } + true + } + + private def gen(suite: Suite): Unit = { suite.sessionSwitcher.useSession("baseline", "Data Gen") val dataGen = suite.createDataGen(scale, genPartitionedData) dataGen.gen() - true } } + +object DataGenOnly { + sealed trait Strategy + case object Skip extends Strategy + case object Once extends Strategy + case object Always extends Strategy +} From b93b098eda072dbb73903550d4848dfde09180c0 Mon Sep 17 00:00:00 2001 From: Shuai li Date: Fri, 26 Jul 2024 11:54:50 +0800 Subject: [PATCH 04/61] [GLUTEN-6589][CH] Mergetree supported spark.sql.caseSensitive (#6592) [CH] Mergetree supported spark.sql.caseSensitive --- .../delta/catalog/ClickHouseTableV2Base.scala | 33 ++++-------- .../utils/MergeTreeDeltaUtil.scala | 11 +++- .../v1/CHMergeTreeWriterInjects.scala | 20 ++------ .../GlutenClickHouseMergeTreeWriteSuite.scala | 51 +++++++++++++++++++ .../gluten/expression/ConverterUtils.scala | 2 +- 5 files changed, 74 insertions(+), 43 deletions(-) diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/catalog/ClickHouseTableV2Base.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/catalog/ClickHouseTableV2Base.scala index 9c129b9f5d918..633d23f77b1ba 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/catalog/ClickHouseTableV2Base.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/catalog/ClickHouseTableV2Base.scala @@ -16,8 +16,11 @@ */ package org.apache.spark.sql.delta.catalog +import org.apache.gluten.expression.ConverterUtils.normalizeColName + import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable} import org.apache.spark.sql.delta.Snapshot +import org.apache.spark.sql.execution.datasources.utils.MergeTreeDeltaUtil import org.apache.hadoop.fs.Path @@ -153,33 +156,15 @@ trait ClickHouseTableV2Base { configs.toMap } - def primaryKey(): String = primaryKeyOption match { - case Some(keys) => keys.mkString(",") - case None => "" - } + def primaryKey(): String = MergeTreeDeltaUtil.columnsToStr(primaryKeyOption) def orderByKey(): String = orderByKeyOption match { - case Some(keys) => keys.mkString(",") + case Some(keys) => keys.map(normalizeColName).mkString(",") case None => "tuple()" } - def lowCardKey(): String = lowCardKeyOption match { - case Some(keys) => keys.mkString(",") - case None => "" - } - - def minmaxIndexKey(): String = minmaxIndexKeyOption match { - case Some(keys) => keys.mkString(",") - case None => "" - } - - def bfIndexKey(): String = bfIndexKeyOption match { - case Some(keys) => keys.mkString(",") - case None => "" - } - - def setIndexKey(): String = setIndexKeyOption match { - case Some(keys) => keys.mkString(",") - case None => "" - } + def lowCardKey(): String = MergeTreeDeltaUtil.columnsToStr(lowCardKeyOption) + def minmaxIndexKey(): String = MergeTreeDeltaUtil.columnsToStr(minmaxIndexKeyOption) + def bfIndexKey(): String = MergeTreeDeltaUtil.columnsToStr(bfIndexKeyOption) + def setIndexKey(): String = MergeTreeDeltaUtil.columnsToStr(setIndexKeyOption) } diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/utils/MergeTreeDeltaUtil.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/utils/MergeTreeDeltaUtil.scala index 954b43b6ab6c4..6b2af0953f00f 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/utils/MergeTreeDeltaUtil.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/utils/MergeTreeDeltaUtil.scala @@ -16,6 +16,8 @@ */ package org.apache.spark.sql.execution.datasources.utils +import org.apache.gluten.expression.ConverterUtils.normalizeColName + object MergeTreeDeltaUtil { val DEFAULT_ORDER_BY_KEY = "tuple()" @@ -25,7 +27,7 @@ object MergeTreeDeltaUtil { primaryKeyOption: Option[Seq[String]]): (String, String) = { val orderByKey = if (orderByKeyOption.isDefined && orderByKeyOption.get.nonEmpty) { - orderByKeyOption.get.mkString(",") + columnsToStr(orderByKeyOption) } else DEFAULT_ORDER_BY_KEY val primaryKey = @@ -33,9 +35,14 @@ object MergeTreeDeltaUtil { !orderByKey.equals(DEFAULT_ORDER_BY_KEY) && primaryKeyOption.isDefined && primaryKeyOption.get.nonEmpty ) { - primaryKeyOption.get.mkString(",") + columnsToStr(primaryKeyOption) } else "" (orderByKey, primaryKey) } + + def columnsToStr(option: Option[Seq[String]]): String = option match { + case Some(keys) => keys.map(normalizeColName).mkString(",") + case None => "" + } } diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHMergeTreeWriterInjects.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHMergeTreeWriterInjects.scala index e11406d566195..237d5a46d69f9 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHMergeTreeWriterInjects.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHMergeTreeWriterInjects.scala @@ -170,22 +170,10 @@ object CHMergeTreeWriterInjects { primaryKeyOption ) - val lowCardKey = lowCardKeyOption match { - case Some(keys) => keys.mkString(",") - case None => "" - } - val minmaxIndexKey = minmaxIndexKeyOption match { - case Some(keys) => keys.mkString(",") - case None => "" - } - val bfIndexKey = bfIndexKeyOption match { - case Some(keys) => keys.mkString(",") - case None => "" - } - val setIndexKey = setIndexKeyOption match { - case Some(keys) => keys.mkString(",") - case None => "" - } + val lowCardKey = MergeTreeDeltaUtil.columnsToStr(lowCardKeyOption) + val minmaxIndexKey = MergeTreeDeltaUtil.columnsToStr(minmaxIndexKeyOption) + val bfIndexKey = MergeTreeDeltaUtil.columnsToStr(bfIndexKeyOption) + val setIndexKey = MergeTreeDeltaUtil.columnsToStr(setIndexKeyOption) val substraitContext = new SubstraitContext val extensionTableNode = ExtensionTableBuilder.makeExtensionTable( diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteSuite.scala index e88eb1fedd426..2563d792b0404 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteSuite.scala @@ -1971,5 +1971,56 @@ class GlutenClickHouseMergeTreeWriteSuite } }) } + + test("test mergetree with column case sensitive") { + spark.sql(s""" + |DROP TABLE IF EXISTS LINEITEM_MERGETREE_CASE_SENSITIVE; + |""".stripMargin) + + spark.sql(s""" + |CREATE TABLE IF NOT EXISTS LINEITEM_MERGETREE_CASE_SENSITIVE + |( + | L_ORDERKEY bigint, + | L_PARTKEY bigint, + | L_SUPPKEY bigint, + | L_LINENUMBER bigint, + | L_QUANTITY double, + | L_EXTENDEDPRICE double, + | L_DISCOUNT double, + | L_TAX double, + | L_RETURNFLAG string, + | L_LINESTATUS string, + | L_SHIPDATE date, + | L_COMMITDATE date, + | L_RECEIPTDATE date, + | L_SHIPINSTRUCT string, + | L_SHIPMODE string, + | L_COMMENT string + |) + |USING clickhouse + |PARTITIONED BY (L_SHIPDATE) + |TBLPROPERTIES (orderByKey='L_DISCOUNT') + |LOCATION '$basePath/LINEITEM_MERGETREE_CASE_SENSITIVE' + |""".stripMargin) + + spark.sql(s""" + | insert into table lineitem_mergetree_case_sensitive + | select * from lineitem + |""".stripMargin) + + val sqlStr = + s""" + |SELECT + | sum(l_extendedprice * l_discount) AS revenue + |FROM + | lineitem_mergetree_case_sensitive + |WHERE + | l_shipdate >= date'1994-01-01' + | AND l_shipdate < date'1994-01-01' + interval 1 year + | AND l_discount BETWEEN 0.06 - 0.01 AND 0.06 + 0.01 + | AND l_quantity < 24 + |""".stripMargin + runTPCHQueryBySQL(6, sqlStr) { _ => } + } } // scalastyle:off line.size.limit diff --git a/gluten-core/src/main/scala/org/apache/gluten/expression/ConverterUtils.scala b/gluten-core/src/main/scala/org/apache/gluten/expression/ConverterUtils.scala index 473ee7f9d62f6..4b929e525197b 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/expression/ConverterUtils.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/expression/ConverterUtils.scala @@ -138,7 +138,7 @@ object ConverterUtils extends Logging { /** Convert StructType to Json */ def convertNamedStructJson(tableSchema: StructType): String = { val typeNodes = ConverterUtils.collectAttributeTypeNodes(tableSchema) - val nameList = tableSchema.fieldNames + val nameList = tableSchema.fieldNames.map(normalizeColName) val structBuilder = Type.Struct.newBuilder for (typeNode <- typeNodes.asScala) { From 06bf90e89911030348af4905283165ed3b811eb8 Mon Sep 17 00:00:00 2001 From: PHILO-HE Date: Fri, 26 Jul 2024 13:18:40 +0800 Subject: [PATCH 05/61] [Minor] Move a test from spark-3.2 module to a common test module (#6585) --- .../spark/sql/GlutenSQLQuerySuite.scala | 20 ------ .../org/apache/gluten/sql/SQLQuerySuite.scala | 66 +++++++++++++++++++ 2 files changed, 66 insertions(+), 20 deletions(-) create mode 100644 gluten-ut/test/src/test/scala/org/apache/gluten/sql/SQLQuerySuite.scala diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenSQLQuerySuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenSQLQuerySuite.scala index e5962ea149d7a..424fd1b3ec8bd 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenSQLQuerySuite.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenSQLQuerySuite.scala @@ -16,8 +16,6 @@ */ package org.apache.spark.sql -import org.apache.gluten.GlutenConfig - import org.apache.spark.SparkException import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.internal.SQLConf @@ -72,24 +70,6 @@ class GlutenSQLQuerySuite extends SQLQuerySuite with GlutenSQLTestsTrait { } } - testGluten("Support run with Vector reader in FileSourceScan or BatchScan") { - withSQLConf( - SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true", - SQLConf.CACHE_VECTORIZED_READER_ENABLED.key -> "true", - GlutenConfig.COLUMNAR_BATCHSCAN_ENABLED.key -> "false", - GlutenConfig.COLUMNAR_FILESCAN_ENABLED.key -> "false" - ) { - withTable("t1") { - sql("""CREATE TABLE t1(name STRING, id BINARY, part BINARY) - |USING PARQUET PARTITIONED BY (part)""".stripMargin) - sql("INSERT INTO t1 PARTITION(part = 'Spark SQL') VALUES('a', X'537061726B2053514C')") - checkAnswer( - sql("SELECT name, cast(id as string), cast(part as string) FROM t1"), - Row("a", "Spark SQL", "Spark SQL")) - } - } - } - testGluten("SPARK-33593: Vector reader got incorrect data with binary partition value") { Seq("false").foreach( value => { diff --git a/gluten-ut/test/src/test/scala/org/apache/gluten/sql/SQLQuerySuite.scala b/gluten-ut/test/src/test/scala/org/apache/gluten/sql/SQLQuerySuite.scala new file mode 100644 index 0000000000000..39b9ee33b9c91 --- /dev/null +++ b/gluten-ut/test/src/test/scala/org/apache/gluten/sql/SQLQuerySuite.scala @@ -0,0 +1,66 @@ +/* + * 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.gluten.sql + +import org.apache.gluten.GlutenConfig +import org.apache.gluten.execution.WholeStageTransformerSuite +import org.apache.gluten.utils.BackendTestUtils +import org.apache.gluten.utils.SystemParameters + +import org.apache.spark.SparkConf +import org.apache.spark.sql.Row +import org.apache.spark.sql.internal.SQLConf + +class SQLQuerySuite extends WholeStageTransformerSuite { + protected val resourcePath: String = null + protected val fileFormat: String = null + override protected val logLevel: String = "INFO" + + override protected def sparkConf: SparkConf = { + val conf = super.sparkConf + .set("spark.plugins", "org.apache.gluten.GlutenPlugin") + .set("spark.default.parallelism", "1") + .set("spark.memory.offHeap.enabled", "true") + .set("spark.memory.offHeap.size", "1024MB") + .set("spark.ui.enabled", "false") + .set("spark.gluten.ui.enabled", "false") + if (BackendTestUtils.isCHBackendLoaded()) { + conf + .set("spark.gluten.sql.enable.native.validation", "false") + .set(GlutenConfig.GLUTEN_LIB_PATH, SystemParameters.getClickHouseLibPath) + } + conf + } + + test("Support run with Vector reader in FileSourceScan or BatchScan") { + withSQLConf( + SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true", + SQLConf.CACHE_VECTORIZED_READER_ENABLED.key -> "true", + GlutenConfig.COLUMNAR_BATCHSCAN_ENABLED.key -> "false", + GlutenConfig.COLUMNAR_FILESCAN_ENABLED.key -> "false" + ) { + withTable("t1") { + sql("""CREATE TABLE t1(name STRING, id BINARY, part BINARY) + |USING PARQUET PARTITIONED BY (part)""".stripMargin) + sql("INSERT INTO t1 PARTITION(part = 'Spark SQL') VALUES('a', X'537061726B2053514C')") + checkAnswer( + sql("SELECT name, cast(id as string), cast(part as string) FROM t1"), + Row("a", "Spark SQL", "Spark SQL")) + } + } + } +} From 5ad819800739766565266402c21e51a1667c0583 Mon Sep 17 00:00:00 2001 From: Shuai li Date: Fri, 26 Jul 2024 14:04:36 +0800 Subject: [PATCH 06/61] [GLUTEN-6604][CH] Fix mergetree partition with whitespace error (#6605) [CH] Fix mergetree partition with whitespace error --- .../utils/MergeTreePartsPartitionsUtil.scala | 2 +- .../MergeTreeFileFormatDataWriter.scala | 3 +- .../GlutenClickHouseMergeTreeWriteSuite.scala | 28 +++++++++++++++++++ 3 files changed, 31 insertions(+), 2 deletions(-) diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/utils/MergeTreePartsPartitionsUtil.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/utils/MergeTreePartsPartitionsUtil.scala index 64e41778cb9b9..03199f7ffe838 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/utils/MergeTreePartsPartitionsUtil.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/utils/MergeTreePartsPartitionsUtil.scala @@ -166,7 +166,7 @@ object MergeTreePartsPartitionsUtil extends Logging { partition => partition.files.map( fs => { - val path = fs.getPath.toString + val path = fs.getPath.toUri.toString val ret = ClickhouseSnapshot.pathToAddMTPCache.getIfPresent(path) if (ret == null) { diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/clickhouse/MergeTreeFileFormatDataWriter.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/clickhouse/MergeTreeFileFormatDataWriter.scala index 712afb3788d10..4f522e218659d 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/clickhouse/MergeTreeFileFormatDataWriter.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/clickhouse/MergeTreeFileFormatDataWriter.scala @@ -306,7 +306,8 @@ abstract class MergeTreeBaseDynamicPartitionDataWriter( releaseCurrentWriter() } - val partDir = partitionValues.map(getPartitionPath(_)) + val partDir = + partitionValues.map(getPartitionPath(_)).map(str => new Path(str).toUri.toASCIIString) partDir.foreach(updatedPartitions.add) val bucketIdStr = bucketId.map(id => f"$id%05d").getOrElse("") diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteSuite.scala index 2563d792b0404..77d7f37c0369a 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteSuite.scala @@ -2022,5 +2022,33 @@ class GlutenClickHouseMergeTreeWriteSuite |""".stripMargin runTPCHQueryBySQL(6, sqlStr) { _ => } } + + test("test mergetree with partition with whitespace") { + spark.sql(s""" + |DROP TABLE IF EXISTS lineitem_mergetree_partition_with_whitespace; + |""".stripMargin) + + spark.sql(s""" + |CREATE TABLE IF NOT EXISTS lineitem_mergetree_partition_with_whitespace + |( + | l_returnflag string, + | l_linestatus string + |) + |USING clickhouse + |PARTITIONED BY (l_returnflag) + |LOCATION '$basePath/lineitem_mergetree_partition_with_whitespace' + |""".stripMargin) + + spark.sql(s""" + | insert into table lineitem_mergetree_partition_with_whitespace + | (l_returnflag, l_linestatus) values ('a A', 'abc') + |""".stripMargin) + + val sqlStr = + s""" + |SELECT * from lineitem_mergetree_partition_with_whitespace + |""".stripMargin + runSql(sqlStr) { _ => } + } } // scalastyle:off line.size.limit From 92a62488a1c56db4ab64760becf89991a3032827 Mon Sep 17 00:00:00 2001 From: Jaime Pan <33685703+NEUpanning@users.noreply.github.com> Date: Fri, 26 Jul 2024 16:45:23 +0800 Subject: [PATCH 07/61] [GLUTEN-6195][VL]Add unit tests for udf (#6603) --- cpp/velox/tests/CMakeLists.txt | 3 +++ cpp/velox/tests/MyUdfTest.cc | 45 ++++++++++++++++++++++++++++++++++ 2 files changed, 48 insertions(+) create mode 100644 cpp/velox/tests/MyUdfTest.cc diff --git a/cpp/velox/tests/CMakeLists.txt b/cpp/velox/tests/CMakeLists.txt index dac83cd87781b..f7bc1cb13ee79 100644 --- a/cpp/velox/tests/CMakeLists.txt +++ b/cpp/velox/tests/CMakeLists.txt @@ -57,3 +57,6 @@ add_velox_test(spark_functions_test SOURCES SparkFunctionTest.cc add_velox_test(execution_ctx_test SOURCES RuntimeTest.cc) add_velox_test(velox_memory_test SOURCES MemoryManagerTest.cc) add_velox_test(buffer_outputstream_test SOURCES BufferOutputStreamTest.cc) +if(BUILD_EXAMPLES) + add_velox_test(MyUdfTest SOURCES MyUdfTest.cc) +endif() diff --git a/cpp/velox/tests/MyUdfTest.cc b/cpp/velox/tests/MyUdfTest.cc new file mode 100644 index 0000000000000..46898b38cfd24 --- /dev/null +++ b/cpp/velox/tests/MyUdfTest.cc @@ -0,0 +1,45 @@ +/* + * 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. + */ + +#include +#include "udf/UdfLoader.h" +#include "velox/expression/VectorFunction.h" +#include "velox/functions/prestosql/tests/utils/FunctionBaseTest.h" +#include "velox/parse/TypeResolver.h" + +using namespace facebook::velox::functions::test; +using namespace facebook::velox; +class MyUdfTest : public FunctionBaseTest { + protected: + static void SetUpTestCase() { + parse::registerTypeResolver(); + auto udfLoader = gluten::UdfLoader::getInstance(); + udfLoader->loadUdfLibraries("../udf/examples/libmyudf.so"); + udfLoader->registerUdf(); + memory::MemoryManager::testingSetInstance({}); + } +}; + +TEST_F(MyUdfTest, myudf1) { + const auto myudf1 = [&](const int64_t& number) { + return evaluateOnce("myudf1(c0)", BIGINT(), std::make_optional(number)); + }; + + EXPECT_EQ(5, myudf1(0)); + EXPECT_EQ(105, myudf1(100)); + EXPECT_EQ(3147483652, myudf1(3147483647)); // int64 +} \ No newline at end of file From 2f2e662667a74f03e17b68878b2b7db8fa76d1aa Mon Sep 17 00:00:00 2001 From: PHILO-HE Date: Fri, 26 Jul 2024 16:51:22 +0800 Subject: [PATCH 08/61] [VL] Minor: Remove deprecated GHA jobs --- .github/workflows/velox_be.yml.deprecated | 700 ------------------ .../workflows/velox_tpch_bench.yml.disabled | 57 -- .../workflows/velox_tpch_merge.yml.disabled | 60 -- .github/workflows/velox_velox_ut.yml.disabled | 67 -- 4 files changed, 884 deletions(-) delete mode 100644 .github/workflows/velox_be.yml.deprecated delete mode 100644 .github/workflows/velox_tpch_bench.yml.disabled delete mode 100644 .github/workflows/velox_tpch_merge.yml.disabled delete mode 100644 .github/workflows/velox_velox_ut.yml.disabled diff --git a/.github/workflows/velox_be.yml.deprecated b/.github/workflows/velox_be.yml.deprecated deleted file mode 100644 index 6ff5ec743c140..0000000000000 --- a/.github/workflows/velox_be.yml.deprecated +++ /dev/null @@ -1,700 +0,0 @@ -# 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. - -name: Velox backend - -on: - pull_request: - paths: - - '.github/**' - - 'pom.xml' - - 'backends-velox/**' - - 'gluten-celeborn/**' - - 'gluten-core/**' - - 'gluten-data/**' - - 'gluten-delta/**' - - 'gluten-iceberg/**' - - 'gluten-ut/**' - - 'shims/**' - - 'tools/gluten-it/**' - - 'tools/gluten-te/**' - - 'ep/build-velox/**' - - 'cpp/*' - - 'cpp/CMake/**' - - 'cpp/velox/**' - - 'cpp/core/**' - - 'dev/**' - -env: - HTTP_PROXY_HOST: proxy-shz.intel.com - HTTP_PROXY_PORT: 911 - PATH_TO_GLUTEN_TE: ./tools/gluten-te - DOCKER_PULL_REGISTRY: 10.1.0.25:5000 - MAVEN_OPTS: -Dmaven.wagon.http.retryHandler.count=3 - -concurrency: - group: ${{ github.repository }}-${{ github.head_ref || github.sha }}-${{ github.workflow }} - cancel-in-progress: true - -jobs: - ubuntu2004-test-spark32: - runs-on: velox-self-hosted - env: - OS_IMAGE_NAME: ubuntu - OS_IMAGE_TAG: 20.04 - steps: - - uses: actions/checkout@v4 - - name: Setup docker container - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/checkout.sh - - name: Build Gluten velox third party - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - cd /opt/gluten/ep/build-velox/src && \ - ./get_velox.sh && \ - ./build_velox.sh --run_setup_script=ON --enable_ep_cache=OFF --build_test_utils=ON' - - name: Build Gluten CPP library - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - cd /opt/gluten/cpp && \ - ./compile.sh --build_velox_backend=ON --build_tests=ON --build_examples=ON --build_benchmarks=ON' - - name: Run CPP unit test - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh 'cd /opt/gluten/cpp/build && \ - ctest -V' - - name: Run HBM CPP unit test - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh 'cd /opt/gluten/cpp/build && \ - cmake -DBUILD_TESTS=ON -DENABLE_HBM=ON .. && \ - cmake --build . --target hbw_allocator_test -- -j && \ - ctest -V -R TestHbw' - - name: Build and run unit test for Spark 3.2.2 (other tests) - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - cd /opt/gluten && \ - mvn clean install -Pspark-3.2 -Pspark-ut -Pbackends-velox -Prss -DargLine="-Dspark.test.home=/opt/spark322" -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.SkipTestTags && \ - mvn test -Pspark-3.2 -Pbackends-velox -DtagsToExclude=None -DtagsToInclude=org.apache.gluten.tags.UDFTest' - # Cpp micro benchmarks will use generated files from unit test in backends-velox module. - - name: Run micro benchmarks - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh 'cd /opt/gluten/cpp/build/velox/benchmarks && \ - ./generic_benchmark --run-example --with-shuffle --threads 1 --iterations 1' - - name: Copy golden files from container to host - if: failure() - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/cp.sh /tmp/tpch-approved-plan/ /tmp/$GITHUB_RUN_ID/spark32/tpch-approved-plan - - name: Upload golden files - if: failure() - uses: actions/upload-artifact@v4 - with: - name: golden-files-spark32 - path: | - /tmp/${{ github.run_id }}/spark32/tpch-approved-plan/** - - name: Clean temp golden files - run: | - rm -rf /tmp/$GITHUB_RUN_ID/spark32 - - name: Exit docker container - if: ${{ always() }} - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/clean.sh - - ubuntu2004-test-spark32-slow: - runs-on: velox-self-hosted - env: - OS_IMAGE_NAME: ubuntu - OS_IMAGE_TAG: 20.04 - steps: - - uses: actions/checkout@v4 - - name: Setup docker container - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/checkout.sh - - name: Build Gluten velox third party - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - cd /opt/gluten/ep/build-velox/src && \ - ./get_velox.sh && \ - ./build_velox.sh --run_setup_script=ON --enable_ep_cache=OFF' - - name: Build Gluten CPP library - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - cd /opt/gluten/cpp && \ - ./compile.sh --build_velox_backend=ON' - - name: Build and run unit test for Spark 3.2.2 (slow tests) - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - cd /opt/gluten && \ - mvn clean install -Pspark-3.2 -Pspark-ut -Pbackends-velox -Prss -Piceberg -Pdelta -DargLine="-Dspark.test.home=/opt/spark322" -DtagsToInclude=org.apache.spark.tags.ExtendedSQLTest' - - name: TPC-H SF1.0 && TPC-DS SF1.0 Parquet local spark3.2 - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh 'cd /opt/gluten/tools/gluten-it && \ - mvn clean install -Pspark-3.2 \ - && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ - --local --preset=velox --benchmark-type=h --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ - && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ - --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1' - - name: Exit docker container - if: ${{ always() }} - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/clean.sh - - ubuntu2004-test-spark33-slow: - runs-on: velox-self-hosted - env: - OS_IMAGE_NAME: ubuntu - OS_IMAGE_TAG: 20.04 - steps: - - uses: actions/checkout@v4 - - name: Setup docker container - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/checkout.sh - - name: Build Gluten velox third party - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - cd /opt/gluten/ep/build-velox/src && \ - ./get_velox.sh && \ - ./build_velox.sh --run_setup_script=ON --enable_ep_cache=OFF' - - name: Build Gluten CPP library - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - cd /opt/gluten/cpp && \ - ./compile.sh --build_velox_backend=ON' - - name: Build and Run unit test for Spark 3.3.1 (slow tests) - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh 'cd /opt/gluten && \ - mvn clean install -Pspark-3.3 -Pbackends-velox -Prss -Piceberg -Pdelta -Pspark-ut -DargLine="-Dspark.test.home=/opt/spark331" -DtagsToInclude=org.apache.spark.tags.ExtendedSQLTest' - - name: TPC-H SF1.0 && TPC-DS SF1.0 Parquet local spark3.3 - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh 'cd /opt/gluten/tools/gluten-it && \ - mvn clean install -Pspark-3.3 \ - && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ - --local --preset=velox --benchmark-type=h --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ - && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ - --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1' - - name: TPC-H SF1.0 && TPC-DS SF1.0 Parquet local spark3.3 Q38 flush - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh 'cd /opt/gluten/tools/gluten-it \ - && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ - --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 --queries=q38 \ - --disable-bhj \ - --extra-conf=spark.gluten.sql.columnar.backend.velox.maxPartialAggregationMemoryRatio=0.1 \ - --extra-conf=spark.gluten.sql.columnar.backend.velox.maxExtendedPartialAggregationMemoryRatio=0.2 \ - --extra-conf=spark.gluten.sql.columnar.backend.velox.abandonPartialAggregationMinPct=100 \ - --extra-conf=spark.gluten.sql.columnar.backend.velox.abandonPartialAggregationMinRows=0' - - name: Exit docker container - if: ${{ always() }} - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/clean.sh - - ubuntu2004-test-spark33: - runs-on: velox-self-hosted - env: - OS_IMAGE_NAME: ubuntu - OS_IMAGE_TAG: 20.04 - steps: - - uses: actions/checkout@v4 - - name: Setup docker container - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/checkout.sh - - name: Build Gluten velox third party - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - cd /opt/gluten/ep/build-velox/src && \ - ./get_velox.sh && \ - ./build_velox.sh --run_setup_script=ON --enable_ep_cache=OFF' - - name: Build Gluten CPP library - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - cd /opt/gluten/cpp && \ - ./compile.sh --build_velox_backend=ON --build_examples=ON' - - name: Build and Run unit test for Spark 3.3.1 (other tests) - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh 'cd /opt/gluten && \ - mvn clean install -Pspark-3.3 -Pbackends-velox -Prss -Piceberg -Pdelta -Pspark-ut -DargLine="-Dspark.test.home=/opt/spark331" -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.SkipTestTags && \ - mvn test -Pspark-3.3 -Pbackends-velox -DtagsToExclude=None -DtagsToInclude=org.apache.gluten.tags.UDFTest' - - name: Copy golden files from container to host - if: failure() - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/cp.sh /tmp/tpch-approved-plan/ /tmp/$GITHUB_RUN_ID/spark33/tpch-approved-plan - - name: Upload golden files - if: failure() - uses: actions/upload-artifact@v4 - with: - name: golden-files-spark33 - path: | - /tmp/${{ github.run_id }}/spark33/tpch-approved-plan/** - - name: Clean temp golden files - run: | - rm -rf /tmp/$GITHUB_RUN_ID/spark33 - - name: Exit docker container - if: ${{ always() }} - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/clean.sh - - ubuntu2004-test-spark34-slow: - runs-on: velox-self-hosted - env: - OS_IMAGE_NAME: ubuntu - OS_IMAGE_TAG: 20.04 - steps: - - uses: actions/checkout@v4 - - name: Setup docker container - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/checkout.sh - - name: Build Gluten velox third party - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - cd /opt/gluten/ep/build-velox/src && \ - ./get_velox.sh && \ - ./build_velox.sh --run_setup_script=ON --enable_ep_cache=OFF' - - name: Build Gluten CPP library - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - cd /opt/gluten/cpp && \ - ./compile.sh --build_velox_backend=ON ' - - name: Build and Run unit test for Spark 3.4.2 (slow tests) - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh 'cd /opt/gluten && \ - mvn clean install -Pspark-3.4 -Pbackends-velox -Prss -Piceberg -Pdelta -Pspark-ut -DargLine="-Dspark.test.home=/opt/spark342" -DtagsToInclude=org.apache.spark.tags.ExtendedSQLTest' - - name: TPC-H SF1.0 && TPC-DS SF1.0 Parquet local spark3.4 - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh 'cd /opt/gluten/tools/gluten-it && \ - mvn clean install -Pspark-3.4 \ - && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ - --local --preset=velox --benchmark-type=h --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ - && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ - --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1' - - name: Exit docker container - if: ${{ always() }} - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/clean.sh - - ubuntu2004-test-spark34: - runs-on: velox-self-hosted - env: - OS_IMAGE_NAME: ubuntu - OS_IMAGE_TAG: 20.04 - steps: - - uses: actions/checkout@v4 - - name: Setup docker container - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/checkout.sh - - name: Build Gluten velox third party - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - cd /opt/gluten/ep/build-velox/src && \ - ./get_velox.sh && \ - ./build_velox.sh --run_setup_script=ON --enable_ep_cache=OFF' - - name: Build Gluten CPP library - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - cd /opt/gluten/cpp && \ - ./compile.sh --build_velox_backend=ON --build_examples=ON' - - name: Build and Run unit test for Spark 3.4.2 (other tests) - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh 'cd /opt/gluten && \ - mvn clean install -Pspark-3.4 -Pbackends-velox -Prss -Piceberg -Pdelta -Pspark-ut -DargLine="-Dspark.test.home=/opt/spark342" -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.SkipTestTags && \ - mvn test -Pspark-3.4 -Pbackends-velox -DtagsToExclude=None -DtagsToInclude=org.apache.gluten.tags.UDFTest' - - name: Copy golden files from container to host - if: failure() - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/cp.sh /tmp/tpch-approved-plan/ /tmp/$GITHUB_RUN_ID/spark34/tpch-approved-plan - - name: Upload golden files - if: failure() - uses: actions/upload-artifact@v4 - with: - name: golden-files-spark34 - path: | - /tmp/${{ github.run_id }}/spark34/tpch-approved-plan/** - - name: Clean temp golden files - run: | - rm -rf /tmp/$GITHUB_RUN_ID/spark34 - - name: Exit docker container - if: ${{ always() }} - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/clean.sh - - ubuntu2004-test-spark35: - runs-on: velox-self-hosted - env: - OS_IMAGE_NAME: ubuntu - OS_IMAGE_TAG: 20.04 - steps: - - uses: actions/checkout@v4 - - name: Setup docker container - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/checkout.sh - - name: Build Gluten velox third party - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - cd /opt/gluten/ep/build-velox/src && \ - ./get_velox.sh && \ - ./build_velox.sh --run_setup_script=ON --enable_ep_cache=OFF' - - name: Build Gluten CPP library - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - cd /opt/gluten/cpp && \ - ./compile.sh --build_velox_backend=ON ' - - name: Build for Spark 3.5 - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - cd /opt/gluten && \ - mvn clean install -Pspark-3.5 -Pbackends-velox -Prss -DskipTests' - - name: TPC-H SF1.0 && TPC-DS SF1.0 Parquet local spark3.5 - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh 'cd /opt/gluten/tools/gluten-it && \ - mvn clean install -Pspark-3.5 \ - && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ - --local --preset=velox --benchmark-type=h --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ - && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ - --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1' - - name: Exit docker container - if: ${{ always() }} - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/clean.sh - - ubuntu2204-test-spark33-spark34: - runs-on: velox-self-hosted - env: - OS_IMAGE_NAME: ubuntu - OS_IMAGE_TAG: 22.04 - steps: - - uses: actions/checkout@v4 - - name: Setup docker container - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/checkout.sh - - name: Build Gluten velox third party - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - cd /opt/gluten/ep/build-velox/src && \ - ./get_velox.sh --enable_hdfs=ON --enable_s3=ON --enable_gcs=ON --enable_abfs=ON && \ - ./build_velox.sh --run_setup_script=ON --enable_ep_cache=OFF --enable_hdfs=ON --enable_s3=ON --enable_gcs=ON --enable_abfs=ON' - - name: Build Gluten CPP library - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - cd /opt/gluten/cpp && \ - ./compile.sh --build_velox_backend=ON --enable_hdfs=ON --enable_s3=ON --enable_gcs=ON --enable_abfs=ON' - - name: Build for Spark 3.3.1 - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - cd /opt/gluten && \ - mvn clean install -Pspark-3.3 -Pbackends-velox -Prss -Piceberg -Pdelta -DskipTests' - - name: TPC-H SF1.0 && TPC-DS SF10.0 Parquet local spark3.3 - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh 'cd /opt/gluten/tools/gluten-it && \ - mvn clean install -Pspark-3.3 \ - && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ - --local --preset=velox --benchmark-type=h --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ - && GLUTEN_IT_JVM_ARGS=-Xmx20G sbin/gluten-it.sh queries-compare \ - --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=30g -s=10.0 --threads=32 --iterations=1' - - name: Build for Spark 3.4.2 - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - cd /opt/gluten && \ - mvn clean install -Pspark-3.4 -Pbackends-velox -Prss -Piceberg -Pdelta -DskipTests' - - name: TPC-H SF1.0 && TPC-DS SF10.0 Parquet local spark3.4 - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh 'cd /opt/gluten/tools/gluten-it && \ - mvn clean install -Pspark-3.4 \ - && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ - --local --preset=velox --benchmark-type=h --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ - && GLUTEN_IT_JVM_ARGS=-Xmx20G sbin/gluten-it.sh queries-compare \ - --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=30g -s=10.0 --threads=32 --iterations=1' - - name: Exit docker container - if: ${{ always() }} - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/clean.sh - - ubuntu2204-test: - runs-on: velox-self-hosted - env: - OS_IMAGE_NAME: ubuntu - OS_IMAGE_TAG: 22.04 - steps: - - uses: actions/checkout@v4 - - name: Setup docker container - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/checkout.sh - - name: Build Gluten velox third party - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - cd /opt/gluten/ep/build-velox/src && \ - ./get_velox.sh --enable_hdfs=ON --enable_s3=ON --enable_gcs=ON --enable_abfs=ON && \ - ./build_velox.sh --run_setup_script=ON --enable_ep_cache=OFF --enable_hdfs=ON --enable_s3=ON --enable_gcs=ON --enable_abfs=ON' - - name: Build Gluten CPP library - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - cd /opt/gluten/cpp && \ - ./compile.sh --build_velox_backend=ON --enable_hdfs=ON --enable_s3=ON --enable_gcs=ON --enable_abfs=ON' - - name: Build for Spark 3.2.2 - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - cd /opt/gluten && \ - mvn clean install -Pspark-3.2 -Pbackends-velox -Prss -Piceberg -Pdelta -DskipTests' - - name: TPC-H SF1.0 && TPC-DS SF10.0 Parquet local spark3.2 - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh 'cd /opt/gluten/tools/gluten-it && \ - mvn clean install -Pspark-3.2 \ - && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ - --local --preset=velox --benchmark-type=h --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ - && GLUTEN_IT_JVM_ARGS=-Xmx20G sbin/gluten-it.sh queries-compare \ - --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=40g -s=10.0 --threads=32 --iterations=1' - - name: TPC-H SF1.0 && TPC-DS SF10.0 Parquet local spark3.2 with Celeborn 0.4.0 - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh \ - 'cd /opt && mkdir -p celeborn && \ - tar xzf apache-celeborn-0.4.0-incubating-bin.tgz -C /opt/celeborn --strip-components=1 && cd celeborn && \ - mv ./conf/celeborn-env.sh.template ./conf/celeborn-env.sh && \ - echo -e "CELEBORN_MASTER_MEMORY=4g\nCELEBORN_WORKER_MEMORY=4g\nCELEBORN_WORKER_OFFHEAP_MEMORY=8g" > ./conf/celeborn-env.sh && \ - echo -e "celeborn.worker.commitFiles.threads 128\nceleborn.worker.sortPartition.threads 64" > ./conf/celeborn-defaults.conf \ - && bash ./sbin/start-master.sh && bash ./sbin/start-worker.sh && \ - cd /opt/gluten/tools/gluten-it && mvn clean install -Pspark-3.2,rss,celeborn-0.4 \ - && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ - --local --preset=velox-with-celeborn --benchmark-type=h --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ - && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ - --local --preset=velox-with-celeborn --benchmark-type=ds --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 && \ - bash /opt/celeborn/sbin/stop-worker.sh \ - && bash /opt/celeborn/sbin/stop-master.sh && rm -rf /opt/celeborn' - - name: TPC-H SF1.0 && TPC-DS SF10.0 Parquet local spark3.2 with Celeborn 0.3.2 - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh \ - 'cd /opt && mkdir -p celeborn && \ - tar xzf apache-celeborn-0.3.2-incubating-bin.tgz -C /opt/celeborn --strip-components=1 && cd celeborn && \ - mv ./conf/celeborn-env.sh.template ./conf/celeborn-env.sh && \ - echo -e "CELEBORN_MASTER_MEMORY=4g\nCELEBORN_WORKER_MEMORY=4g\nCELEBORN_WORKER_OFFHEAP_MEMORY=8g" > ./conf/celeborn-env.sh && \ - echo -e "celeborn.worker.commitFiles.threads 128\nceleborn.worker.sortPartition.threads 64" > ./conf/celeborn-defaults.conf \ - && bash ./sbin/start-master.sh && bash ./sbin/start-worker.sh && \ - cd /opt/gluten/tools/gluten-it && mvn clean install -Pspark-3.2,rss \ - && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ - --local --preset=velox-with-celeborn --benchmark-type=h --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ - && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ - --local --preset=velox-with-celeborn --benchmark-type=ds --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 && \ - bash /opt/celeborn/sbin/stop-worker.sh \ - && bash /opt/celeborn/sbin/stop-master.sh' - - name: Exit docker container - if: ${{ always() }} - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/clean.sh - - centos8-test: - runs-on: velox-self-hosted - env: - OS_IMAGE_NAME: centos - OS_IMAGE_TAG: 8 - steps: - - uses: actions/checkout@v4 - - name: Setup docker container - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/checkout.sh - - name: Build Gluten velox third party - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - source /env.sh && \ - sudo yum -y install patch && \ - cd /opt/gluten/ep/build-velox/src && \ - ./get_velox.sh --enable_hdfs=ON --enable_s3=ON --enable_gcs=ON --enable_abfs=ON && \ - ./build_velox.sh --run_setup_script=ON --enable_ep_cache=OFF --enable_hdfs=ON --enable_s3=ON --enable_gcs=ON --enable_abfs=ON' - - name: Build Gluten CPP library - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - source /env.sh && \ - cd /opt/gluten/cpp && \ - ./compile.sh --build_velox_backend=ON --enable_hdfs=ON --enable_s3=ON --enable_gcs=ON --enable_abfs=ON' - - name: Build for Spark 3.2.2 - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - cd /opt/gluten && \ - mvn clean install -Pspark-3.2 -Pbackends-velox -Prss -Piceberg -Pdelta -DskipTests' - - name: TPC-H SF1.0 && TPC-DS SF30.0 Parquet local spark3.2 - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh 'cd /opt/gluten/tools/gluten-it && \ - mvn clean install -Pspark-3.2 \ - && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ - --local --preset=velox --benchmark-type=h --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ - && GLUTEN_IT_JVM_ARGS=-Xmx50G sbin/gluten-it.sh queries-compare \ - --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=50g -s=30.0 --threads=32 --iterations=1' - - name: TPC-H SF1.0 && TPC-DS SF30.0 Parquet local spark3.2 random kill tasks - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh 'cd /opt/gluten/tools/gluten-it && \ - mvn clean install -Pspark-3.2 \ - && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries \ - --local --preset=velox --benchmark-type=h --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 --data-gen-strategy=skip --random-kill-tasks \ - && GLUTEN_IT_JVM_ARGS=-Xmx50G sbin/gluten-it.sh queries \ - --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=50g -s=30.0 --threads=32 --iterations=1 --data-gen-strategy=skip --random-kill-tasks' - - name: Exit docker container - if: ${{ always() }} - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/clean.sh - - centos7-test: - runs-on: velox-self-hosted - env: - OS_IMAGE_NAME: centos - OS_IMAGE_TAG: 7 - steps: - - uses: actions/checkout@v4 - - name: Setup docker container - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/checkout.sh - - name: Build Gluten velox third party - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - yum -y install epel-release centos-release-scl patch sudo && \ - cd /opt/gluten/ep/build-velox/src && \ - source /opt/rh/devtoolset-9/enable && \ - ./get_velox.sh --enable_hdfs=ON --enable_s3=ON --enable_gcs=ON --enable_abfs=ON && \ - ./build_velox.sh --run_setup_script=ON --enable_ep_cache=OFF --enable_s3=ON --enable_gcs=ON --enable_abfs=ON --enable_hdfs=ON' - - name: Build Gluten CPP library - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - cd /opt/gluten/cpp && \ - source /opt/rh/devtoolset-9/enable && \ - ./compile.sh --build_velox_backend=ON --enable_hdfs=ON --enable_s3=ON --enable_gcs=ON --enable_abfs=ON' - - name: Build for Spark 3.2.2 - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - cd /opt/gluten && \ - mvn clean install -Pspark-3.2 -Pbackends-velox -Prss -Piceberg -Pdelta -DskipTests' - - name: TPC-H SF1.0 && TPC-DS SF30.0 Parquet local spark3.2 - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh 'cd /opt/gluten/tools/gluten-it && \ - mvn clean install -Pspark-3.2 \ - && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ - --local --preset=velox --benchmark-type=h --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ - && GLUTEN_IT_JVM_ARGS=-Xmx50G sbin/gluten-it.sh queries-compare \ - --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=50g -s=30.0 --threads=32 --iterations=1' - - name: TPC-DS SF30.0 Parquet local spark3.2 Q67/Q95 low memory, memory isolation off - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh 'cd /opt/gluten/tools/gluten-it && \ - mvn clean install -Pspark-3.2 \ - && GLUTEN_IT_JVM_ARGS=-Xmx50G sbin/gluten-it.sh parameterized \ - --local --preset=velox --benchmark-type=ds --error-on-memleak --queries=q67,q95 -s=30.0 --threads=12 --shuffle-partitions=72 --iterations=1 \ - --data-gen-strategy=skip -m=OffHeapExecutionMemory \ - -d=ISOLATION:OFF,spark.gluten.memory.isolation=false \ - -d=OFFHEAP_SIZE:5g,spark.memory.offHeap.size=5g \ - -d=OFFHEAP_SIZE:3g,spark.memory.offHeap.size=3g \ - -d=OVER_ACQUIRE:0.3,spark.gluten.memory.overAcquiredMemoryRatio=0.3 \ - -d=OVER_ACQUIRE:0.5,spark.gluten.memory.overAcquiredMemoryRatio=0.5' - - name: (To be fixed) TPC-DS SF30.0 Parquet local spark3.2 Q67/Q95 low memory, memory isolation on - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh 'cd /opt/gluten/tools/gluten-it && \ - mvn clean install -Pspark-3.2 \ - && GLUTEN_IT_JVM_ARGS=-Xmx50G sbin/gluten-it.sh parameterized \ - --local --preset=velox --benchmark-type=ds --error-on-memleak --queries=q67,q95 -s=30.0 --threads=12 --shuffle-partitions=72 --iterations=1 \ - --data-gen-strategy=skip -m=OffHeapExecutionMemory \ - -d=ISOLATION:ON,spark.gluten.memory.isolation=true,spark.memory.storageFraction=0.1 \ - -d=OFFHEAP_SIZE:5g,spark.memory.offHeap.size=5g \ - -d=OFFHEAP_SIZE:3g,spark.memory.offHeap.size=3g \ - -d=OVER_ACQUIRE:0.3,spark.gluten.memory.overAcquiredMemoryRatio=0.3 \ - -d=OVER_ACQUIRE:0.5,spark.gluten.memory.overAcquiredMemoryRatio=0.5' || true - - name: TPC-DS SF30.0 Parquet local spark3.2 Q23A/Q23B low memory - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh 'cd /opt/gluten/tools/gluten-it && \ - GLUTEN_IT_JVM_ARGS=-Xmx50G sbin/gluten-it.sh parameterized \ - --local --preset=velox --benchmark-type=ds --error-on-memleak --queries=q23a,q23b -s=30.0 --threads=12 --shuffle-partitions=72 --iterations=1 \ - --data-gen-strategy=skip -m=OffHeapExecutionMemory \ - -d=ISOLATION:OFF,spark.gluten.memory.isolation=false \ - -d=ISOLATION:ON,spark.gluten.memory.isolation=true,spark.memory.storageFraction=0.1 \ - -d=OFFHEAP_SIZE:2g,spark.memory.offHeap.size=2g \ - -d=FLUSH_MODE:DISABLED,spark.gluten.sql.columnar.backend.velox.flushablePartialAggregation=false,spark.gluten.sql.columnar.backend.velox.maxPartialAggregationMemoryRatio=1.0,spark.gluten.sql.columnar.backend.velox.maxExtendedPartialAggregationMemoryRatio=1.0,spark.gluten.sql.columnar.backend.velox.abandonPartialAggregationMinPct=100,spark.gluten.sql.columnar.backend.velox.abandonPartialAggregationMinRows=0 \ - -d=FLUSH_MODE:ABANDONED,spark.gluten.sql.columnar.backend.velox.maxPartialAggregationMemoryRatio=1.0,spark.gluten.sql.columnar.backend.velox.maxExtendedPartialAggregationMemoryRatio=1.0,spark.gluten.sql.columnar.backend.velox.abandonPartialAggregationMinPct=0,spark.gluten.sql.columnar.backend.velox.abandonPartialAggregationMinRows=0 \ - -d=FLUSH_MODE:FLUSHED,spark.gluten.sql.columnar.backend.velox.maxPartialAggregationMemoryRatio=0.05,spark.gluten.sql.columnar.backend.velox.maxExtendedPartialAggregationMemoryRatio=0.1,spark.gluten.sql.columnar.backend.velox.abandonPartialAggregationMinPct=100,spark.gluten.sql.columnar.backend.velox.abandonPartialAggregationMinRows=0' - - name: (To be fixed) TPC-DS SF30.0 Parquet local spark3.2 Q97 low memory # The case currently causes crash with "free: invalid size". - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh 'cd /opt/gluten/tools/gluten-it && \ - GLUTEN_IT_JVM_ARGS=-Xmx50G sbin/gluten-it.sh parameterized \ - --local --preset=velox --benchmark-type=ds --error-on-memleak --queries=q97 -s=30.0 --threads=12 --shuffle-partitions=72 --iterations=1 \ - --data-gen-strategy=skip -m=OffHeapExecutionMemory \ - -d=ISOLATION:OFF,spark.gluten.memory.isolation=false \ - -d=ISOLATION:ON,spark.gluten.memory.isolation=true,spark.memory.storageFraction=0.1 \ - -d=OFFHEAP_SIZE:2g,spark.memory.offHeap.size=2g \ - -d=OFFHEAP_SIZE:1g,spark.memory.offHeap.size=1g' || true - - name: Exit docker container - if: ${{ always() }} - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/clean.sh - - - static-build-centos7-test: - runs-on: velox-self-hosted - steps: - - uses: actions/checkout@v4 - - name: Setup docker container - run: | - docker run --rm --init --privileged --ulimit nofile=65536:65536 --ulimit core=-1 --security-opt seccomp=unconfined \ - -v $PWD:/opt/gluten --name static-build-test-$GITHUB_RUN_ID -e NUM_THREADS=30 -detach $DOCKER_PULL_REGISTRY/gluten-te/gluten-buildenv-centos:7 \ - bash -c 'cd /opt/gluten && sleep 14400' - - name: Build Gluten CPP library - run: | - docker exec -i static-build-test-$GITHUB_RUN_ID bash -c ' - source /env.sh && \ - sudo yum -y install patch && \ - cd /opt/gluten && \ - sudo -E ./dev/vcpkg/setup-build-depends.sh && \ - source ./dev/vcpkg/env.sh && \ - ./dev/builddeps-veloxbe.sh --enable_vcpkg=ON --build_tests=OFF --build_benchmarks=OFF --enable_s3=ON \ - --enable_gcs=ON --enable_hdfs=ON --enable_abfs=ON --build_type=Debug' - - name: Build for Spark 3.2.2 - run: | - docker exec static-build-test-$GITHUB_RUN_ID bash -c ' - cd /opt/gluten && \ - mvn clean install -Pspark-3.2 -Pbackends-velox -Prss -Piceberg -Pdelta -DskipTests && \ - cd /opt/gluten/tools/gluten-it && \ - mvn clean install -Pspark-3.2' - - name: TPC-H SF1.0 && TPC-DS SF1.0 Parquet local spark3.2 (ubuntu 20.04) - run: | - docker run --rm --init --privileged --ulimit nofile=65536:65536 --ulimit core=-1 --security-opt seccomp=unconfined \ - -v $PWD:/opt/gluten --name static-build-test-$GITHUB_RUN_ID-tpc -e NUM_THREADS=30 ubuntu:20.04 \ - bash -c 'apt-get update -y && DEBIAN_FRONTEND=noninteractive apt-get install openjdk-8-jre -y \ - && cd /opt/gluten/tools/gluten-it \ - && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ - --local --preset=velox --benchmark-type=h --error-on-memleak --disable-aqe --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ - && GLUTEN_IT_JVM_ARGS=-Xmx10G sbin/gluten-it.sh queries-compare \ - --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=20g -s=1.0 --threads=32 --iterations=1' - - name: TPC-H SF1.0 && TPC-DS SF1.0 Parquet local spark3.2 (ubuntu 22.04) - run: | - docker run --rm --init --privileged --ulimit nofile=65536:65536 --ulimit core=-1 --security-opt seccomp=unconfined \ - -v $PWD:/opt/gluten --name static-build-test-$GITHUB_RUN_ID-tpc -e NUM_THREADS=30 ubuntu:22.04 \ - bash -c 'apt-get update -y && DEBIAN_FRONTEND=noninteractive apt-get install openjdk-8-jre -y \ - && cd /opt/gluten/tools/gluten-it \ - && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ - --local --preset=velox --benchmark-type=h --error-on-memleak --disable-aqe --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ - && GLUTEN_IT_JVM_ARGS=-Xmx10G sbin/gluten-it.sh queries-compare \ - --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=20g -s=1.0 --threads=32 --iterations=1' - - name: Exit docker container - if: ${{ always() }} - run: | - docker stop static-build-test-$GITHUB_RUN_ID || true - - build-script-test: - runs-on: velox-self-hosted - env: - OS_IMAGE_NAME: centos - OS_IMAGE_TAG: 8 - steps: - - uses: actions/checkout@v4 - - name: Setup docker container - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/checkout.sh - - name: Build Script Test - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/exec.sh ' - source /env.sh && \ - sudo yum -y install patch && \ - cd /opt/gluten/ && \ - ./dev/package.sh' - - name: Exit docker container - if: ${{ always() }} - run: | - $PATH_TO_GLUTEN_TE/$OS_IMAGE_NAME/gha/gha-checkout/clean.sh diff --git a/.github/workflows/velox_tpch_bench.yml.disabled b/.github/workflows/velox_tpch_bench.yml.disabled deleted file mode 100644 index 6be12305c623b..0000000000000 --- a/.github/workflows/velox_tpch_bench.yml.disabled +++ /dev/null @@ -1,57 +0,0 @@ -# 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. - -name: Velox backend Performance Tests On Comments - -on: - issue_comment: - types: [created, edited, deleted] - -env: - JENKINS_HOST: 10.0.0.68 - JENKINS_PORT: 8080 - JENKINS_USER: jenkins:11fd1b5a82bfd638bd9b3749c96b324ff2 - -jobs: - velox-trigger-benchmark-on-comment: - # check the comment if it contains the keywords - if: ${{ github.event.issue.pull_request && contains(github.event.comment.body, '/Benchmark Velox') }} - runs-on: velox-self-hosted - container: ubuntu:22.04 - steps: - - uses: actions/checkout@v4 - - run: apt-get update && apt-get install ca-certificates -y && update-ca-certificates - - run: sed -i 's/http\:\/\/archive.ubuntu.com/https\:\/\/mirrors.ustc.edu.cn/g' /etc/apt/sources.list - - run: apt-get update - - run: apt-get install -y curl - - name: trigger_tpch_bench - run: | - branchname=origin/pr/${{ github.event.issue.number }}/head - curl --noproxy $JENKINS_HOST -L -X POST http://$JENKINS_HOST:$JENKINS_PORT/job/Gluten_Perf_Github_Action_Check/build --user $JENKINS_USER --data-urlencode json='{"parameter": [{"name":"sha1", "value":"'$branchname'"}]}' - velox-trigger-tpcds-on-comment: - # check the comment if it contains the keywords - if: ${{ github.event.issue.pull_request && contains(github.event.comment.body, '/Benchmark Velox TPCDS') }} - runs-on: velox-self-hosted - container: ubuntu:22.04 - steps: - - uses: actions/checkout@v4 - - run: apt-get update && apt-get install ca-certificates -y && update-ca-certificates - - run: sed -i 's/http\:\/\/archive.ubuntu.com/https\:\/\/mirrors.ustc.edu.cn/g' /etc/apt/sources.list - - run: apt-get update - - run: apt-get install -y curl - - name: trigger_tpch_bench - run: | - branchname=origin/pr/${{ github.event.issue.number }}/head - curl --noproxy $JENKINS_HOST -L -X POST http://$JENKINS_HOST:$JENKINS_PORT/job/Gluten_Perf_DS_Github_Action_Check/build --user $JENKINS_USER --data-urlencode json='{"parameter": [{"name":"sha1", "value":"'$branchname'"}]}' diff --git a/.github/workflows/velox_tpch_merge.yml.disabled b/.github/workflows/velox_tpch_merge.yml.disabled deleted file mode 100644 index 5012e830111ea..0000000000000 --- a/.github/workflows/velox_tpch_merge.yml.disabled +++ /dev/null @@ -1,60 +0,0 @@ -# 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. - -name: Velox backend Performance Tests on Merge - -on: - pull_request: - types: [ closed ] - branches: - - main - paths: - - '.github/**' - - 'pom.xml' - - 'backends-velox/**' - - 'gluten-celeborn/**' - - 'gluten-core/**' - - 'gluten-data/**' - - 'gluten-ut/**' - - 'shims/**' - - 'tools/gluten-it/**' - - 'tools/gluten-te/**' - - 'ep/build-velox/**' - - 'cpp/*' - - 'cpp/CMake/**' - - 'cpp/velox/**' - - 'cpp/core/**' - - 'dev/**' - -env: - JENKINS_HOST: 10.0.0.68 - JENKINS_PORT: 8080 - JENKINS_USER: jenkins:11fd1b5a82bfd638bd9b3749c96b324ff2 - -jobs: - velox-trigger-benchmark: - if: github.event.pull_request.merged == true - runs-on: velox-self-hosted - container: ubuntu:22.04 - steps: - - uses: actions/checkout@v4 - - run: apt-get update && apt-get install ca-certificates -y && update-ca-certificates - - run: sed -i 's/http\:\/\/archive.ubuntu.com/https\:\/\/mirrors.ustc.edu.cn/g' /etc/apt/sources.list - - run: apt-get update - - run: apt-get install -y curl - - name: trigger_tpch_bench - run: | - branchname=origin/pr/${{github.event.number}}/head - curl --noproxy $JENKINS_HOST -L -X POST http://$JENKINS_HOST:$JENKINS_PORT/job/Gluten_Perf_Github_Action_Check/build --user $JENKINS_USER --data-urlencode json='{"parameter": [{"name":"sha1", "value":"'$branchname'"}]}' diff --git a/.github/workflows/velox_velox_ut.yml.disabled b/.github/workflows/velox_velox_ut.yml.disabled deleted file mode 100644 index efdf512983d6d..0000000000000 --- a/.github/workflows/velox_velox_ut.yml.disabled +++ /dev/null @@ -1,67 +0,0 @@ -# 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. - -name: Velox backend Velox Unit test - - -on: - pull_request: - paths: - - '.github/workflows/velox_velox_ut.yml' - - 'dev/**' - - 'ep/**' #get_velox change - -concurrency: - group: ${{ github.repository }}-${{ github.head_ref || github.sha }}-${{ github.workflow }} - cancel-in-progress: true - -jobs: - run-velox-unit-test: - runs-on: ubuntu-20.04 - env: - CCACHE_DIR: "${{ github.workspace }}/.ccache" - container: ghcr.io/facebookincubator/velox-dev:amd64-ubuntu-22.04-avx - steps: - - uses: actions/checkout@v2 - - name: Get Ccache - uses: actions/cache/restore@v3 - with: - path: '${{ env.CCACHE_DIR }}' - key: ccache-ubuntu-release-default - - name: Ensure Cache Dirs Exists - working-directory: ${{ github.workspace }} - run: | - mkdir -p '${{ env.CCACHE_DIR }}' - - name: Build Gluten velox third party - run: | - rm -rf /opt/miniconda-for-velox/ - cd ep/build-velox/src && \ - ./get_velox.sh - cd ../build/velox_ep/ - git reset --hard - make EXTRA_CMAKE_FLAGS="-DVELOX_ENABLE_PARQUET=ON -DVELOX_BUILD_TESTING=ON -DVELOX_BUILD_TEST_UTILS=ON" - - - name: CCache after - run: | - ccache -vs - - - uses: actions/cache/save@v3 - with: - path: '${{ env.CCACHE_DIR }}' - key: ccache-ubuntu-release-default - - - name: Run Tests - run: | - cd ${GITHUB_WORKSPACE}/ep/build-velox/build/velox_ep/_build/release && ctest -E "velox_cache_test|velox_exec_test" -j 4 --output-on-failure --no-tests=error From bfefae3f767fc0379c5b73277205211ff873aa0b Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Fri, 26 Jul 2024 16:54:24 +0800 Subject: [PATCH 09/61] [VL] Daily Update Velox Version (2024_07_26) (#6597) --- ep/build-velox/src/get_velox.sh | 2 +- ep/build-velox/src/modify_velox.patch | 26 ++++++++++--------- .../utils/velox/VeloxTestSettings.scala | 2 ++ .../utils/velox/VeloxTestSettings.scala | 2 ++ .../utils/velox/VeloxTestSettings.scala | 2 ++ .../utils/velox/VeloxTestSettings.scala | 2 ++ 6 files changed, 23 insertions(+), 13 deletions(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 1315f11248545..841787d64c237 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_07_25-1 +VELOX_BRANCH=2024_07_26 VELOX_HOME="" OS=`uname -s` diff --git a/ep/build-velox/src/modify_velox.patch b/ep/build-velox/src/modify_velox.patch index 5fd8b8b1eda8e..533b493539e4c 100644 --- a/ep/build-velox/src/modify_velox.patch +++ b/ep/build-velox/src/modify_velox.patch @@ -96,10 +96,10 @@ index 56b673e87..ef48ae9d9 100644 SOURCE_SUBDIR cpp CMAKE_ARGS ${ARROW_CMAKE_ARGS} diff --git a/CMakeLists.txt b/CMakeLists.txt -index bb7c49907..3372d48b4 100644 +index 2dc95f972..391485879 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt -@@ -234,10 +234,15 @@ if(VELOX_ENABLE_ABFS) +@@ -236,10 +236,15 @@ if(VELOX_ENABLE_ABFS) endif() if(VELOX_ENABLE_HDFS) @@ -119,7 +119,7 @@ index bb7c49907..3372d48b4 100644 add_definitions(-DVELOX_ENABLE_HDFS3) endif() -@@ -378,7 +383,7 @@ resolve_dependency(Boost 1.77.0 COMPONENTS ${BOOST_INCLUDE_LIBRARIES}) +@@ -380,7 +385,7 @@ resolve_dependency(Boost 1.77.0 COMPONENTS ${BOOST_INCLUDE_LIBRARIES}) # for reference. find_package(range-v3) set_source(gflags) @@ -129,15 +129,17 @@ index bb7c49907..3372d48b4 100644 # This is a bit convoluted, but we want to be able to use gflags::gflags as a # target even when velox is built as a subproject which uses diff --git a/velox/common/process/tests/CMakeLists.txt b/velox/common/process/tests/CMakeLists.txt -index 6797697a1..3e241f8f7 100644 +index f5e6aae72..4c4afe70b 100644 --- a/velox/common/process/tests/CMakeLists.txt +++ b/velox/common/process/tests/CMakeLists.txt -@@ -18,4 +18,4 @@ add_executable(velox_process_test ProfilerTest.cpp ThreadLocalRegistryTest.cpp - add_test(velox_process_test velox_process_test) - - target_link_libraries(velox_process_test PRIVATE velox_process fmt::fmt gtest -- velox_time gtest_main) -+ velox_time gtest_main glog::glog gflags::gflags) +@@ -24,4 +24,6 @@ target_link_libraries( + fmt::fmt + gtest + velox_time +- gtest_main) ++ gtest_main ++ glog::glog ++ gflags::gflags) diff --git a/velox/connectors/hive/storage_adapters/abfs/RegisterAbfsFileSystem.cpp b/velox/connectors/hive/storage_adapters/abfs/RegisterAbfsFileSystem.cpp index e2a638df6..e383cf205 100644 --- a/velox/connectors/hive/storage_adapters/abfs/RegisterAbfsFileSystem.cpp @@ -164,10 +166,10 @@ index 10ee508ba..027a58ecc 100644 } diff --git a/velox/dwio/parquet/writer/arrow/tests/CMakeLists.txt b/velox/dwio/parquet/writer/arrow/tests/CMakeLists.txt -index 2cabfc29a..54329ce23 100644 +index 97266c253..11d88dcc4 100644 --- a/velox/dwio/parquet/writer/arrow/tests/CMakeLists.txt +++ b/velox/dwio/parquet/writer/arrow/tests/CMakeLists.txt -@@ -39,7 +39,9 @@ target_link_libraries( +@@ -38,7 +38,9 @@ target_link_libraries( gtest gtest_main arrow diff --git a/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index d5e8df63869ef..f8a075ec0c41f 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -220,6 +220,8 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("to_timestamp exception mode") // Replaced by a gluten test to pass timezone through config. .exclude("from_unixtime") + // https://github.com/facebookincubator/velox/pull/10563/files#diff-140dc50e6dac735f72d29014da44b045509df0dd1737f458de1fe8cfd33d8145 + .excludeGlutenTest("from_unixtime") enableSuite[GlutenDecimalExpressionSuite] enableSuite[GlutenDecimalPrecisionSuite] enableSuite[GlutenStringFunctionsSuite] diff --git a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index fcc2bd3432623..6c21fc7cc2ba1 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -135,6 +135,8 @@ class VeloxTestSettings extends BackendTestSettings { // Replaced by a gluten test to pass timezone through config. .exclude("from_unixtime") .exclude("test timestamp add") + // https://github.com/facebookincubator/velox/pull/10563/files#diff-140dc50e6dac735f72d29014da44b045509df0dd1737f458de1fe8cfd33d8145 + .excludeGlutenTest("from_unixtime") enableSuite[GlutenDecimalExpressionSuite] enableSuite[GlutenDecimalPrecisionSuite] enableSuite[GlutenHashExpressionsSuite] diff --git a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index 52efa4b254424..95a5fe107f474 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -113,6 +113,8 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("to_timestamp exception mode") // Replaced by a gluten test to pass timezone through config. .exclude("from_unixtime") + // https://github.com/facebookincubator/velox/pull/10563/files#diff-140dc50e6dac735f72d29014da44b045509df0dd1737f458de1fe8cfd33d8145 + .excludeGlutenTest("from_unixtime") enableSuite[GlutenDecimalExpressionSuite] enableSuite[GlutenDecimalPrecisionSuite] enableSuite[GlutenHashExpressionsSuite] diff --git a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index 71348c823f125..e8d8730e93663 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -114,6 +114,8 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("to_timestamp exception mode") // Replaced by a gluten test to pass timezone through config. .exclude("from_unixtime") + // https://github.com/facebookincubator/velox/pull/10563/files#diff-140dc50e6dac735f72d29014da44b045509df0dd1737f458de1fe8cfd33d8145 + .excludeGlutenTest("from_unixtime") enableSuite[GlutenDecimalExpressionSuite] enableSuite[GlutenDecimalPrecisionSuite] enableSuite[GlutenHashExpressionsSuite] From 39ea7a9ce990383e34463c5acc8e11f3ccf43227 Mon Sep 17 00:00:00 2001 From: Kyligence Git Date: Sat, 27 Jul 2024 02:49:19 -0500 Subject: [PATCH 10/61] [GLUTEN-1632][CH]Daily Update Clickhouse Version (20240727) (#6611) * [GLUTEN-1632][CH]Daily Update Clickhouse Version (20240727) * fix ut due to https://github.com/ClickHouse/ClickHouse/pull/67149 --------- Co-authored-by: kyligence-git Co-authored-by: Chang Chen --- cpp-ch/clickhouse.version | 4 ++-- cpp-ch/local-engine/Common/CHUtil.cpp | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version index 082ea747bc622..a67f0227c0eb3 100644 --- a/cpp-ch/clickhouse.version +++ b/cpp-ch/clickhouse.version @@ -1,4 +1,4 @@ CH_ORG=Kyligence -CH_BRANCH=rebase_ch/20240719 -CH_COMMIT=3a2e993b1d4 +CH_BRANCH=rebase_ch/20240727 +CH_COMMIT=7d0829e25b9 diff --git a/cpp-ch/local-engine/Common/CHUtil.cpp b/cpp-ch/local-engine/Common/CHUtil.cpp index 1be9c09a8c266..787277dbefb18 100644 --- a/cpp-ch/local-engine/Common/CHUtil.cpp +++ b/cpp-ch/local-engine/Common/CHUtil.cpp @@ -766,7 +766,7 @@ void BackendInitializerUtil::initSettings(std::map & b settings.set("input_format_parquet_import_nested", true); settings.set("input_format_json_read_numbers_as_strings", true); settings.set("input_format_json_read_bools_as_numbers", false); - settings.set("input_format_json_ignore_key_case", true); + settings.set("input_format_json_case_insensitive_column_matching", true); settings.set("input_format_csv_trim_whitespaces", false); settings.set("input_format_csv_allow_cr_end_of_line", true); settings.set("output_format_orc_string_as_string", true); From bbb2e2f5e87cc726020cab45777232511b7b9d85 Mon Sep 17 00:00:00 2001 From: Mingliang Zhu Date: Sat, 27 Jul 2024 22:25:44 +0800 Subject: [PATCH 11/61] [VL] Fix std::min params type mismatch in Apple clang 15 (#6593) --- cpp/velox/shuffle/VeloxSortShuffleWriter.cc | 6 +++--- cpp/velox/shuffle/VeloxSortShuffleWriter.h | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/cpp/velox/shuffle/VeloxSortShuffleWriter.cc b/cpp/velox/shuffle/VeloxSortShuffleWriter.cc index cb81ea5f6c8a2..7c033fb98bf81 100644 --- a/cpp/velox/shuffle/VeloxSortShuffleWriter.cc +++ b/cpp/velox/shuffle/VeloxSortShuffleWriter.cc @@ -161,7 +161,7 @@ arrow::Status VeloxSortShuffleWriter::insert(const facebook::velox::RowVectorPtr auto rows = maxRowsToInsert(rowOffset, remainingRows); if (rows == 0) { auto minSizeRequired = fixedRowSize_ ? fixedRowSize_.value() : rowSizes_[rowOffset + 1] - rowSizes_[rowOffset]; - acquireNewBuffer(memLimit, minSizeRequired); + acquireNewBuffer((uint64_t)memLimit, minSizeRequired); rows = maxRowsToInsert(rowOffset, remainingRows); ARROW_RETURN_IF( rows == 0, arrow::Status::Invalid("Failed to insert rows. Remaining rows: " + std::to_string(remainingRows))); @@ -277,8 +277,8 @@ uint32_t VeloxSortShuffleWriter::maxRowsToInsert(uint32_t offset, uint32_t rows) return iter - beginIter; } -void VeloxSortShuffleWriter::acquireNewBuffer(int64_t memLimit, uint64_t minSizeRequired) { - auto size = std::max(std::min((uint64_t)memLimit >> 2, 64UL * 1024 * 1024), minSizeRequired); +void VeloxSortShuffleWriter::acquireNewBuffer(uint64_t memLimit, uint64_t minSizeRequired) { + auto size = std::max(std::min(memLimit >> 2, 64UL * 1024 * 1024), minSizeRequired); // Allocating new buffer can trigger spill. auto newBuffer = facebook::velox::AlignedBuffer::allocate(size, veloxPool_.get(), 0); pages_.emplace_back(std::move(newBuffer)); diff --git a/cpp/velox/shuffle/VeloxSortShuffleWriter.h b/cpp/velox/shuffle/VeloxSortShuffleWriter.h index 8f01997b11e14..6ac5308d0bd8c 100644 --- a/cpp/velox/shuffle/VeloxSortShuffleWriter.h +++ b/cpp/velox/shuffle/VeloxSortShuffleWriter.h @@ -76,7 +76,7 @@ class VeloxSortShuffleWriter final : public VeloxShuffleWriter { uint32_t maxRowsToInsert(uint32_t offset, uint32_t rows); - void acquireNewBuffer(int64_t memLimit, uint64_t minSizeRequired); + void acquireNewBuffer(uint64_t memLimit, uint64_t minSizeRequired); void growArrayIfNecessary(uint32_t rows); From a784e45662117b2192a166d24d463257df7311c4 Mon Sep 17 00:00:00 2001 From: lgbo Date: Mon, 29 Jul 2024 10:00:01 +0800 Subject: [PATCH 12/61] [GLUTEN-6544][CH] Support existence join (#6548) * support existence join * fixed tests --- .../gluten/vectorized/StorageJoinBuilder.java | 2 + ...oadcastNestedLoopJoinExecTransformer.scala | 16 +++++++- .../execution/CHHashJoinExecTransformer.scala | 37 ++++++++++++++++++- .../gluten/utils/CHJoinValidateUtil.scala | 4 ++ .../GlutenClickHouseTPCDSAbstractSuite.scala | 11 +++--- ...kHouseTPCDSParquetSortMergeJoinSuite.scala | 5 ++- .../execution/GlutenClickHouseTPCHSuite.scala | 31 ++++++++++++++++ .../benchmarks/CHHashBuildBenchmark.scala | 2 +- cpp-ch/local-engine/Common/CHUtil.cpp | 8 +++- cpp-ch/local-engine/Common/CHUtil.h | 2 +- .../Join/BroadCastJoinBuilder.cpp | 3 +- .../local-engine/Join/BroadCastJoinBuilder.h | 1 + cpp-ch/local-engine/Parser/JoinRelParser.cpp | 35 ++++++++++++++++-- cpp-ch/local-engine/Parser/JoinRelParser.h | 2 + cpp-ch/local-engine/local_engine_jni.cpp | 3 +- 15 files changed, 144 insertions(+), 18 deletions(-) diff --git a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/StorageJoinBuilder.java b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/StorageJoinBuilder.java index 27725998feebe..ae7b89120cd4d 100644 --- a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/StorageJoinBuilder.java +++ b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/StorageJoinBuilder.java @@ -45,6 +45,7 @@ private static native long nativeBuild( String joinKeys, int joinType, boolean hasMixedFiltCondition, + boolean isExistenceJoin, byte[] namedStruct); private StorageJoinBuilder() {} @@ -89,6 +90,7 @@ public static long build( joinKey, joinType, broadCastContext.hasMixedFiltCondition(), + broadCastContext.isExistenceJoin(), toNameStruct(output).toByteArray()); } diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHBroadcastNestedLoopJoinExecTransformer.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHBroadcastNestedLoopJoinExecTransformer.scala index d1dc76045338d..3aab5a6eb9986 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHBroadcastNestedLoopJoinExecTransformer.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHBroadcastNestedLoopJoinExecTransformer.scala @@ -23,6 +23,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.rpc.GlutenDriverEndpoint import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.optimizer.BuildSide +import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.{InnerLike, JoinType, LeftSemi} import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} import org.apache.spark.sql.execution.joins.BuildSideRelation @@ -44,6 +45,13 @@ case class CHBroadcastNestedLoopJoinExecTransformer( condition ) { + private val finalJoinType = joinType match { + case ExistenceJoin(_) => + LeftSemi + case _ => + joinType + } + override def columnarInputRDDs: Seq[RDD[ColumnarBatch]] = { val streamedRDD = getColumnarInputRDDs(streamedPlan) val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) @@ -57,7 +65,13 @@ case class CHBroadcastNestedLoopJoinExecTransformer( } val broadcast = buildPlan.executeBroadcast[BuildSideRelation]() val context = - BroadCastHashJoinContext(Seq.empty, joinType, false, buildPlan.output, buildBroadcastTableId) + BroadCastHashJoinContext( + Seq.empty, + finalJoinType, + false, + joinType.isInstanceOf[ExistenceJoin], + buildPlan.output, + buildBroadcastTableId) val broadcastRDD = CHBroadcastBuildSideRDD(sparkContext, broadcast, context) streamedRDD :+ broadcastRDD } diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashJoinExecTransformer.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashJoinExecTransformer.scala index 48870892d290e..c44156373528f 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashJoinExecTransformer.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashJoinExecTransformer.scala @@ -29,6 +29,8 @@ import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} import org.apache.spark.sql.execution.joins.BuildSideRelation import org.apache.spark.sql.vectorized.ColumnarBatch +import io.substrait.proto.JoinRel + case class CHShuffledHashJoinExecTransformer( leftKeys: Seq[Expression], rightKeys: Seq[Expression], @@ -82,6 +84,7 @@ case class BroadCastHashJoinContext( buildSideJoinKeys: Seq[Expression], joinType: JoinType, hasMixedFiltCondition: Boolean, + isExistenceJoin: Boolean, buildSideStructure: Seq[Attribute], buildHashTableId: String) @@ -112,7 +115,7 @@ case class CHBroadcastHashJoinExecTransformer( override protected def doValidateInternal(): ValidationResult = { val shouldFallback = CHJoinValidateUtil.shouldFallback( - BroadcastHashJoinStrategy(joinType), + BroadcastHashJoinStrategy(finalJoinType), left.outputSet, right.outputSet, condition) @@ -141,8 +144,9 @@ case class CHBroadcastHashJoinExecTransformer( val context = BroadCastHashJoinContext( buildKeyExprs, - joinType, + finalJoinType, isMixedCondition(condition), + joinType.isInstanceOf[ExistenceJoin], buildPlan.output, buildHashTableId) val broadcastRDD = CHBroadcastBuildSideRDD(sparkContext, broadcast, context) @@ -161,4 +165,33 @@ case class CHBroadcastHashJoinExecTransformer( } res } + + // ExistenceJoin is introduced in #SPARK-14781. It returns all rows from the left table with + // a new column to indecate whether the row is matched in the right table. + // Indeed, the ExistenceJoin is transformed into left any join in CH. + // We don't have left any join in substrait, so use left semi join instead. + // and isExistenceJoin is set to true to indicate that it is an existence join. + private val finalJoinType = joinType match { + case ExistenceJoin(_) => + LeftSemi + case _ => + joinType + } + override protected lazy val substraitJoinType: JoinRel.JoinType = { + joinType match { + case _: InnerLike => + JoinRel.JoinType.JOIN_TYPE_INNER + case FullOuter => + JoinRel.JoinType.JOIN_TYPE_OUTER + case LeftOuter | RightOuter => + JoinRel.JoinType.JOIN_TYPE_LEFT + case LeftSemi | ExistenceJoin(_) => + JoinRel.JoinType.JOIN_TYPE_LEFT_SEMI + case LeftAnti => + JoinRel.JoinType.JOIN_TYPE_ANTI + case _ => + // TODO: Support cross join with Cross Rel + JoinRel.JoinType.UNRECOGNIZED + } + } } diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/utils/CHJoinValidateUtil.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/utils/CHJoinValidateUtil.scala index dae8e6e073a18..08b5ef5b2ef0d 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/utils/CHJoinValidateUtil.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/utils/CHJoinValidateUtil.scala @@ -55,6 +55,7 @@ object CHJoinValidateUtil extends Logging { var shouldFallback = false val joinType = joinStrategy.joinType if (joinType.toString.contains("ExistenceJoin")) { + logError("Fallback for join type ExistenceJoin") return true } if (joinType.sql.contains("INNER")) { @@ -78,6 +79,9 @@ object CHJoinValidateUtil extends Logging { case _ => false } } + if (shouldFallback) { + logError(s"Fallback for join type $joinType") + } shouldFallback } } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSAbstractSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSAbstractSuite.scala index f0712bf5af106..9787182ed93f5 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSAbstractSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSAbstractSuite.scala @@ -58,13 +58,14 @@ abstract class GlutenClickHouseTPCDSAbstractSuite Seq("q" + "%d".format(queryNum)) } val noFallBack = queryNum match { - case i if i == 10 || i == 16 || i == 35 || i == 45 || i == 94 => - // Q10 BroadcastHashJoin, ExistenceJoin - // Q16 ShuffledHashJoin, NOT condition - // Q35 BroadcastHashJoin, ExistenceJoin - // Q45 BroadcastHashJoin, ExistenceJoin + case i if !isAqe && (i == 10 || i == 16 || i == 35 || i == 94) => + // q10 smj + existence join + // q16 smj + left semi + not condition + // q35 smj + existence join // Q94 BroadcastHashJoin, LeftSemi, NOT condition (false, false) + case i if isAqe && (i == 16 || i == 94) => + (false, false) case other => (true, false) } sqlNums.map((_, noFallBack._1, noFallBack._2)) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSParquetSortMergeJoinSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSParquetSortMergeJoinSuite.scala index 3f7816cb84ad0..3ec4e31a41096 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSParquetSortMergeJoinSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSParquetSortMergeJoinSuite.scala @@ -23,12 +23,15 @@ import org.apache.spark.SparkConf class GlutenClickHouseTPCDSParquetSortMergeJoinSuite extends GlutenClickHouseTPCDSAbstractSuite { override protected def excludedTpcdsQueries: Set[String] = Set( - // fallback due to left semi/anti + // fallback due to left semi/anti/existence join "q8", + "q10", "q14a", "q14b", + "116", "q23a", "q23b", + "q35", "q38", "q51", "q69", diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala index d26891ddb1eaa..1c09449c817fb 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala @@ -500,5 +500,36 @@ class GlutenClickHouseTPCHSuite extends GlutenClickHouseTPCHAbstractSuite { compareResultsAgainstVanillaSpark(sql2, true, { _ => }) } + + test("existence join") { + spark.sql("create table t1(a int, b int) using parquet") + spark.sql("create table t2(a int, b int) using parquet") + spark.sql("insert into t1 values(0, 0), (1, 2), (2, 3), (3, 4), (null, 5), (6, null)") + spark.sql("insert into t2 values(0, 0), (1, 2), (2, 3), (2,4), (null, 5), (6, null)") + + val sql1 = """ + |select * from t1 where exists (select 1 from t2 where t1.a = t2.a) or t1.a > 1 + |""".stripMargin + compareResultsAgainstVanillaSpark(sql1, true, { _ => }) + + val sql2 = """ + |select * from t1 where exists (select 1 from t2 where t1.a = t2.a) or t1.a > 3 + |""".stripMargin + compareResultsAgainstVanillaSpark(sql2, true, { _ => }) + + val sql3 = """ + |select * from t1 where exists (select 1 from t2 where t1.a = t2.a) or t1.b > 0 + |""".stripMargin + compareResultsAgainstVanillaSpark(sql3, true, { _ => }) + + val sql4 = """ + |select * from t1 where exists (select 1 from t2 + |where t1.a = t2.a and t1.b = t2.b) or t1.a > 0 + |""".stripMargin + compareResultsAgainstVanillaSpark(sql4, true, { _ => }) + + spark.sql("drop table t1") + spark.sql("drop table t2") + } } // scalastyle:off line.size.limit diff --git a/backends-clickhouse/src/test/scala/org/apache/spark/sql/execution/benchmarks/CHHashBuildBenchmark.scala b/backends-clickhouse/src/test/scala/org/apache/spark/sql/execution/benchmarks/CHHashBuildBenchmark.scala index 8d4bee5546253..141bf5eea5cb9 100644 --- a/backends-clickhouse/src/test/scala/org/apache/spark/sql/execution/benchmarks/CHHashBuildBenchmark.scala +++ b/backends-clickhouse/src/test/scala/org/apache/spark/sql/execution/benchmarks/CHHashBuildBenchmark.scala @@ -104,7 +104,7 @@ object CHHashBuildBenchmark extends SqlBasedBenchmark with CHSqlBasedBenchmark w ( countsAndBytes.flatMap(_._2), countsAndBytes.map(_._1).sum, - BroadCastHashJoinContext(Seq(child.output.head), Inner, false, child.output, "") + BroadCastHashJoinContext(Seq(child.output.head), Inner, false, false, child.output, "") ) } } diff --git a/cpp-ch/local-engine/Common/CHUtil.cpp b/cpp-ch/local-engine/Common/CHUtil.cpp index 787277dbefb18..3a699b50e302e 100644 --- a/cpp-ch/local-engine/Common/CHUtil.cpp +++ b/cpp-ch/local-engine/Common/CHUtil.cpp @@ -1089,14 +1089,18 @@ void JoinUtil::reorderJoinOutput(DB::QueryPlan & plan, DB::Names cols) plan.addStep(std::move(project_step)); } -std::pair JoinUtil::getJoinKindAndStrictness(substrait::JoinRel_JoinType join_type) +std::pair +JoinUtil::getJoinKindAndStrictness(substrait::JoinRel_JoinType join_type, bool is_existence_join) { switch (join_type) { case substrait::JoinRel_JoinType_JOIN_TYPE_INNER: return {DB::JoinKind::Inner, DB::JoinStrictness::All}; - case substrait::JoinRel_JoinType_JOIN_TYPE_LEFT_SEMI: + case substrait::JoinRel_JoinType_JOIN_TYPE_LEFT_SEMI: { + if (is_existence_join) + return {DB::JoinKind::Left, DB::JoinStrictness::Any}; return {DB::JoinKind::Left, DB::JoinStrictness::Semi}; + } case substrait::JoinRel_JoinType_JOIN_TYPE_ANTI: return {DB::JoinKind::Left, DB::JoinStrictness::Anti}; case substrait::JoinRel_JoinType_JOIN_TYPE_LEFT: diff --git a/cpp-ch/local-engine/Common/CHUtil.h b/cpp-ch/local-engine/Common/CHUtil.h index 98139fb49a5b3..b45c6ab3c4d21 100644 --- a/cpp-ch/local-engine/Common/CHUtil.h +++ b/cpp-ch/local-engine/Common/CHUtil.h @@ -313,7 +313,7 @@ class JoinUtil { public: static void reorderJoinOutput(DB::QueryPlan & plan, DB::Names cols); - static std::pair getJoinKindAndStrictness(substrait::JoinRel_JoinType join_type); + static std::pair getJoinKindAndStrictness(substrait::JoinRel_JoinType join_type, bool is_existence_join); static std::pair getCrossJoinKindAndStrictness(substrait::CrossRel_JoinType join_type); }; diff --git a/cpp-ch/local-engine/Join/BroadCastJoinBuilder.cpp b/cpp-ch/local-engine/Join/BroadCastJoinBuilder.cpp index 4d5eae6dc0b57..c21cc8ba35246 100644 --- a/cpp-ch/local-engine/Join/BroadCastJoinBuilder.cpp +++ b/cpp-ch/local-engine/Join/BroadCastJoinBuilder.cpp @@ -99,6 +99,7 @@ std::shared_ptr buildJoin( const std::string & join_keys, jint join_type, bool has_mixed_join_condition, + bool is_existence_join, const std::string & named_struct) { auto join_key_list = Poco::StringTokenizer(join_keys, ","); @@ -112,7 +113,7 @@ std::shared_ptr buildJoin( if (key.starts_with("BuiltBNLJBroadcastTable-")) std::tie(kind, strictness) = JoinUtil::getCrossJoinKindAndStrictness(static_cast(join_type)); else - std::tie(kind, strictness) = JoinUtil::getJoinKindAndStrictness(static_cast(join_type)); + std::tie(kind, strictness) = JoinUtil::getJoinKindAndStrictness(static_cast(join_type), is_existence_join); substrait::NamedStruct substrait_struct; diff --git a/cpp-ch/local-engine/Join/BroadCastJoinBuilder.h b/cpp-ch/local-engine/Join/BroadCastJoinBuilder.h index 3d2e67f9df101..a97bd77a84d09 100644 --- a/cpp-ch/local-engine/Join/BroadCastJoinBuilder.h +++ b/cpp-ch/local-engine/Join/BroadCastJoinBuilder.h @@ -37,6 +37,7 @@ std::shared_ptr buildJoin( const std::string & join_keys, jint join_type, bool has_mixed_join_condition, + bool is_existence_join, const std::string & named_struct); void cleanBuildHashTable(const std::string & hash_table_id, jlong instance); std::shared_ptr getJoin(const std::string & hash_table_id); diff --git a/cpp-ch/local-engine/Parser/JoinRelParser.cpp b/cpp-ch/local-engine/Parser/JoinRelParser.cpp index 460311e289b67..24ba7acdb6545 100644 --- a/cpp-ch/local-engine/Parser/JoinRelParser.cpp +++ b/cpp-ch/local-engine/Parser/JoinRelParser.cpp @@ -33,6 +33,7 @@ #include #include #include +#include #include @@ -50,13 +51,13 @@ using namespace DB; namespace local_engine { -std::shared_ptr createDefaultTableJoin(substrait::JoinRel_JoinType join_type) +std::shared_ptr createDefaultTableJoin(substrait::JoinRel_JoinType join_type, bool is_existence_join) { auto & global_context = SerializedPlanParser::global_context; auto table_join = std::make_shared( global_context->getSettings(), global_context->getGlobalTemporaryVolume(), global_context->getTempDataOnDisk()); - std::pair kind_and_strictness = JoinUtil::getJoinKindAndStrictness(join_type); + std::pair kind_and_strictness = JoinUtil::getJoinKindAndStrictness(join_type, is_existence_join); table_join->setKind(kind_and_strictness.first); table_join->setStrictness(kind_and_strictness.second); return table_join; @@ -218,7 +219,7 @@ DB::QueryPlanPtr JoinRelParser::parseJoin(const substrait::JoinRel & join, DB::Q renamePlanColumns(*left, *right, *storage_join); } - auto table_join = createDefaultTableJoin(join.type()); + auto table_join = createDefaultTableJoin(join.type(), join_opt_info.is_existence_join); DB::Block right_header_before_convert_step = right->getCurrentDataStream().header; addConvertStep(*table_join, *left, *right); @@ -350,11 +351,39 @@ DB::QueryPlanPtr JoinRelParser::parseJoin(const substrait::JoinRel & join, DB::Q query_plan = std::make_unique(); query_plan->unitePlans(std::move(join_step), {std::move(plans)}); } + JoinUtil::reorderJoinOutput(*query_plan, after_join_names); + /// Need to project the right table column into boolean type + if (join_opt_info.is_existence_join) + { + existenceJoinPostProject(*query_plan, left_names); + } return query_plan; } + +/// We use left any join to implement ExistenceJoin. +/// The result columns of ExistenceJoin are left table columns + one flag column. +/// The flag column indicates whether a left row is matched or not. We build the flag column here. +/// The input plan's header is left table columns + right table columns. If one row in the right row is null, +/// we mark the flag 0, otherwise mark it 1. +void JoinRelParser::existenceJoinPostProject(DB::QueryPlan & plan, const DB::Names & left_input_cols) +{ + auto actions_dag = std::make_shared(plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); + const auto * right_col_node = actions_dag->getInputs().back(); + auto function_builder = DB::FunctionFactory::instance().get("isNotNull", getContext()); + const auto * not_null_node = &actions_dag->addFunction(function_builder, {right_col_node}, right_col_node->result_name); + actions_dag->addOrReplaceInOutputs(*not_null_node); + DB::Names required_cols = left_input_cols; + required_cols.emplace_back(not_null_node->result_name); + actions_dag->removeUnusedActions(required_cols); + auto project_step = std::make_unique(plan.getCurrentDataStream(), actions_dag); + project_step->setStepDescription("ExistenceJoin Post Project"); + steps.emplace_back(project_step.get()); + plan.addStep(std::move(project_step)); +} + void JoinRelParser::addConvertStep(TableJoin & table_join, DB::QueryPlan & left, DB::QueryPlan & right) { /// If the columns name in right table is duplicated with left table, we need to rename the right table's columns. diff --git a/cpp-ch/local-engine/Parser/JoinRelParser.h b/cpp-ch/local-engine/Parser/JoinRelParser.h index e6d31e6d31d6e..ee1155cb47128 100644 --- a/cpp-ch/local-engine/Parser/JoinRelParser.h +++ b/cpp-ch/local-engine/Parser/JoinRelParser.h @@ -66,6 +66,8 @@ class JoinRelParser : public RelParser void addPostFilter(DB::QueryPlan & plan, const substrait::JoinRel & join); + void existenceJoinPostProject(DB::QueryPlan & plan, const DB::Names & left_input_cols); + static std::unordered_set extractTableSidesFromExpression( const substrait::Expression & expr, const DB::Block & left_header, const DB::Block & right_header); }; diff --git a/cpp-ch/local-engine/local_engine_jni.cpp b/cpp-ch/local-engine/local_engine_jni.cpp index 17d087bb82ff4..a6ca55052ef9d 100644 --- a/cpp-ch/local-engine/local_engine_jni.cpp +++ b/cpp-ch/local-engine/local_engine_jni.cpp @@ -1094,6 +1094,7 @@ JNIEXPORT jlong Java_org_apache_gluten_vectorized_StorageJoinBuilder_nativeBuild jstring join_key_, jint join_type_, jboolean has_mixed_join_condition, + jboolean is_existence_join, jbyteArray named_struct) { LOCAL_ENGINE_JNI_METHOD_START @@ -1107,7 +1108,7 @@ JNIEXPORT jlong Java_org_apache_gluten_vectorized_StorageJoinBuilder_nativeBuild DB::CompressedReadBuffer input(read_buffer_from_java_array); local_engine::configureCompressedReadBuffer(input); const auto * obj = make_wrapper(local_engine::BroadCastJoinBuilder::buildJoin( - hash_table_id, input, row_count_, join_key, join_type_, has_mixed_join_condition, struct_string)); + hash_table_id, input, row_count_, join_key, join_type_, has_mixed_join_condition, is_existence_join, struct_string)); return obj->instance(); LOCAL_ENGINE_JNI_METHOD_END(env, 0) } From 03765af4a3de690ce82ba0408f60821ff404961f Mon Sep 17 00:00:00 2001 From: Zhen Wang <643348094@qq.com> Date: Mon, 29 Jul 2024 11:04:35 +0800 Subject: [PATCH 13/61] [VL] Improve package scripts (#6569) --- dev/build-thirdparty.sh | 85 ++++++++++++++++++++++++++++++++++++++ dev/package-vcpkg.sh | 12 +++++- dev/package.sh | 91 ++++++----------------------------------- 3 files changed, 107 insertions(+), 81 deletions(-) create mode 100755 dev/build-thirdparty.sh diff --git a/dev/build-thirdparty.sh b/dev/build-thirdparty.sh new file mode 100755 index 0000000000000..90fc5e58fbd54 --- /dev/null +++ b/dev/build-thirdparty.sh @@ -0,0 +1,85 @@ +#!/bin/bash + +set -eux + +CURRENT_DIR=$(cd "$(dirname "$BASH_SOURCE")"; pwd) +GLUTEN_DIR="$CURRENT_DIR/.." +THIRDPARTY_LIB="$GLUTEN_DIR/package/target/thirdparty-lib" +LINUX_OS=$(. /etc/os-release && echo ${ID}) +VERSION=$(. /etc/os-release && echo ${VERSION_ID}) +ARCH=`uname -m` + +mkdir -p $THIRDPARTY_LIB +function process_setup_ubuntu_2004 { + cp /usr/lib/${ARCH}-linux-gnu/{libroken.so.18,libasn1.so.8,libcrypto.so.1.1,libnghttp2.so.14,libnettle.so.7,libhogweed.so.5,librtmp.so.1,libssh.so.4,libssl.so.1.1,liblber-2.4.so.2,libsasl2.so.2,libwind.so.0,libheimbase.so.1,libhcrypto.so.4,libhx509.so.5,libkrb5.so.26,libheimntlm.so.0,libgssapi.so.3,libldap_r-2.4.so.2,libcurl.so.4,libdouble-conversion.so.3,libevent-2.1.so.7,libgflags.so.2.2,libunwind.so.8,libglog.so.0,libidn.so.11,libntlm.so.0,libgsasl.so.7,libicudata.so.66,libicuuc.so.66,libxml2.so.2,libre2.so.5,libsnappy.so.1,libpsl.so.5,libbrotlidec.so.1,libbrotlicommon.so.1,libthrift-0.13.0.so} $THIRDPARTY_LIB/ + cp /usr/local/lib/{libprotobuf.so.32,libhdfs3.so.1,libboost_context.so.1.84.0,libboost_regex.so.1.84.0} $THIRDPARTY_LIB/ +} + +function process_setup_ubuntu_2204 { + cp /usr/lib/${ARCH}-linux-gnu/{libre2.so.9,libdouble-conversion.so.3,libidn.so.12,libglog.so.0,libgflags.so.2.2,libevent-2.1.so.7,libsnappy.so.1,libunwind.so.8,libcurl.so.4,libxml2.so.2,libgsasl.so.7,libicui18n.so.70,libicuuc.so.70,libnghttp2.so.14,libldap-2.5.so.0,liblber-2.5.so.0,libntlm.so.0,librtmp.so.1,libsasl2.so.2,libssh.so.4,libicudata.so.70,libthrift-0.16.0.so} $THIRDPARTY_LIB/ + cp /usr/local/lib/{libhdfs3.so.1,libprotobuf.so.32,libboost_context.so.1.84.0,libboost_regex.so.1.84.0} $THIRDPARTY_LIB/ +} + +function process_setup_centos_9 { + cp /lib64/{libre2.so.9,libdouble-conversion.so.3,libevent-2.1.so.7,libdwarf.so.0,libgsasl.so.7,libicudata.so.67,libicui18n.so.67,libicuuc.so.67,libidn.so.12,libntlm.so.0,libsodium.so.23} $THIRDPARTY_LIB/ + cp /usr/local/lib/{libhdfs3.so.1,libboost_context.so.1.84.0,libboost_filesystem.so.1.84.0,libboost_program_options.so.1.84.0,libboost_regex.so.1.84.0,libboost_system.so.1.84.0,libboost_thread.so.1.84.0,libboost_atomic.so.1.84.0,libprotobuf.so.32} $THIRDPARTY_LIB/ + cp /usr/local/lib64/{libgflags.so.2.2,libglog.so.1} $THIRDPARTY_LIB/ +} + +function process_setup_centos_8 { + cp /usr/lib64/{libre2.so.0,libdouble-conversion.so.3,libevent-2.1.so.6,libdwarf.so.1,libgsasl.so.7,libicudata.so.60,libicui18n.so.60,libicuuc.so.60,libidn.so.11,libntlm.so.0,libsodium.so.23} $THIRDPARTY_LIB/ + cp /usr/local/lib/{libhdfs3.so.1,libboost_context.so.1.84.0,libboost_filesystem.so.1.84.0,libboost_program_options.so.1.84.0,libboost_regex.so.1.84.0,libboost_system.so.1.84.0,libboost_thread.so.1.84.0,libboost_atomic.so.1.84.0,libprotobuf.so.32} $THIRDPARTY_LIB/ + cp /usr/local/lib64/{libgflags.so.2.2,libglog.so.1} $THIRDPARTY_LIB/ +} + +function process_setup_centos_7 { + cp /usr/local/lib64/{libgflags.so.2.2,libglog.so.0} $THIRDPARTY_LIB/ + cp /usr/lib64/{libdouble-conversion.so.1,libevent-2.0.so.5,libzstd.so.1,libntlm.so.0,libgsasl.so.7,liblz4.so.1} $THIRDPARTY_LIB/ + cp /usr/local/lib/{libre2.so.10,libhdfs3.so.1,libboost_context.so.1.84.0,libboost_filesystem.so.1.84.0,libboost_program_options.so.1.84.0,libboost_system.so.1.84.0,libboost_thread.so.1.84.0,libboost_regex.so.1.84.0,libboost_atomic.so.1.84.0,libprotobuf.so.32} $THIRDPARTY_LIB/ +} + +function process_setup_debian_11 { + cp /usr/lib/x86_64-linux-gnu/{libre2.so.9,libthrift-0.13.0.so,libdouble-conversion.so.3,libevent-2.1.so.7,libgflags.so.2.2,libglog.so.0,libsnappy.so.1,libunwind.so.8,libcurl.so.4,libicui18n.so.67,libicuuc.so.67,libnghttp2.so.14,librtmp.so.1,libssh2.so.1,libpsl.so.5,libldap_r-2.4.so.2,liblber-2.4.so.2,libbrotlidec.so.1,libicudata.so.67,libsasl2.so.2,libbrotlicommon.so.1} $THIRDPARTY_LIB/ + cp /usr/local/lib/{libhdfs3.so.1,libprotobuf.so.32,libboost_context.so.1.84.0,libboost_regex.so.1.84.0} $THIRDPARTY_LIB/ +} + +function process_setup_debian_12 { + cp /usr/lib/x86_64-linux-gnu/{libthrift-0.17.0.so,libdouble-conversion.so.3,libevent-2.1.so.7,libgflags.so.2.2,libglog.so.1,libsnappy.so.1,libunwind.so.8,libcurl.so.4,libicui18n.so.72,libicuuc.so.72,libnghttp2.so.14,librtmp.so.1,libssh2.so.1,libpsl.so.5,libldap-2.5.so.0,liblber-2.5.so.0,libbrotlidec.so.1,libicudata.so.72,libsasl2.so.2,libbrotlicommon.so.1,libcrypto.so.3,libssl.so.3,libgssapi_krb5.so.2,libkrb5.so.3,libk5crypto.so.3,libkrb5support.so.0,libkeyutils.so.1} $THIRDPARTY_LIB/ + cp /usr/local/lib/{libprotobuf.so.32,libhdfs3.so.1,libboost_context.so.1.84.0,libboost_regex.so.1.84.0} $THIRDPARTY_LIB/ +} + +if [[ "$LINUX_OS" == "ubuntu" || "$LINUX_OS" == "pop" ]]; then + if [ "$VERSION" == "20.04" ]; then + process_setup_ubuntu_2004 + elif [ "$VERSION" == "22.04" ]; then + process_setup_ubuntu_2204 + fi +elif [ "$LINUX_OS" == "centos" ]; then + if [ "$VERSION" == "9" ]; then + process_setup_centos_9 + elif [ "$VERSION" == "8" ]; then + process_setup_centos_8 + elif [ "$VERSION" == "7" ]; then + process_setup_centos_7 + fi +elif [ "$LINUX_OS" == "alinux" ]; then + if [ "${VERSION:0:1}" == "3" ]; then + process_setup_centos_8 + elif [ "${VERSION:0:1}" == "2" ]; then + process_setup_centos_7 + fi +elif [ "$LINUX_OS" == "tencentos" ]; then + if [ "$VERSION" == "2.4" ]; then + process_setup_centos_7 + elif [ "$VERSION" == "3.2" ]; then + process_setup_centos_8 + fi +elif [ "$LINUX_OS" == "debian" ]; then + if [ "$VERSION" == "11" ]; then + process_setup_debian_11 + elif [ "$VERSION" == "12" ]; then + process_setup_debian_12 + fi +fi +cd $THIRDPARTY_LIB/ +jar cvf gluten-thirdparty-lib-$LINUX_OS-$VERSION-$ARCH.jar ./ diff --git a/dev/package-vcpkg.sh b/dev/package-vcpkg.sh index 2cb4637816fb8..a153210748b8b 100755 --- a/dev/package-vcpkg.sh +++ b/dev/package-vcpkg.sh @@ -1,9 +1,12 @@ #!/bin/bash -set -ex +set -eux CURRENT_DIR=$(cd "$(dirname "$BASH_SOURCE")"; pwd) GLUTEN_DIR="$CURRENT_DIR/.." +LINUX_OS=$(. /etc/os-release && echo ${ID}) +VERSION=$(. /etc/os-release && echo ${VERSION_ID}) +ARCH=`uname -m` cd "$GLUTEN_DIR" if [ "$LINUX_OS" == "centos" ]; then @@ -13,5 +16,10 @@ if [ "$LINUX_OS" == "centos" ]; then source /opt/rh/devtoolset-9/enable fi fi + +# prepare vcpkg environments source ./dev/vcpkg/env.sh -./dev/buildbundle-veloxbe.sh --build_tests=ON --build_benchmarks=ON --enable_s3=ON --enable_hdfs=ON + +# build gluten with velox backend, prompt always respond y +export PROMPT_ALWAYS_RESPOND=y +./dev/buildbundle-veloxbe.sh --build_tests=ON --build_benchmarks=ON --enable_s3=ON --enable_hdfs=ON "$@" diff --git a/dev/package.sh b/dev/package.sh index db9125c2f473f..6da750b44476d 100755 --- a/dev/package.sh +++ b/dev/package.sh @@ -4,89 +4,22 @@ set -eux CURRENT_DIR=$(cd "$(dirname "$BASH_SOURCE")"; pwd) GLUTEN_DIR="$CURRENT_DIR/.." -THIRDPARTY_LIB="$GLUTEN_DIR/package/target/thirdparty-lib" LINUX_OS=$(. /etc/os-release && echo ${ID}) VERSION=$(. /etc/os-release && echo ${VERSION_ID}) ARCH=`uname -m` -# compile gluten jar -$GLUTEN_DIR/dev/builddeps-veloxbe.sh --build_tests=ON --build_benchmarks=ON --enable_s3=ON --enable_hdfs=ON -mvn clean package -Pbackends-velox -Pceleborn -Puniffle -Pspark-3.2 -DskipTests -mvn clean package -Pbackends-velox -Pceleborn -Puniffle -Pspark-3.3 -DskipTests -mvn clean package -Pbackends-velox -Pceleborn -Puniffle -Pspark-3.4 -DskipTests -mvn clean package -Pbackends-velox -Pceleborn -Puniffle -Pspark-3.5 -DskipTests - -mkdir -p $THIRDPARTY_LIB -function process_setup_ubuntu_2004 { - cp /usr/lib/${ARCH}-linux-gnu/{libroken.so.18,libasn1.so.8,libcrypto.so.1.1,libnghttp2.so.14,libnettle.so.7,libhogweed.so.5,librtmp.so.1,libssh.so.4,libssl.so.1.1,liblber-2.4.so.2,libsasl2.so.2,libwind.so.0,libheimbase.so.1,libhcrypto.so.4,libhx509.so.5,libkrb5.so.26,libheimntlm.so.0,libgssapi.so.3,libldap_r-2.4.so.2,libcurl.so.4,libdouble-conversion.so.3,libevent-2.1.so.7,libgflags.so.2.2,libunwind.so.8,libglog.so.0,libidn.so.11,libntlm.so.0,libgsasl.so.7,libicudata.so.66,libicuuc.so.66,libxml2.so.2,libre2.so.5,libsnappy.so.1,libpsl.so.5,libbrotlidec.so.1,libbrotlicommon.so.1,libthrift-0.13.0.so} $THIRDPARTY_LIB/ - cp /usr/local/lib/{libprotobuf.so.32,libhdfs3.so.1,libboost_context.so.1.84.0,libboost_regex.so.1.84.0} $THIRDPARTY_LIB/ -} - -function process_setup_ubuntu_2204 { - cp /usr/lib/${ARCH}-linux-gnu/{libre2.so.9,libdouble-conversion.so.3,libidn.so.12,libglog.so.0,libgflags.so.2.2,libevent-2.1.so.7,libsnappy.so.1,libunwind.so.8,libcurl.so.4,libxml2.so.2,libgsasl.so.7,libicui18n.so.70,libicuuc.so.70,libnghttp2.so.14,libldap-2.5.so.0,liblber-2.5.so.0,libntlm.so.0,librtmp.so.1,libsasl2.so.2,libssh.so.4,libicudata.so.70,libthrift-0.16.0.so} $THIRDPARTY_LIB/ - cp /usr/local/lib/{libhdfs3.so.1,libprotobuf.so.32,libboost_context.so.1.84.0,libboost_regex.so.1.84.0} $THIRDPARTY_LIB/ -} - -function process_setup_centos_9 { - cp /lib64/{libre2.so.9,libdouble-conversion.so.3,libevent-2.1.so.7,libdwarf.so.0,libgsasl.so.7,libicudata.so.67,libicui18n.so.67,libicuuc.so.67,libidn.so.12,libntlm.so.0,libsodium.so.23} $THIRDPARTY_LIB/ - cp /usr/local/lib/{libhdfs3.so.1,libboost_context.so.1.84.0,libboost_filesystem.so.1.84.0,libboost_program_options.so.1.84.0,libboost_regex.so.1.84.0,libboost_system.so.1.84.0,libboost_thread.so.1.84.0,libboost_atomic.so.1.84.0,libprotobuf.so.32} $THIRDPARTY_LIB/ - cp /usr/local/lib64/{libgflags.so.2.2,libglog.so.1} $THIRDPARTY_LIB/ -} - -function process_setup_centos_8 { - cp /usr/lib64/{libre2.so.0,libdouble-conversion.so.3,libevent-2.1.so.6,libdwarf.so.1,libgsasl.so.7,libicudata.so.60,libicui18n.so.60,libicuuc.so.60,libidn.so.11,libntlm.so.0,libsodium.so.23} $THIRDPARTY_LIB/ - cp /usr/local/lib/{libhdfs3.so.1,libboost_context.so.1.84.0,libboost_filesystem.so.1.84.0,libboost_program_options.so.1.84.0,libboost_regex.so.1.84.0,libboost_system.so.1.84.0,libboost_thread.so.1.84.0,libboost_atomic.so.1.84.0,libprotobuf.so.32} $THIRDPARTY_LIB/ - cp /usr/local/lib64/{libgflags.so.2.2,libglog.so.1} $THIRDPARTY_LIB/ -} - -function process_setup_centos_7 { - cp /usr/local/lib64/{libgflags.so.2.2,libglog.so.0} $THIRDPARTY_LIB/ - cp /usr/lib64/{libdouble-conversion.so.1,libevent-2.0.so.5,libzstd.so.1,libntlm.so.0,libgsasl.so.7,liblz4.so.1} $THIRDPARTY_LIB/ - cp /usr/local/lib/{libre2.so.10,libhdfs3.so.1,libboost_context.so.1.84.0,libboost_filesystem.so.1.84.0,libboost_program_options.so.1.84.0,libboost_system.so.1.84.0,libboost_thread.so.1.84.0,libboost_regex.so.1.84.0,libboost_atomic.so.1.84.0,libprotobuf.so.32} $THIRDPARTY_LIB/ -} - -function process_setup_debian_11 { - cp /usr/lib/x86_64-linux-gnu/{libre2.so.9,libthrift-0.13.0.so,libdouble-conversion.so.3,libevent-2.1.so.7,libgflags.so.2.2,libglog.so.0,libsnappy.so.1,libunwind.so.8,libcurl.so.4,libicui18n.so.67,libicuuc.so.67,libnghttp2.so.14,librtmp.so.1,libssh2.so.1,libpsl.so.5,libldap_r-2.4.so.2,liblber-2.4.so.2,libbrotlidec.so.1,libicudata.so.67,libsasl2.so.2,libbrotlicommon.so.1} $THIRDPARTY_LIB/ - cp /usr/local/lib/{libhdfs3.so.1,libprotobuf.so.32,libboost_context.so.1.84.0,libboost_regex.so.1.84.0} $THIRDPARTY_LIB/ -} - -function process_setup_debian_12 { - cp /usr/lib/x86_64-linux-gnu/{libthrift-0.17.0.so,libdouble-conversion.so.3,libevent-2.1.so.7,libgflags.so.2.2,libglog.so.1,libsnappy.so.1,libunwind.so.8,libcurl.so.4,libicui18n.so.72,libicuuc.so.72,libnghttp2.so.14,librtmp.so.1,libssh2.so.1,libpsl.so.5,libldap-2.5.so.0,liblber-2.5.so.0,libbrotlidec.so.1,libicudata.so.72,libsasl2.so.2,libbrotlicommon.so.1,libcrypto.so.3,libssl.so.3,libgssapi_krb5.so.2,libkrb5.so.3,libk5crypto.so.3,libkrb5support.so.0,libkeyutils.so.1} $THIRDPARTY_LIB/ - cp /usr/local/lib/{libprotobuf.so.32,libhdfs3.so.1,libboost_context.so.1.84.0,libboost_regex.so.1.84.0} $THIRDPARTY_LIB/ -} - -if [[ "$LINUX_OS" == "ubuntu" || "$LINUX_OS" == "pop" ]]; then - if [ "$VERSION" == "20.04" ]; then - process_setup_ubuntu_2004 - elif [ "$VERSION" == "22.04" ]; then - process_setup_ubuntu_2204 - fi -elif [ "$LINUX_OS" == "centos" ]; then - if [ "$VERSION" == "9" ]; then - process_setup_centos_9 - elif [ "$VERSION" == "8" ]; then - process_setup_centos_8 +cd "$GLUTEN_DIR" +if [ "$LINUX_OS" == "centos" ]; then + if [ "$VERSION" == "8" ]; then + source /opt/rh/gcc-toolset-9/enable elif [ "$VERSION" == "7" ]; then - process_setup_centos_7 - fi -elif [ "$LINUX_OS" == "alinux" ]; then - if [ "${VERSION:0:1}" == "3" ]; then - process_setup_centos_8 - elif [ "${VERSION:0:1}" == "2" ]; then - process_setup_centos_7 - fi -elif [ "$LINUX_OS" == "tencentos" ]; then - if [ "$VERSION" == "2.4" ]; then - process_setup_centos_7 - elif [ "$VERSION" == "3.2" ]; then - process_setup_centos_8 - fi -elif [ "$LINUX_OS" == "debian" ]; then - if [ "$VERSION" == "11" ]; then - process_setup_debian_11 - elif [ "$VERSION" == "12" ]; then - process_setup_debian_12 + source /opt/rh/devtoolset-9/enable fi fi -cd $THIRDPARTY_LIB/ -jar cvf gluten-thirdparty-lib-$LINUX_OS-$VERSION-$ARCH.jar ./ + +# build gluten with velox backend, prompt always respond y +export PROMPT_ALWAYS_RESPOND=y +./dev/buildbundle-veloxbe.sh --build_tests=ON --build_benchmarks=ON --enable_s3=ON --enable_hdfs=ON "$@" + +# make thirdparty package +./dev/build-thirdparty.sh From cba12050f4b89fc9a4d09e7cf75486d74fd94465 Mon Sep 17 00:00:00 2001 From: Zhen Li <10524738+zhli1142015@users.noreply.github.com> Date: Mon, 29 Jul 2024 14:23:23 +0800 Subject: [PATCH 14/61] [VL] Enable timestamp and binary type for HLL agg function (#6619) [VL] Enable timestamp and binary type for HLL agg function. --- .../apache/gluten/extension/HLLRewriteRule.scala | 2 +- .../execution/VeloxAggregateFunctionsSuite.scala | 14 ++++++++++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/backends-velox/src/main/scala/org/apache/gluten/extension/HLLRewriteRule.scala b/backends-velox/src/main/scala/org/apache/gluten/extension/HLLRewriteRule.scala index 7bae64ff8d592..2b17cbfd65aea 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/extension/HLLRewriteRule.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/extension/HLLRewriteRule.scala @@ -58,7 +58,7 @@ case class HLLRewriteRule(spark: SparkSession) extends Rule[LogicalPlan] { // be parsed by velox, it would cause the error: 'Unexpected type of HLL'. dataType match { case BooleanType | ByteType | ShortType | IntegerType | LongType | FloatType | DoubleType | - StringType | _: CharType | _: DecimalType | DateType => + StringType | _: CharType | _: DecimalType | DateType | TimestampType | BinaryType => true case _ => false } diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxAggregateFunctionsSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxAggregateFunctionsSuite.scala index 992106d131e67..270334011da7b 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxAggregateFunctionsSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxAggregateFunctionsSuite.scala @@ -26,6 +26,8 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ +import java.sql.Timestamp + abstract class VeloxAggregateFunctionsSuite extends VeloxWholeStageTransformerSuite { protected val rootPath: String = getClass.getResource("/").getPath @@ -565,6 +567,18 @@ abstract class VeloxAggregateFunctionsSuite extends VeloxWholeStageTransformerSu "select approx_count_distinct(l_discount), count(distinct l_orderkey) from lineitem") { checkGlutenOperatorMatch[HashAggregateExecTransformer] } + withTempPath { + path => + val t1 = Timestamp.valueOf("2024-08-22 10:10:10.010") + val t2 = Timestamp.valueOf("2014-12-31 00:00:00.012") + val t3 = Timestamp.valueOf("1968-12-31 23:59:59.001") + Seq(t1, t2, t3).toDF("t").write.parquet(path.getCanonicalPath) + + spark.read.parquet(path.getCanonicalPath).createOrReplaceTempView("view") + runQueryAndCompare("select approx_count_distinct(t) from view") { + checkGlutenOperatorMatch[HashAggregateExecTransformer] + } + } } test("max_by") { From 681b8a8dacf00b32479720b120d897caade00b75 Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Mon, 29 Jul 2024 16:09:40 +0800 Subject: [PATCH 15/61] [VL] Expose API SparkMemoryUtil.dumpMemoryManagerStats(tmm: TaskMemoryManager) for debugging purpose --- .../apache/spark/memory/SparkMemoryUtil.scala | 140 +++++++++--------- 1 file changed, 71 insertions(+), 69 deletions(-) diff --git a/gluten-core/src/main/scala/org/apache/spark/memory/SparkMemoryUtil.scala b/gluten-core/src/main/scala/org/apache/spark/memory/SparkMemoryUtil.scala index 178310fd64976..6fd2d7a3ec102 100644 --- a/gluten-core/src/main/scala/org/apache/spark/memory/SparkMemoryUtil.scala +++ b/gluten-core/src/main/scala/org/apache/spark/memory/SparkMemoryUtil.scala @@ -56,80 +56,82 @@ object SparkMemoryUtil { smp.memoryFree + emp.memoryFree } + def dumpMemoryManagerStats(tmm: TaskMemoryManager): String = { + val stats = tmm.synchronized { + val consumers = consumersField.get(tmm).asInstanceOf[util.HashSet[MemoryConsumer]] + + // create stats map + val statsMap = new util.HashMap[String, MemoryUsageStats]() + consumers.asScala.foreach { + case mt: KnownNameAndStats => + statsMap.put(mt.name(), mt.stats()) + case mc => + statsMap.put( + mc.toString, + MemoryUsageStats + .newBuilder() + .setCurrent(mc.getUsed) + .setPeak(-1L) + .build()) + } + Preconditions.checkState(statsMap.size() == consumers.size()) + + // add root + new KnownNameAndStats { + override def name(): String = s"Task.${taskIdField.get(tmm)}" + + override def stats(): MemoryUsageStats = MemoryUsageStats + .newBuilder() + .setCurrent(tmm.getMemoryConsumptionForThisTask) + .setPeak(-1L) + .putAllChildren(statsMap) + .build() + } + } + + prettyPrintStats("Memory consumer stats: ", stats) + } + def dumpMemoryTargetStats(target: MemoryTarget): String = { - def collectRootStats(target: MemoryTarget): KnownNameAndStats = { - target.accept(new MemoryTargetVisitor[KnownNameAndStats] { - override def visit(overAcquire: OverAcquire): KnownNameAndStats = { - overAcquire.getTarget.accept(this) - } - - @nowarn - override def visit(regularMemoryConsumer: RegularMemoryConsumer): KnownNameAndStats = { - collectFromTaskMemoryManager(regularMemoryConsumer.getTaskMemoryManager) - } - - override def visit(throwOnOomMemoryTarget: ThrowOnOomMemoryTarget): KnownNameAndStats = { - throwOnOomMemoryTarget.target().accept(this) - } - - override def visit(treeMemoryConsumer: TreeMemoryConsumer): KnownNameAndStats = { - collectFromTaskMemoryManager(treeMemoryConsumer.getTaskMemoryManager) - } - - override def visit(node: TreeMemoryTargets.Node): KnownNameAndStats = { - node.parent().accept(this) // walk up to find the one bound with task memory manager - } - - private def collectFromTaskMemoryManager(tmm: TaskMemoryManager): KnownNameAndStats = { - tmm.synchronized { - val consumers = consumersField.get(tmm).asInstanceOf[util.HashSet[MemoryConsumer]] - - // create stats map - val statsMap = new util.HashMap[String, MemoryUsageStats]() - consumers.asScala.foreach { - case mt: KnownNameAndStats => - statsMap.put(mt.name(), mt.stats()) - case mc => - statsMap.put( - mc.toString, - MemoryUsageStats - .newBuilder() - .setCurrent(mc.getUsed) - .setPeak(-1L) - .build()) - } - Preconditions.checkState(statsMap.size() == consumers.size()) - - // add root - new KnownNameAndStats { - override def name(): String = s"Task.${taskIdField.get(tmm)}" - - override def stats(): MemoryUsageStats = MemoryUsageStats - .newBuilder() - .setCurrent(tmm.getMemoryConsumptionForThisTask) - .setPeak(-1L) - .putAllChildren(statsMap) - .build() - } - } - } + target.accept(new MemoryTargetVisitor[String] { + override def visit(overAcquire: OverAcquire): String = { + overAcquire.getTarget.accept(this) + } - override def visit(loggingMemoryTarget: LoggingMemoryTarget): KnownNameAndStats = { - loggingMemoryTarget.delegated().accept(this) - } + @nowarn + override def visit(regularMemoryConsumer: RegularMemoryConsumer): String = { + collectFromTaskMemoryManager(regularMemoryConsumer.getTaskMemoryManager) + } - override def visit(noopMemoryTarget: NoopMemoryTarget): KnownNameAndStats = { - noopMemoryTarget - } + override def visit(throwOnOomMemoryTarget: ThrowOnOomMemoryTarget): String = { + throwOnOomMemoryTarget.target().accept(this) + } - override def visit(dynamicOffHeapSizingMemoryTarget: DynamicOffHeapSizingMemoryTarget) - : KnownNameAndStats = { - dynamicOffHeapSizingMemoryTarget.delegated().accept(this) - } - }) - } + override def visit(treeMemoryConsumer: TreeMemoryConsumer): String = { + collectFromTaskMemoryManager(treeMemoryConsumer.getTaskMemoryManager) + } + + override def visit(node: TreeMemoryTargets.Node): String = { + node.parent().accept(this) // walk up to find the one bound with task memory manager + } + + private def collectFromTaskMemoryManager(tmm: TaskMemoryManager): String = { + dumpMemoryManagerStats(tmm) + } - prettyPrintStats("Memory consumer stats: ", collectRootStats(target)) + override def visit(loggingMemoryTarget: LoggingMemoryTarget): String = { + loggingMemoryTarget.delegated().accept(this) + } + + override def visit(noopMemoryTarget: NoopMemoryTarget): String = { + prettyPrintStats("No-op memory manager stats: ", noopMemoryTarget) + } + + override def visit( + dynamicOffHeapSizingMemoryTarget: DynamicOffHeapSizingMemoryTarget): String = { + dynamicOffHeapSizingMemoryTarget.delegated().accept(this) + } + }) } def prettyPrintStats(title: String, stats: KnownNameAndStats): String = { From 408f4cbb0a9769ed8ea86d4cd6cb5cfa3eb62527 Mon Sep 17 00:00:00 2001 From: Rong Ma Date: Mon, 29 Jul 2024 16:14:51 +0800 Subject: [PATCH 16/61] [VL] Row based sort follow-up (#6579) --- .../backendsapi/velox/VeloxMetricsApi.scala | 1 + cpp/core/CMakeLists.txt | 1 - cpp/core/jni/JniWrapper.cc | 6 +- cpp/core/shuffle/Options.cc | 18 -- cpp/core/shuffle/Options.h | 7 + cpp/velox/shuffle/RadixSort.h | 157 +++++++++++++ cpp/velox/shuffle/VeloxShuffleReader.cc | 4 +- cpp/velox/shuffle/VeloxShuffleReader.h | 3 + cpp/velox/shuffle/VeloxSortShuffleWriter.cc | 217 ++++++++++++------ cpp/velox/shuffle/VeloxSortShuffleWriter.h | 29 ++- cpp/velox/tests/VeloxShuffleWriterTest.cc | 11 +- .../utils/tests/VeloxShuffleWriterTestBase.h | 4 +- ...VeloxCelebornColumnarBatchSerializer.scala | 2 + .../VeloxCelebornColumnarShuffleWriter.scala | 17 +- .../vectorized/ShuffleWriterJniWrapper.java | 10 + .../spark/shuffle/ColumnarShuffleWriter.scala | 11 +- .../VeloxUniffleColumnarShuffleWriter.java | 8 +- .../org/apache/gluten/GlutenConfig.scala | 1 - 18 files changed, 384 insertions(+), 123 deletions(-) delete mode 100644 cpp/core/shuffle/Options.cc create mode 100644 cpp/velox/shuffle/RadixSort.h diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxMetricsApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxMetricsApi.scala index 30b08749e9070..c05ce7cdcd4ff 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxMetricsApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxMetricsApi.scala @@ -272,6 +272,7 @@ class VeloxMetricsApi extends MetricsApi with Logging { "compressTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time to compress"), "decompressTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time to decompress"), "deserializeTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time to deserialize"), + "shuffleWallTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "shuffle wall time"), // For hash shuffle writer, the peak bytes represents the maximum split buffer size. // For sort shuffle writer, the peak bytes represents the maximum // row buffer + sort buffer size. diff --git a/cpp/core/CMakeLists.txt b/cpp/core/CMakeLists.txt index caad8db1eb9f5..ef21ccbe855aa 100644 --- a/cpp/core/CMakeLists.txt +++ b/cpp/core/CMakeLists.txt @@ -191,7 +191,6 @@ set(SPARK_COLUMNAR_PLUGIN_SRCS shuffle/FallbackRangePartitioner.cc shuffle/HashPartitioner.cc shuffle/LocalPartitionWriter.cc - shuffle/Options.cc shuffle/Partitioner.cc shuffle/Partitioning.cc shuffle/Payload.cc diff --git a/cpp/core/jni/JniWrapper.cc b/cpp/core/jni/JniWrapper.cc index 3d6da31c7e75c..f39f9c92333e9 100644 --- a/cpp/core/jni/JniWrapper.cc +++ b/cpp/core/jni/JniWrapper.cc @@ -755,6 +755,8 @@ JNIEXPORT jlong JNICALL Java_org_apache_gluten_vectorized_ShuffleWriterJniWrappe jint compressionLevel, jint compressionThreshold, jstring compressionModeJstr, + jint sortBufferInitialSize, + jboolean useRadixSort, jstring dataFileJstr, jint numSubDirs, jstring localDirsJstr, @@ -780,7 +782,9 @@ JNIEXPORT jlong JNICALL Java_org_apache_gluten_vectorized_ShuffleWriterJniWrappe .partitioning = gluten::toPartitioning(jStringToCString(env, partitioningNameJstr)), .taskAttemptId = (int64_t)taskAttemptId, .startPartitionId = startPartitionId, - .shuffleWriterType = gluten::ShuffleWriter::stringToType(jStringToCString(env, shuffleWriterTypeJstr))}; + .shuffleWriterType = gluten::ShuffleWriter::stringToType(jStringToCString(env, shuffleWriterTypeJstr)), + .sortBufferInitialSize = sortBufferInitialSize, + .useRadixSort = static_cast(useRadixSort)}; // Build PartitionWriterOptions. auto partitionWriterOptions = PartitionWriterOptions{ diff --git a/cpp/core/shuffle/Options.cc b/cpp/core/shuffle/Options.cc deleted file mode 100644 index 8e05a10d68594..0000000000000 --- a/cpp/core/shuffle/Options.cc +++ /dev/null @@ -1,18 +0,0 @@ -/* - * 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. - */ - -#include "shuffle/Options.h" diff --git a/cpp/core/shuffle/Options.h b/cpp/core/shuffle/Options.h index 757950d03443a..11fa037eb5a6f 100644 --- a/cpp/core/shuffle/Options.h +++ b/cpp/core/shuffle/Options.h @@ -35,9 +35,12 @@ static constexpr int32_t kDefaultBufferAlignment = 64; static constexpr double kDefaultBufferReallocThreshold = 0.25; static constexpr double kDefaultMergeBufferThreshold = 0.25; static constexpr bool kEnableBufferedWrite = true; +static constexpr bool kDefaultUseRadixSort = true; +static constexpr int32_t kDefaultSortBufferSize = 4096; enum ShuffleWriterType { kHashShuffle, kSortShuffle, kRssSortShuffle }; enum PartitionWriterType { kLocal, kRss }; +enum SortAlgorithm { kRadixSort, kQuickSort }; struct ShuffleReaderOptions { arrow::Compression::type compressionType = arrow::Compression::type::LZ4_FRAME; @@ -56,6 +59,10 @@ struct ShuffleWriterOptions { int32_t startPartitionId = 0; int64_t threadId = -1; ShuffleWriterType shuffleWriterType = kHashShuffle; + + // Sort shuffle writer. + int32_t sortBufferInitialSize = kDefaultSortBufferSize; + bool useRadixSort = kDefaultUseRadixSort; }; struct PartitionWriterOptions { diff --git a/cpp/velox/shuffle/RadixSort.h b/cpp/velox/shuffle/RadixSort.h new file mode 100644 index 0000000000000..17f05d349c8e2 --- /dev/null +++ b/cpp/velox/shuffle/RadixSort.h @@ -0,0 +1,157 @@ +/* + * 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. + */ +#include +#include +#include +#include + +namespace gluten { + +template +class RadixSort { + public: + /** + * Sorts a given array of longs using least-significant-digit radix sort. This routine assumes + * you have extra space at the end of the array at least equal to the number of records. The + * sort is destructive and may relocate the data positioned within the array. + * + * @param array array of long elements followed by at least that many empty slots. + * @param numRecords number of data records in the array. + * @param startByteIndex the first byte (in range [0, 7]) to sort each long by, counting from the + * least significant byte. + * @param endByteIndex the last byte (in range [0, 7]) to sort each long by, counting from the + * least significant byte. Must be greater than startByteIndex. + * + * @return The starting index of the sorted data within the given array. We return this instead + * of always copying the data back to position zero for efficiency. + */ + static int32_t sort(Element* array, size_t size, int64_t numRecords, int32_t startByteIndex, int32_t endByteIndex) { + assert(startByteIndex >= 0 && "startByteIndex should >= 0"); + assert(endByteIndex <= 7 && "endByteIndex should <= 7"); + assert(endByteIndex > startByteIndex); + assert(numRecords * 2 <= size); + + int64_t inIndex = 0; + int64_t outIndex = numRecords; + + if (numRecords > 0) { + auto counts = getCounts(array, numRecords, startByteIndex, endByteIndex); + + for (auto i = startByteIndex; i <= endByteIndex; i++) { + if (!counts[i].empty()) { + sortAtByte(array, numRecords, counts[i], i, inIndex, outIndex); + std::swap(inIndex, outIndex); + } + } + } + + return static_cast(inIndex); + } + + private: + /** + * Performs a partial sort by copying data into destination offsets for each byte value at the + * specified byte offset. + * + * @param array array to partially sort. + * @param numRecords number of data records in the array. + * @param counts counts for each byte value. This routine destructively modifies this array. + * @param byteIdx the byte in a long to sort at, counting from the least significant byte. + * @param inIndex the starting index in the array where input data is located. + * @param outIndex the starting index where sorted output data should be written. + */ + static void sortAtByte( + Element* array, + int64_t numRecords, + std::vector& counts, + int32_t byteIdx, + int64_t inIndex, + int64_t outIndex) { + assert(counts.size() == 256); + + auto offsets = transformCountsToOffsets(counts, outIndex); + + for (auto offset = inIndex; offset < inIndex + numRecords; ++offset) { + auto bucket = (array[offset] >> (byteIdx * 8)) & 0xff; + array[offsets[bucket]++] = array[offset]; + } + } + + /** + * Computes a value histogram for each byte in the given array. + * + * @param array array to count records in. + * @param numRecords number of data records in the array. + * @param startByteIndex the first byte to compute counts for (the prior are skipped). + * @param endByteIndex the last byte to compute counts for. + * + * @return a vector of eight 256-element count arrays, one for each byte starting from the least + * significant byte. If the byte does not need sorting the vector entry will be empty. + */ + static std::vector> + getCounts(Element* array, int64_t numRecords, int32_t startByteIndex, int32_t endByteIndex) { + std::vector> counts; + counts.resize(8); + + // Optimization: do a fast pre-pass to determine which byte indices we can skip for sorting. + // If all the byte values at a particular index are the same we don't need to count it. + int64_t bitwiseMax = 0; + int64_t bitwiseMin = -1L; + for (auto offset = 0; offset < numRecords; ++offset) { + auto value = array[offset]; + bitwiseMax |= value; + bitwiseMin &= value; + } + auto bitsChanged = bitwiseMin ^ bitwiseMax; + + // Compute counts for each byte index. + for (auto i = startByteIndex; i <= endByteIndex; i++) { + if (((bitsChanged >> (i * 8)) & 0xff) != 0) { + counts[i].resize(256); + for (auto offset = 0; offset < numRecords; ++offset) { + counts[i][(array[offset] >> (i * 8)) & 0xff]++; + } + } + } + + return counts; + } + + /** + * Transforms counts into the proper output offsets for the sort type. + * + * @param counts counts for each byte value. This routine destructively modifies this vector. + * @param numRecords number of data records in the original data array. + * @param outputOffset output offset in bytes from the base array object. + * + * @return the input counts vector. + */ + static std::vector& transformCountsToOffsets(std::vector& counts, int64_t outputOffset) { + assert(counts.size() == 256); + + int64_t pos = outputOffset; + for (auto i = 0; i < 256; i++) { + auto tmp = counts[i & 0xff]; + counts[i & 0xff] = pos; + pos += tmp; + } + + return counts; + } +}; + +} // namespace gluten diff --git a/cpp/velox/shuffle/VeloxShuffleReader.cc b/cpp/velox/shuffle/VeloxShuffleReader.cc index e55f4d01de821..ab93d9a33d04a 100644 --- a/cpp/velox/shuffle/VeloxShuffleReader.cc +++ b/cpp/velox/shuffle/VeloxShuffleReader.cc @@ -428,8 +428,8 @@ std::shared_ptr VeloxSortShuffleReaderDeserializer::deserializeTo auto buffer = cur->second; const auto* rawBuffer = buffer->as(); while (rowOffset_ < cur->first && readRows < batchSize_) { - auto rowSize = *(uint32_t*)(rawBuffer + byteOffset_); - byteOffset_ += sizeof(uint32_t); + auto rowSize = *(RowSizeType*)(rawBuffer + byteOffset_) - sizeof(RowSizeType); + byteOffset_ += sizeof(RowSizeType); data.push_back(std::string_view(rawBuffer + byteOffset_, rowSize)); byteOffset_ += rowSize; ++rowOffset_; diff --git a/cpp/velox/shuffle/VeloxShuffleReader.h b/cpp/velox/shuffle/VeloxShuffleReader.h index 96273948526da..2be913aa13a72 100644 --- a/cpp/velox/shuffle/VeloxShuffleReader.h +++ b/cpp/velox/shuffle/VeloxShuffleReader.h @@ -20,6 +20,7 @@ #include "operators/serializer/VeloxColumnarBatchSerializer.h" #include "shuffle/Payload.h" #include "shuffle/ShuffleReader.h" +#include "shuffle/VeloxSortShuffleWriter.h" #include "velox/type/Type.h" #include "velox/vector/ComplexVector.h" @@ -64,6 +65,8 @@ class VeloxHashShuffleReaderDeserializer final : public ColumnarBatchIterator { class VeloxSortShuffleReaderDeserializer final : public ColumnarBatchIterator { public: + using RowSizeType = VeloxSortShuffleWriter::RowSizeType; + VeloxSortShuffleReaderDeserializer( std::shared_ptr in, const std::shared_ptr& schema, diff --git a/cpp/velox/shuffle/VeloxSortShuffleWriter.cc b/cpp/velox/shuffle/VeloxSortShuffleWriter.cc index 7c033fb98bf81..0015ba9d36c3c 100644 --- a/cpp/velox/shuffle/VeloxSortShuffleWriter.cc +++ b/cpp/velox/shuffle/VeloxSortShuffleWriter.cc @@ -29,6 +29,10 @@ namespace gluten { namespace { constexpr uint32_t kMaskLower27Bits = (1 << 27) - 1; constexpr uint64_t kMaskLower40Bits = (1UL << 40) - 1; +constexpr uint32_t kPartitionIdStartByteIndex = 5; +constexpr uint32_t kPartitionIdEndByteIndex = 7; + +constexpr uint32_t kSortedBufferSize = 1 * 1024 * 1024; uint64_t toCompactRowId(uint32_t partitionId, uint32_t pageNumber, uint32_t offsetInPage) { // |63 partitionId(24) |39 inputIndex(13) |26 rowIndex(27) | @@ -62,9 +66,7 @@ VeloxSortShuffleWriter::VeloxSortShuffleWriter( ShuffleWriterOptions options, std::shared_ptr veloxPool, arrow::MemoryPool* pool) - : VeloxShuffleWriter(numPartitions, std::move(partitionWriter), std::move(options), std::move(veloxPool), pool), - allocator_{std::make_unique(veloxPool_.get())}, - array_{SortArray{Allocator(allocator_.get())}} {} + : VeloxShuffleWriter(numPartitions, std::move(partitionWriter), std::move(options), std::move(veloxPool), pool) {} arrow::Status VeloxSortShuffleWriter::write(std::shared_ptr cb, int64_t memLimit) { ARROW_ASSIGN_OR_RAISE(auto rv, getPeeledRowVector(cb)); @@ -76,10 +78,14 @@ arrow::Status VeloxSortShuffleWriter::write(std::shared_ptr cb, i arrow::Status VeloxSortShuffleWriter::stop() { ARROW_RETURN_IF(evictState_ == EvictState::kUnevictable, arrow::Status::Invalid("Unevictable state in stop.")); - EvictGuard evictGuard{evictState_}; - stopped_ = true; - RETURN_NOT_OK(evictAllPartitions()); + if (offset_ > 0) { + RETURN_NOT_OK(evictAllPartitions()); + } + array_.reset(); + sortedBuffer_.reset(); + pages_.clear(); + pageAddresses_.clear(); RETURN_NOT_OK(partitionWriter_->stop(&metrics_)); return arrow::Status::OK(); } @@ -89,7 +95,6 @@ arrow::Status VeloxSortShuffleWriter::reclaimFixedSize(int64_t size, int64_t* ac *actual = 0; return arrow::Status::OK(); } - EvictGuard evictGuard{evictState_}; auto beforeReclaim = veloxPool_->usedBytes(); RETURN_NOT_OK(evictAllPartitions()); *actual = beforeReclaim - veloxPool_->usedBytes(); @@ -100,7 +105,9 @@ arrow::Status VeloxSortShuffleWriter::init() { ARROW_RETURN_IF( options_.partitioning == Partitioning::kSingle, arrow::Status::Invalid("VeloxSortShuffleWriter doesn't support single partition.")); - array_.resize(initialSize_); + initArray(); + sortedBuffer_ = facebook::velox::AlignedBuffer::allocate(kSortedBufferSize, veloxPool_.get()); + rawBuffer_ = sortedBuffer_->asMutable(); return arrow::Status::OK(); } @@ -108,6 +115,9 @@ void VeloxSortShuffleWriter::initRowType(const facebook::velox::RowVectorPtr& rv if (UNLIKELY(!rowType_)) { rowType_ = facebook::velox::asRowType(rv->type()); fixedRowSize_ = facebook::velox::row::CompactRow::fixedRowSize(rowType_); + if (fixedRowSize_) { + *fixedRowSize_ += sizeof(RowSizeType); + } } } @@ -148,11 +158,16 @@ arrow::Status VeloxSortShuffleWriter::insert(const facebook::velox::RowVectorPtr facebook::velox::row::CompactRow row(vector); if (!fixedRowSize_) { - rowSizes_.resize(inputRows + 1); - rowSizes_[0] = 0; + rowSize_.resize(inputRows); + rowSizePrefixSum_.resize(inputRows + 1); + rowSizePrefixSum_[0] = 0; for (auto i = 0; i < inputRows; ++i) { - rowSizes_[i + 1] = rowSizes_[i] + row.rowSize(i); + auto rowSize = row.rowSize(i) + sizeof(RowSizeType); + rowSize_[i] = rowSize; + rowSizePrefixSum_[i + 1] = rowSizePrefixSum_[i] + rowSize; } + } else { + rowSize_.resize(inputRows, *fixedRowSize_); } uint32_t rowOffset = 0; @@ -160,13 +175,16 @@ arrow::Status VeloxSortShuffleWriter::insert(const facebook::velox::RowVectorPtr auto remainingRows = inputRows - rowOffset; auto rows = maxRowsToInsert(rowOffset, remainingRows); if (rows == 0) { - auto minSizeRequired = fixedRowSize_ ? fixedRowSize_.value() : rowSizes_[rowOffset + 1] - rowSizes_[rowOffset]; + auto minSizeRequired = fixedRowSize_ ? fixedRowSize_.value() : rowSize_[rowOffset]; acquireNewBuffer((uint64_t)memLimit, minSizeRequired); rows = maxRowsToInsert(rowOffset, remainingRows); ARROW_RETURN_IF( rows == 0, arrow::Status::Invalid("Failed to insert rows. Remaining rows: " + std::to_string(remainingRows))); } + // Spill to avoid offset_ overflow. RETURN_NOT_OK(maybeSpill(rows)); + // Allocate newArray can trigger spill. + growArrayIfNecessary(rows); insertRows(row, rowOffset, rows); rowOffset += rows; } @@ -174,35 +192,48 @@ arrow::Status VeloxSortShuffleWriter::insert(const facebook::velox::RowVectorPtr } void VeloxSortShuffleWriter::insertRows(facebook::velox::row::CompactRow& row, uint32_t offset, uint32_t rows) { - // Allocate newArray can trigger spill. - growArrayIfNecessary(rows); + VELOX_CHECK(!pages_.empty()); for (auto i = offset; i < offset + rows; ++i) { + auto pid = row2Partition_[i]; + arrayPtr_[offset_++] = toCompactRowId(pid, pageNumber_, pageCursor_); + // size(RowSize) | bytes + memcpy(currentPage_ + pageCursor_, &rowSize_[i], sizeof(RowSizeType)); + pageCursor_ += sizeof(RowSizeType); auto size = row.serialize(i, currentPage_ + pageCursor_); - array_[offset_++] = {toCompactRowId(row2Partition_[i], pageNumber_, pageCursor_), size}; pageCursor_ += size; + VELOX_DCHECK_LE(pageCursor_, currenPageSize_); } } arrow::Status VeloxSortShuffleWriter::maybeSpill(int32_t nextRows) { if ((uint64_t)offset_ + nextRows > std::numeric_limits::max()) { - EvictGuard evictGuard{evictState_}; RETURN_NOT_OK(evictAllPartitions()); } return arrow::Status::OK(); } arrow::Status VeloxSortShuffleWriter::evictAllPartitions() { + EvictGuard evictGuard{evictState_}; + + auto numRecords = offset_; + int32_t begin = 0; { ScopedTimer timer(&sortTime_); - // TODO: Add radix sort to align with Spark. - std::sort(array_.begin(), array_.begin() + offset_); + if (options_.useRadixSort) { + begin = RadixSort::sort( + arrayPtr_, arraySize_, numRecords, kPartitionIdStartByteIndex, kPartitionIdEndByteIndex); + } else { + auto ptr = arrayPtr_; + qsort(ptr, numRecords, sizeof(uint64_t), compare); + (void)ptr; + } } - size_t begin = 0; - size_t cur = 0; - auto pid = extractPartitionId(array_[begin].first); - while (++cur < offset_) { - auto curPid = extractPartitionId(array_[cur].first); + auto end = begin + numRecords; + auto cur = begin; + auto pid = extractPartitionId(arrayPtr_[begin]); + while (++cur < end) { + auto curPid = extractPartitionId(arrayPtr_[cur]); if (curPid != pid) { RETURN_NOT_OK(evictPartition(pid, begin, cur)); pid = curPid; @@ -211,53 +242,61 @@ arrow::Status VeloxSortShuffleWriter::evictAllPartitions() { } RETURN_NOT_OK(evictPartition(pid, begin, cur)); - pageCursor_ = 0; - pages_.clear(); - pageAddresses_.clear(); - - offset_ = 0; - array_.clear(); - - sortedBuffer_ = nullptr; - if (!stopped_) { - // Allocate array_ can trigger spill. - array_.resize(initialSize_); + // Preserve the last page for use. + auto numPages = pages_.size(); + while (--numPages) { + pages_.pop_front(); + } + auto& page = pages_.back(); + // Clear page for serialization. + memset(page->asMutable(), 0, page->size()); + // currentPage_ should always point to the last page. + VELOX_CHECK(currentPage_ == page->asMutable()); + + pageAddresses_.resize(1); + pageAddresses_[0] = currentPage_; + pageNumber_ = 0; + pageCursor_ = 0; + + // Reset and reallocate array_ to minimal size. Allocate array_ can trigger spill. + offset_ = 0; + initArray(); } return arrow::Status::OK(); } arrow::Status VeloxSortShuffleWriter::evictPartition(uint32_t partitionId, size_t begin, size_t end) { + ScopedTimer timer(&sortTime_); // Serialize [begin, end) - uint32_t numRows = end - begin; - uint64_t rawSize = numRows * sizeof(RowSizeType); - for (auto i = begin; i < end; ++i) { - rawSize += array_[i].second; - } - - if (sortedBuffer_ == nullptr || sortedBuffer_->size() < rawSize) { - sortedBuffer_ = nullptr; - sortedBuffer_ = facebook::velox::AlignedBuffer::allocate(rawSize, veloxPool_.get()); - } - auto* rawBuffer = sortedBuffer_->asMutable(); - uint64_t offset = 0; - for (auto i = begin; i < end; ++i) { - // size(size_t) | bytes - auto size = array_[i].second; - memcpy(rawBuffer + offset, &size, sizeof(RowSizeType)); - offset += sizeof(RowSizeType); - auto index = extractPageNumberAndOffset(array_[i].first); - memcpy(rawBuffer + offset, pageAddresses_[index.first] + index.second, size); + char* addr; + uint32_t size; + + auto index = begin; + while (index < end) { + auto pageIndex = extractPageNumberAndOffset(arrayPtr_[index]); + addr = pageAddresses_[pageIndex.first] + pageIndex.second; + size = *(RowSizeType*)addr; + if (offset + size > kSortedBufferSize) { + VELOX_CHECK(offset > 0); + auto payload = std::make_unique( + index - begin, + nullptr, + std::vector>{std::make_shared(rawBuffer_, offset)}); + RETURN_NOT_OK( + partitionWriter_->evict(partitionId, std::move(payload), Evict::type::kSortSpill, false, false, stopped_)); + begin = index; + offset = 0; + } + gluten::fastCopy(rawBuffer_ + offset, addr, size); offset += size; + index++; } - VELOX_CHECK_EQ(offset, rawSize); - - auto rawData = sortedBuffer_->as(); - std::vector> buffers; - buffers.push_back(std::make_shared(rawData, rawSize)); - - auto payload = std::make_unique(numRows, nullptr, std::move(buffers)); + auto payload = std::make_unique( + end - begin, + nullptr, + std::vector>{std::make_shared(rawBuffer_, offset)}); RETURN_NOT_OK( partitionWriter_->evict(partitionId, std::move(payload), Evict::type::kSortSpill, false, false, stopped_)); return arrow::Status::OK(); @@ -272,8 +311,8 @@ uint32_t VeloxSortShuffleWriter::maxRowsToInsert(uint32_t offset, uint32_t rows) if (fixedRowSize_) { return std::min((uint32_t)(remainingBytes / (fixedRowSize_.value())), rows); } - auto beginIter = rowSizes_.begin() + 1 + offset; - auto iter = std::upper_bound(beginIter, rowSizes_.end(), remainingBytes); + auto beginIter = rowSizePrefixSum_.begin() + 1 + offset; + auto iter = std::upper_bound(beginIter, rowSizePrefixSum_.end(), remainingBytes); return iter - beginIter; } @@ -281,23 +320,54 @@ void VeloxSortShuffleWriter::acquireNewBuffer(uint64_t memLimit, uint64_t minSiz auto size = std::max(std::min(memLimit >> 2, 64UL * 1024 * 1024), minSizeRequired); // Allocating new buffer can trigger spill. auto newBuffer = facebook::velox::AlignedBuffer::allocate(size, veloxPool_.get(), 0); + // If spill triggered, clear pages_. + if (offset_ == 0 && pages_.size() > 0) { + pageAddresses_.clear(); + pages_.clear(); + } + currentPage_ = newBuffer->asMutable(); + pageAddresses_.emplace_back(currentPage_); pages_.emplace_back(std::move(newBuffer)); + pageCursor_ = 0; pageNumber_ = pages_.size() - 1; - currentPage_ = pages_.back()->asMutable(); - pageAddresses_.emplace_back(currentPage_); + currenPageSize_ = pages_.back()->size(); } void VeloxSortShuffleWriter::growArrayIfNecessary(uint32_t rows) { - auto arraySize = (uint32_t)array_.size(); - auto usableCapacity = useRadixSort_ ? arraySize / 2 : arraySize; - while (offset_ + rows > usableCapacity) { - arraySize <<= 1; - usableCapacity = useRadixSort_ ? arraySize / 2 : arraySize; + auto newSize = newArraySize(rows); + if (newSize > arraySize_) { + // May trigger spill. + auto newSizeBytes = newSize * sizeof(uint64_t); + auto newArray = facebook::velox::AlignedBuffer::allocate(newSizeBytes, veloxPool_.get()); + // Check if already satisfies. + if (newArraySize(rows) > arraySize_) { + auto newPtr = newArray->asMutable(); + if (offset_ > 0) { + gluten::fastCopy(newPtr, arrayPtr_, offset_ * sizeof(uint64_t)); + } + arraySize_ = newSize; + arrayPtr_ = newPtr; + array_.reset(); + array_.swap(newArray); + } } - if (arraySize != array_.size()) { - array_.resize(arraySize); +} + +uint32_t VeloxSortShuffleWriter::newArraySize(uint32_t rows) { + auto newSize = arraySize_; + auto usableCapacity = options_.useRadixSort ? newSize / 2 : newSize; + while (offset_ + rows > usableCapacity) { + newSize <<= 1; + usableCapacity = options_.useRadixSort ? newSize / 2 : newSize; } + return newSize; +} + +void VeloxSortShuffleWriter::initArray() { + arraySize_ = options_.sortBufferInitialSize; + array_ = facebook::velox::AlignedBuffer::allocate(arraySize_ * sizeof(uint64_t), veloxPool_.get()); + arrayPtr_ = array_->asMutable(); } int64_t VeloxSortShuffleWriter::peakBytesAllocated() const { @@ -311,4 +381,9 @@ int64_t VeloxSortShuffleWriter::totalSortTime() const { int64_t VeloxSortShuffleWriter::totalC2RTime() const { return c2rTime_; } + +int VeloxSortShuffleWriter::compare(const void* a, const void* b) { + // No same values. + return *(uint64_t*)a > *(uint64_t*)b ? 1 : -1; +} } // namespace gluten diff --git a/cpp/velox/shuffle/VeloxSortShuffleWriter.h b/cpp/velox/shuffle/VeloxSortShuffleWriter.h index 6ac5308d0bd8c..747593ae457d9 100644 --- a/cpp/velox/shuffle/VeloxSortShuffleWriter.h +++ b/cpp/velox/shuffle/VeloxSortShuffleWriter.h @@ -17,6 +17,7 @@ #pragma once +#include "shuffle/RadixSort.h" #include "shuffle/VeloxShuffleWriter.h" #include @@ -31,6 +32,8 @@ namespace gluten { class VeloxSortShuffleWriter final : public VeloxShuffleWriter { public: + using RowSizeType = uint32_t; + static arrow::Result> create( uint32_t numPartitions, std::unique_ptr partitionWriter, @@ -80,27 +83,28 @@ class VeloxSortShuffleWriter final : public VeloxShuffleWriter { void growArrayIfNecessary(uint32_t rows); - using RowSizeType = uint32_t; - using ElementType = std::pair; - using Allocator = facebook::velox::StlAllocator; - using SortArray = std::vector; + uint32_t newArraySize(uint32_t rows); + + void initArray(); + + static int compare(const void* a, const void* b); - std::unique_ptr allocator_; // Stores compact row id -> row - SortArray array_; + facebook::velox::BufferPtr array_; + uint64_t* arrayPtr_; + uint32_t arraySize_; uint32_t offset_{0}; - std::vector pages_; + std::list pages_; std::vector pageAddresses_; char* currentPage_; uint32_t pageNumber_; uint32_t pageCursor_; - - // FIXME: Use configuration to replace hardcode. - uint32_t initialSize_ = 4096; - bool useRadixSort_ = false; + // For debug. + uint32_t currenPageSize_; facebook::velox::BufferPtr sortedBuffer_; + uint8_t* rawBuffer_; // Row ID -> Partition ID // subscript: The index of row in the current input RowVector @@ -110,7 +114,8 @@ class VeloxSortShuffleWriter final : public VeloxShuffleWriter { std::shared_ptr rowType_; std::optional fixedRowSize_; - std::vector rowSizes_; + std::vector rowSize_; + std::vector rowSizePrefixSum_; int64_t c2rTime_{0}; int64_t sortTime_{0}; diff --git a/cpp/velox/tests/VeloxShuffleWriterTest.cc b/cpp/velox/tests/VeloxShuffleWriterTest.cc index 6b86f6a0a15c3..af9d5a58db0d4 100644 --- a/cpp/velox/tests/VeloxShuffleWriterTest.cc +++ b/cpp/velox/tests/VeloxShuffleWriterTest.cc @@ -70,10 +70,12 @@ std::vector createShuffleTestParams() { std::vector mergeBufferSizes = {0, 3, 4, 10, 4096}; for (const auto& compression : compressions) { + for (auto useRadixSort : {true, false}) { + params.push_back(ShuffleTestParams{ + ShuffleWriterType::kSortShuffle, PartitionWriterType::kLocal, compression, 0, 0, useRadixSort}); + } params.push_back( - ShuffleTestParams{ShuffleWriterType::kSortShuffle, PartitionWriterType::kLocal, compression, 0, 0}); - params.push_back( - ShuffleTestParams{ShuffleWriterType::kRssSortShuffle, PartitionWriterType::kRss, compression, 0, 0}); + ShuffleTestParams{ShuffleWriterType::kRssSortShuffle, PartitionWriterType::kRss, compression, 0, 0, false}); for (const auto compressionThreshold : compressionThresholds) { for (const auto mergeBufferSize : mergeBufferSizes) { params.push_back(ShuffleTestParams{ @@ -81,7 +83,8 @@ std::vector createShuffleTestParams() { PartitionWriterType::kLocal, compression, compressionThreshold, - mergeBufferSize}); + mergeBufferSize, + false /* unused */}); } params.push_back(ShuffleTestParams{ ShuffleWriterType::kHashShuffle, PartitionWriterType::kRss, compression, compressionThreshold, 0}); diff --git a/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h b/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h index f9c2b1d073390..d32e3272186b5 100644 --- a/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h +++ b/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h @@ -68,12 +68,13 @@ struct ShuffleTestParams { arrow::Compression::type compressionType; int32_t compressionThreshold; int32_t mergeBufferSize; + bool useRadixSort; std::string toString() const { std::ostringstream out; out << "shuffleWriterType = " << shuffleWriterType << ", partitionWriterType = " << partitionWriterType << ", compressionType = " << compressionType << ", compressionThreshold = " << compressionThreshold - << ", mergeBufferSize = " << mergeBufferSize; + << ", mergeBufferSize = " << mergeBufferSize << ", useRadixSort = " << (useRadixSort ? "true" : "false"); return out.str(); } }; @@ -250,6 +251,7 @@ class VeloxShuffleWriterTest : public ::testing::TestWithParam> records) throws IOException { compressionLevel, compressThreshold, GlutenConfig.getConf().columnarShuffleCompressionMode(), + (int) (long) sparkConf.get(package$.MODULE$.SHUFFLE_SORT_INIT_BUFFER_SIZE()), + (boolean) sparkConf.get(package$.MODULE$.SHUFFLE_SORT_USE_RADIXSORT()), bufferSize, bufferSize, partitionPusher, @@ -180,7 +182,7 @@ public long spill(MemoryTarget self, Spiller.Phase phase, long size) { LOG.debug("jniWrapper.write rows {}, split bytes {}", cb.numRows(), bytes); columnarDep.metrics().get("dataSize").get().add(bytes); // this metric replace part of uniffle shuffle write time - columnarDep.metrics().get("splitTime").get().add(System.nanoTime() - startTime); + columnarDep.metrics().get("shuffleWallTime").get().add(System.nanoTime() - startTime); columnarDep.metrics().get("numInputRows").get().add(cb.numRows()); columnarDep.metrics().get("inputBatches").get().add(1); shuffleWriteMetrics.incRecordsWritten(cb.numRows()); @@ -193,13 +195,13 @@ public long spill(MemoryTarget self, Spiller.Phase phase, long size) { throw new IllegalStateException("nativeShuffleWriter should not be -1L"); } splitResult = jniWrapper.stop(nativeShuffleWriter); + columnarDep.metrics().get("shuffleWallTime").get().add(System.nanoTime() - startTime); columnarDep .metrics() .get("splitTime") .get() .add( - System.nanoTime() - - startTime + columnarDep.metrics().get("shuffleWallTime").get().value() - splitResult.getTotalPushTime() - splitResult.getTotalWriteTime() - splitResult.getTotalCompressTime()); diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index 5547feafe331a..586a277f78554 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -137,7 +137,6 @@ class GlutenConfig(conf: SQLConf) extends Logging { conf .getConfString("spark.celeborn.client.spark.shuffle.writer", GLUTEN_HASH_SHUFFLE_WRITER) .toLowerCase(Locale.ROOT) - .replace(GLUTEN_SORT_SHUFFLE_WRITER, GLUTEN_RSS_SORT_SHUFFLE_WRITER) def enableColumnarShuffle: Boolean = conf.getConf(COLUMNAR_SHUFFLE_ENABLED) From ca49ab9ce13e2c96250446ef6f3deb0539eef801 Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Mon, 29 Jul 2024 16:56:48 +0800 Subject: [PATCH 17/61] [VL] Daily Update Velox Version (2024_07_29) (#6616) Co-authored-by: joey.ljy --- ep/build-velox/src/get_velox.sh | 28 +++++------- ep/build-velox/src/setup-centos7.sh | 18 +++----- ep/build-velox/src/setup-centos8.sh | 68 +++++++++-------------------- 3 files changed, 38 insertions(+), 76 deletions(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 841787d64c237..d9b22482d399b 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_07_26 +VELOX_BRANCH=2024_07_29 VELOX_HOME="" OS=`uname -s` @@ -70,25 +70,22 @@ function process_setup_ubuntu { ensure_pattern_matched '\${SUDO} apt install -y libunwind-dev' scripts/setup-ubuntu.sh sed -i 's/${SUDO} apt install -y libunwind-dev//' scripts/setup-ubuntu.sh ensure_pattern_matched 'ccache' scripts/setup-ubuntu.sh - sed -i '/ccache/a\ *thrift* \\' scripts/setup-ubuntu.sh - sed -i '/ccache/a\ libiberty-dev \\' scripts/setup-ubuntu.sh - sed -i '/ccache/a\ libxml2-dev \\' scripts/setup-ubuntu.sh - sed -i '/ccache/a\ libkrb5-dev \\' scripts/setup-ubuntu.sh - sed -i '/ccache/a\ libgsasl7-dev \\' scripts/setup-ubuntu.sh - sed -i '/ccache/a\ libuuid1 \\' scripts/setup-ubuntu.sh - sed -i '/ccache/a\ uuid-dev \\' scripts/setup-ubuntu.sh - sed -i '/ccache/a\ curl \\' scripts/setup-ubuntu.sh + sed -i '/ccache/a\ *thrift* \\' scripts/setup-ubuntu.sh + sed -i '/ccache/a\ libiberty-dev \\' scripts/setup-ubuntu.sh + sed -i '/ccache/a\ libxml2-dev \\' scripts/setup-ubuntu.sh + sed -i '/ccache/a\ libkrb5-dev \\' scripts/setup-ubuntu.sh + sed -i '/ccache/a\ libgsasl7-dev \\' scripts/setup-ubuntu.sh + sed -i '/ccache/a\ libuuid1 \\' scripts/setup-ubuntu.sh + sed -i '/ccache/a\ uuid-dev \\' scripts/setup-ubuntu.sh ensure_pattern_matched 'libgmock-dev' scripts/setup-ubuntu.sh sed -i '/libgmock-dev/d' scripts/setup-ubuntu.sh # resolved by ep/build-velox/build/velox_ep/CMake/resolve_dependency_modules/gtest.cmake - ensure_pattern_matched 'github_checkout boostorg\/boost \"\${BOOST_VERSION}\" --recursive' scripts/setup-ubuntu.sh - sed -i 's/github_checkout boostorg\/boost \"\${BOOST_VERSION}\" --recursive/wget_and_untar https:\/\/github.com\/boostorg\/boost\/releases\/download\/boost-1.84.0\/boost-1.84.0.tar.gz boost \&\& cd boost/g' scripts/setup-ubuntu.sh ensure_pattern_matched 'function install_folly' scripts/setup-ubuntu.sh - sed -i '/^function install_folly.*/i function install_protobuf {\n wget https://github.com/protocolbuffers/protobuf/releases/download/v21.4/protobuf-all-21.4.tar.gz\n tar -xzf protobuf-all-21.4.tar.gz\n cd protobuf-21.4\n ./configure CXXFLAGS="-fPIC" --prefix=/usr/local\n make "-j$(nproc)"\n sudo make install\n sudo ldconfig\n}\n' scripts/setup-ubuntu.sh + sed -i '/^function install_folly.*/i function install_protobuf {\n wget_and_untar https://github.com/protocolbuffers/protobuf/releases/download/v21.4/protobuf-all-21.4.tar.gz protobuf\n (\n cd protobuf\n ./configure CXXFLAGS="-fPIC" --prefix=/usr/local\n make "-j$(nproc)"\n sudo make install\n sudo ldconfig\n )\n}\n' scripts/setup-ubuntu.sh ensure_pattern_matched ' run_and_time install_folly' scripts/setup-ubuntu.sh sed -i '/^ run_and_time install_folly/a \ \ run_and_time install_protobuf' scripts/setup-ubuntu.sh # Required by lib hdfs. ensure_pattern_matched 'ccache ' scripts/setup-ubuntu.sh - sed -i '/ccache /a\ yasm \\' scripts/setup-ubuntu.sh + sed -i '/ccache /a\ yasm \\' scripts/setup-ubuntu.sh ensure_pattern_matched 'run_and_time install_conda' scripts/setup-ubuntu.sh sed -i '/run_and_time install_conda/d' scripts/setup-ubuntu.sh # Just depends on Gluten to install arrow libs since Gluten will apply some patches to Arrow source and uses different build options. @@ -107,11 +104,10 @@ function process_setup_centos9 { ensure_pattern_matched 'dnf_install' scripts/setup-centos9.sh sed -i 's/dnf_install ninja-build cmake curl ccache gcc-toolset-12 git/dnf_install ninja-build cmake curl ccache gcc-toolset-12/' scripts/setup-centos9.sh - sed -i '/^function dnf_install/i\DEPENDENCY_DIR=${DEPENDENCY_DIR:-$(pwd)}' scripts/setup-centos9.sh - sed -i '/^dnf_install autoconf/a\dnf_install libxml2-devel libgsasl-devel libuuid-devel' scripts/setup-centos9.sh + sed -i '/^.*dnf_install autoconf/a\ dnf_install libxml2-devel libgsasl-devel libuuid-devel' scripts/setup-centos9.sh ensure_pattern_matched 'install_gflags' scripts/setup-centos9.sh - sed -i '/^function install_gflags.*/i function install_openssl {\n wget_and_untar https://github.com/openssl/openssl/archive/refs/tags/OpenSSL_1_1_1s.tar.gz openssl \n cd openssl \n ./config no-shared && make depend && make && sudo make install \n cd ..\n}\n' scripts/setup-centos9.sh + sed -i '/^function install_gflags.*/i function install_openssl {\n wget_and_untar https://github.com/openssl/openssl/archive/refs/tags/OpenSSL_1_1_1s.tar.gz openssl \n cd openssl \n ./config no-shared && make depend && make && sudo make install \n cd ..\n}\n' scripts/setup-centos9.sh ensure_pattern_matched 'install_fbthrift' scripts/setup-centos9.sh sed -i '/^ run_and_time install_fbthrift/a \ run_and_time install_openssl' scripts/setup-centos9.sh diff --git a/ep/build-velox/src/setup-centos7.sh b/ep/build-velox/src/setup-centos7.sh index 6f76112ae6a46..34d7bcfb65c5d 100755 --- a/ep/build-velox/src/setup-centos7.sh +++ b/ep/build-velox/src/setup-centos7.sh @@ -74,8 +74,8 @@ function install_ninja { function install_folly { cd "${DEPENDENCY_DIR}" - github_checkout facebook/folly "${FB_OS_VERSION}" - cmake_install -DBUILD_TESTS=OFF -DFOLLY_HAVE_INT128_T=ON + wget_and_untar https://github.com/facebook/folly/archive/refs/tags/${FB_OS_VERSION}.tar.gz folly + cmake_install folly -DBUILD_TESTS=OFF -DFOLLY_HAVE_INT128_T=ON } function install_conda { @@ -99,22 +99,19 @@ function install_openssl { function install_gflags { cd "${DEPENDENCY_DIR}" wget_and_untar https://github.com/gflags/gflags/archive/v2.2.2.tar.gz gflags - cd gflags - cmake_install -DBUILD_SHARED_LIBS=ON -DBUILD_STATIC_LIBS=ON -DBUILD_gflags_LIB=ON -DLIB_SUFFIX=64 -DCMAKE_INSTALL_PREFIX:PATH=/usr/local + cmake_install gflags -DBUILD_SHARED_LIBS=ON -DBUILD_STATIC_LIBS=ON -DBUILD_gflags_LIB=ON -DLIB_SUFFIX=64 -DCMAKE_INSTALL_PREFIX:PATH=/usr/local } function install_glog { cd "${DEPENDENCY_DIR}" wget_and_untar https://github.com/google/glog/archive/v0.5.0.tar.gz glog - cd glog - cmake_install -DBUILD_SHARED_LIBS=ON -DBUILD_STATIC_LIBS=ON -DCMAKE_INSTALL_PREFIX:PATH=/usr/local + cmake_install glog -DBUILD_SHARED_LIBS=ON -DBUILD_STATIC_LIBS=ON -DCMAKE_INSTALL_PREFIX:PATH=/usr/local } function install_snappy { cd "${DEPENDENCY_DIR}" wget_and_untar https://github.com/google/snappy/archive/1.1.8.tar.gz snappy - cd snappy - cmake_install -DSNAPPY_BUILD_TESTS=OFF + cmake_install snappy -DSNAPPY_BUILD_TESTS=OFF } function install_dwarf { @@ -213,10 +210,7 @@ function install_duckdb { if $BUILD_DUCKDB ; then echo 'Building DuckDB' wget_and_untar https://github.com/duckdb/duckdb/archive/refs/tags/v0.8.1.tar.gz duckdb - ( - cd duckdb - cmake_install -DBUILD_UNITTESTS=OFF -DENABLE_SANITIZER=OFF -DENABLE_UBSAN=OFF -DBUILD_SHELL=OFF -DEXPORT_DLL_SYMBOLS=OFF -DCMAKE_BUILD_TYPE=Release - ) + cmake_install duckdb -DBUILD_UNITTESTS=OFF -DENABLE_SANITIZER=OFF -DENABLE_UBSAN=OFF -DBUILD_SHELL=OFF -DEXPORT_DLL_SYMBOLS=OFF -DCMAKE_BUILD_TYPE=Release fi } diff --git a/ep/build-velox/src/setup-centos8.sh b/ep/build-velox/src/setup-centos8.sh index c8078893ae685..771c2ab835dfe 100755 --- a/ep/build-velox/src/setup-centos8.sh +++ b/ep/build-velox/src/setup-centos8.sh @@ -41,7 +41,10 @@ BUILD_DUCKDB="${BUILD_DUCKDB:-true}" export CC=/opt/rh/gcc-toolset-9/root/bin/gcc export CXX=/opt/rh/gcc-toolset-9/root/bin/g++ -DEPENDENCY_DIR=${DEPENDENCY_DIR:-$(pwd)} +FB_OS_VERSION="v2024.05.20.00" +FMT_VERSION="10.1.1" +BOOST_VERSION="boost-1.84.0" + function dnf_install { dnf install -y -q --setopt=install_weak_deps=False "$@" } @@ -74,27 +77,22 @@ function install_conda { function install_openssl { wget_and_untar https://github.com/openssl/openssl/archive/refs/tags/OpenSSL_1_1_1s.tar.gz openssl - cd openssl - ./config no-shared && make depend && make && sudo make install - cd .. + ( + cd openssl + ./config no-shared && make depend && make && sudo make install + ) } function install_gflags { # Remove an older version if present. dnf remove -y gflags wget_and_untar https://github.com/gflags/gflags/archive/v2.2.2.tar.gz gflags - ( - cd gflags - cmake_install -DBUILD_SHARED_LIBS=ON -DBUILD_STATIC_LIBS=ON -DBUILD_gflags_LIB=ON -DLIB_SUFFIX=64 - ) + cmake_install gflags -DBUILD_SHARED_LIBS=ON -DBUILD_STATIC_LIBS=ON -DBUILD_gflags_LIB=ON -DLIB_SUFFIX=64 } function install_glog { wget_and_untar https://github.com/google/glog/archive/v0.6.0.tar.gz glog - ( - cd glog - cmake_install -DBUILD_SHARED_LIBS=ON - ) + cmake_install glog -DBUILD_SHARED_LIBS=ON } function install_lzo { @@ -108,7 +106,7 @@ function install_lzo { } function install_boost { - wget_and_untar https://github.com/boostorg/boost/releases/download/boost-1.84.0/boost-1.84.0.tar.gz boost + wget_and_untar https://github.com/boostorg/boost/releases/download/${BOOST_VERSION}/${BOOST_VERSION}.tar.gz boost ( cd boost ./bootstrap.sh --prefix=/usr/local @@ -118,18 +116,12 @@ function install_boost { function install_snappy { wget_and_untar https://github.com/google/snappy/archive/1.1.8.tar.gz snappy - ( - cd snappy - cmake_install -DSNAPPY_BUILD_TESTS=OFF - ) + cmake_install snappy -DSNAPPY_BUILD_TESTS=OFF } function install_fmt { - wget_and_untar https://github.com/fmtlib/fmt/archive/10.1.1.tar.gz fmt - ( - cd fmt - cmake_install -DFMT_TEST=OFF - ) + wget_and_untar https://github.com/fmtlib/fmt/archive/${FMT_VERSION}.tar.gz fmt + cmake_install fmt -DFMT_TEST=OFF } function install_protobuf { @@ -143,56 +135,36 @@ function install_protobuf { ) } -FB_OS_VERSION="v2024.05.20.00" - function install_fizz { wget_and_untar https://github.com/facebookincubator/fizz/archive/refs/tags/${FB_OS_VERSION}.tar.gz fizz - ( - cd fizz/fizz - cmake_install -DBUILD_TESTS=OFF - ) + cmake_install fizz/fizz -DBUILD_TESTS=OFF } function install_folly { wget_and_untar https://github.com/facebook/folly/archive/refs/tags/${FB_OS_VERSION}.tar.gz folly - ( - cd folly - cmake_install -DFOLLY_HAVE_INT128_T=ON - ) + cmake_install folly -DFOLLY_HAVE_INT128_T=ON } function install_wangle { wget_and_untar https://github.com/facebook/wangle/archive/refs/tags/${FB_OS_VERSION}.tar.gz wangle - ( - cd wangle/wangle - cmake_install -DBUILD_TESTS=OFF - ) + cmake_install wangle/wangle -DBUILD_TESTS=OFF } function install_fbthrift { wget_and_untar https://github.com/facebook/fbthrift/archive/refs/tags/${FB_OS_VERSION}.tar.gz fbthrift - ( - cd fbthrift - cmake_install -Denable_tests=OFF -DBUILD_SHARED_LIBS=OFF -DBUILD_TESTS=OFF -DBUILD_SHARED_LIBS=OFF - ) + cmake_install fbthrift -Denable_tests=OFF -DBUILD_SHARED_LIBS=OFF -DBUILD_TESTS=OFF -DBUILD_SHARED_LIBS=OFF } function install_mvfst { wget_and_untar https://github.com/facebook/mvfst/archive/refs/tags/${FB_OS_VERSION}.tar.gz mvfst - ( - cd mvfst - cmake_install -DBUILD_TESTS=OFF - ) + cmake_install mvfst -DBUILD_TESTS=OFF } function install_duckdb { if $BUILD_DUCKDB ; then echo 'Building DuckDB' wget_and_untar https://github.com/duckdb/duckdb/archive/refs/tags/v0.8.1.tar.gz duckdb - ( - cd duckdb - cmake_install -DBUILD_UNITTESTS=OFF -DENABLE_SANITIZER=OFF -DENABLE_UBSAN=OFF -DBUILD_SHELL=OFF -DEXPORT_DLL_SYMBOLS=OFF -DCMAKE_BUILD_TYPE=Release - ) + cmake_install duckdb -DBUILD_UNITTESTS=OFF -DENABLE_SANITIZER=OFF -DENABLE_UBSAN=OFF -DBUILD_SHELL=OFF -DEXPORT_DLL_SYMBOLS=OFF -DCMAKE_BUILD_TYPE=Release fi } From 4ae223cc7f3151aafb2f771ae01ad28c26013885 Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Tue, 30 Jul 2024 08:25:28 +0800 Subject: [PATCH 18/61] [VL] Daily Update Velox Version (2024_07_30) (#6626) --- ep/build-velox/src/get_velox.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index d9b22482d399b..774ec07886337 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_07_29 +VELOX_BRANCH=2024_07_30 VELOX_HOME="" OS=`uname -s` From 73fd854e25e84c0c6258e82517f657705516fc88 Mon Sep 17 00:00:00 2001 From: lgbo Date: Tue, 30 Jul 2024 10:03:07 +0800 Subject: [PATCH 19/61] fix a bug in serializing aggregating keys which are complicated types (#6624) --- .../GlutenClickHouseTPCHSaltNullParquetSuite.scala | 12 ++++++++++++ cpp-ch/clickhouse.version | 2 +- 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSaltNullParquetSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSaltNullParquetSuite.scala index 6d5c9c4635dff..694a9f253becb 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSaltNullParquetSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSaltNullParquetSuite.scala @@ -2781,5 +2781,17 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr compareResultsAgainstVanillaSpark(sql, true, { _ => }) } } + + test("GLUTEN-6583 serializing bug in aggregating with nullable compilated type keys") { + val sql = """ + |select n_regionkey, x, count(1) from ( + | select n_regionkey, if(n_regionkey = 'xx', null, x) as x from ( + | select n_regionkey, array(n_name, if(n_name != 'KENYA', n_name, null)) as x + | from nation + | ) + |) group by n_regionkey, x; + |""".stripMargin + compareResultsAgainstVanillaSpark(sql, true, { _ => }) + } } // scalastyle:on line.size.limit diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version index a67f0227c0eb3..9284537ad1e75 100644 --- a/cpp-ch/clickhouse.version +++ b/cpp-ch/clickhouse.version @@ -1,4 +1,4 @@ CH_ORG=Kyligence CH_BRANCH=rebase_ch/20240727 -CH_COMMIT=7d0829e25b9 +CH_COMMIT=d09605082e3 From c0d633dd57423d84634b38bcbf9248241d8f5f8a Mon Sep 17 00:00:00 2001 From: Rong Ma Date: Tue, 30 Jul 2024 12:44:01 +0800 Subject: [PATCH 20/61] [VL] Row-based sort shuffle follow-up (minor) (#6628) --- cpp/velox/shuffle/RadixSort.h | 10 +- cpp/velox/shuffle/VeloxSortShuffleWriter.cc | 7 +- docs/Configuration.md | 120 ++++++++++---------- 3 files changed, 68 insertions(+), 69 deletions(-) diff --git a/cpp/velox/shuffle/RadixSort.h b/cpp/velox/shuffle/RadixSort.h index 17f05d349c8e2..90921962a17f9 100644 --- a/cpp/velox/shuffle/RadixSort.h +++ b/cpp/velox/shuffle/RadixSort.h @@ -21,7 +21,9 @@ namespace gluten { -template +// Spark radix sort implementation. This implementation is for shuffle sort only as it removes unused +// params (desc, signed) in shuffle. +// https://github.com/apache/spark/blob/308669fc301916837bacb7c3ec1ecef93190c094/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RadixSort.java#L25 class RadixSort { public: /** @@ -39,7 +41,7 @@ class RadixSort { * @return The starting index of the sorted data within the given array. We return this instead * of always copying the data back to position zero for efficiency. */ - static int32_t sort(Element* array, size_t size, int64_t numRecords, int32_t startByteIndex, int32_t endByteIndex) { + static int32_t sort(uint64_t* array, size_t size, int64_t numRecords, int32_t startByteIndex, int32_t endByteIndex) { assert(startByteIndex >= 0 && "startByteIndex should >= 0"); assert(endByteIndex <= 7 && "endByteIndex should <= 7"); assert(endByteIndex > startByteIndex); @@ -75,7 +77,7 @@ class RadixSort { * @param outIndex the starting index where sorted output data should be written. */ static void sortAtByte( - Element* array, + uint64_t* array, int64_t numRecords, std::vector& counts, int32_t byteIdx, @@ -103,7 +105,7 @@ class RadixSort { * significant byte. If the byte does not need sorting the vector entry will be empty. */ static std::vector> - getCounts(Element* array, int64_t numRecords, int32_t startByteIndex, int32_t endByteIndex) { + getCounts(uint64_t* array, int64_t numRecords, int32_t startByteIndex, int32_t endByteIndex) { std::vector> counts; counts.resize(8); diff --git a/cpp/velox/shuffle/VeloxSortShuffleWriter.cc b/cpp/velox/shuffle/VeloxSortShuffleWriter.cc index 0015ba9d36c3c..d7db69659d253 100644 --- a/cpp/velox/shuffle/VeloxSortShuffleWriter.cc +++ b/cpp/velox/shuffle/VeloxSortShuffleWriter.cc @@ -220,12 +220,9 @@ arrow::Status VeloxSortShuffleWriter::evictAllPartitions() { { ScopedTimer timer(&sortTime_); if (options_.useRadixSort) { - begin = RadixSort::sort( - arrayPtr_, arraySize_, numRecords, kPartitionIdStartByteIndex, kPartitionIdEndByteIndex); + begin = RadixSort::sort(arrayPtr_, arraySize_, numRecords, kPartitionIdStartByteIndex, kPartitionIdEndByteIndex); } else { - auto ptr = arrayPtr_; - qsort(ptr, numRecords, sizeof(uint64_t), compare); - (void)ptr; + std::sort(arrayPtr_, arrayPtr_ + numRecords); } } diff --git a/docs/Configuration.md b/docs/Configuration.md index 2c2bd4de11f24..4e47564a67119 100644 --- a/docs/Configuration.md +++ b/docs/Configuration.md @@ -11,65 +11,66 @@ You can add these configurations into spark-defaults.conf to enable or disable t ## Spark parameters -| Parameters | Description | Recommend Setting | -|------------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|------------------------------------------------------| -| spark.driver.extraClassPath | To add Gluten Plugin jar file in Spark Driver | /path/to/jar_file | -| spark.executor.extraClassPath | To add Gluten Plugin jar file in Spark Executor | /path/to/jar_file | -| spark.executor.memory | To set up how much memory to be used for Spark Executor. | | -| spark.memory.offHeap.size | To set up how much memory to be used for Java OffHeap.
Please notice Gluten Plugin will leverage this setting to allocate memory space for native usage even offHeap is disabled.
The value is based on your system and it is recommended to set it larger if you are facing Out of Memory issue in Gluten Plugin | 30G | -| spark.sql.sources.useV1SourceList | Choose to use V1 source | avro | -| spark.sql.join.preferSortMergeJoin | To turn off preferSortMergeJoin in Spark | false | -| spark.plugins | To load Gluten's components by Spark's plug-in loader | org.apache.gluten.GlutenPlugin | -| spark.shuffle.manager | To turn on Gluten Columnar Shuffle Plugin | org.apache.spark.shuffle.sort.ColumnarShuffleManager | -| spark.gluten.enabled | Enable Gluten, default is true. Just an experimental property. Recommend to enable/disable Gluten through the setting for `spark.plugins`. | true | -| spark.gluten.memory.isolation | (Experimental) Enable isolated memory mode. If true, Gluten controls the maximum off-heap memory can be used by each task to X, X = executor memory / max task slots. It's recommended to set true if Gluten serves concurrent queries within a single session, since not all memory Gluten allocated is guaranteed to be spillable. In the case, the feature should be enabled to avoid OOM. Note when true, setting spark.memory.storageFraction to a lower value is suggested since storage memory is considered non-usable by Gluten. | false | -| spark.gluten.ras.enabled | Experimental: Enables RAS (relation algebra selector) during physical planning to generate more efficient query plan. Note, this feature is still in development and may not bring performance profits. | false | -| spark.gluten.sql.columnar.maxBatchSize | Number of rows to be processed in each batch. Default value is 4096. | 4096 | -| spark.gluten.sql.columnar.scanOnly | When enabled, this config will overwrite all other operators' enabling, and only Scan and Filter pushdown will be offloaded to native. | false | -| spark.gluten.sql.columnar.batchscan | Enable or Disable Columnar BatchScan, default is true | true | -| spark.gluten.sql.columnar.hashagg | Enable or Disable Columnar Hash Aggregate, default is true | true | -| spark.gluten.sql.columnar.project | Enable or Disable Columnar Project, default is true | true | -| spark.gluten.sql.columnar.filter | Enable or Disable Columnar Filter, default is true | true | -| spark.gluten.sql.columnar.sort | Enable or Disable Columnar Sort, default is true | true | -| spark.gluten.sql.columnar.window | Enable or Disable Columnar Window, default is true | true | -| spark.gluten.sql.columnar.shuffledHashJoin | Enable or Disable ShuffledHashJoin, default is true | true | -| spark.gluten.sql.columnar.forceShuffledHashJoin | Force to use ShuffledHashJoin over SortMergeJoin, default is true. For queries that can benefit from storaged patitioned join, please set it to false. | true | -| spark.gluten.sql.columnar.sortMergeJoin | Enable or Disable Columnar Sort Merge Join, default is true | true | -| spark.gluten.sql.columnar.union | Enable or Disable Columnar Union, default is true | true | -| spark.gluten.sql.columnar.expand | Enable or Disable Columnar Expand, default is true | true | -| spark.gluten.sql.columnar.generate | Enable or Disable Columnar Generate, default is true | true | -| spark.gluten.sql.columnar.limit | Enable or Disable Columnar Limit, default is true | true | -| spark.gluten.sql.columnar.tableCache | Enable or Disable Columnar Table Cache, default is false | true | -| spark.gluten.sql.columnar.broadcastExchange | Enable or Disable Columnar Broadcast Exchange, default is true | true | -| spark.gluten.sql.columnar.broadcastJoin | Enable or Disable Columnar BroadcastHashJoin, default is true | true | -| spark.gluten.sql.columnar.shuffle.sort.threshold | The threshold to determine whether to use sort-based columnar shuffle. Sort-based shuffle will be used if the number of partitions is greater than this threshold. | 100000 | -| spark.gluten.sql.columnar.shuffle.codec | Set up the codec to be used for Columnar Shuffle. If this configuration is not set, will check the value of spark.io.compression.codec. By default, Gluten use software compression. Valid options for software compression are lz4, zstd. Valid options for QAT and IAA is gzip. | lz4 | -| spark.gluten.sql.columnar.shuffle.codecBackend | Enable using hardware accelerators for shuffle de/compression. Valid options are QAT and IAA. | | -| spark.gluten.sql.columnar.shuffle.compressionMode | Setting different compression mode in shuffle, Valid options are buffer and rowvector, buffer option compress each buffer of RowVector individually into one pre-allocated large buffer, rowvector option first copies each buffer of RowVector to a large buffer and then compress the entire buffer in one go. | buffer | -| spark.gluten.sql.columnar.shuffle.compression.threshold | If number of rows in a batch falls below this threshold, will copy all buffers into one buffer to compress. | 100 | -| spark.gluten.sql.columnar.shuffle.realloc.threshold | Set the threshold to dynamically adjust the size of shuffle split buffers. The size of each split buffer is recalculated for each incoming batch of data. If the new size deviates from the current partition buffer size by a factor outside the range of [1 - threshold, 1 + threshold], the split buffer will be re-allocated using the newly calculated size | 0.25 | -| spark.gluten.sql.columnar.shuffle.merge.threshold | Set the threshold control the minimum merged size. When a partition buffer is full, and the number of rows is below (`threshold * spark.gluten.sql.columnar.maxBatchSize`), it will be saved for merging. | 0.25 | -| spark.gluten.sql.columnar.numaBinding | Set up NUMABinding, default is false | true | -| spark.gluten.sql.columnar.coreRange | Set up the core range for NUMABinding, only works when numaBinding set to true.
The setting is based on the number of cores in your system. Use 72 cores as an example. | 0-17,36-53 |18-35,54-71 | -| spark.gluten.sql.columnar.wholeStage.fallback.threshold | Configure the threshold for whether whole stage will fall back in AQE supported case by counting the number of ColumnarToRow & vanilla leaf node | \>= 1 | -| spark.gluten.sql.columnar.query.fallback.threshold | Configure the threshold for whether query will fall back by counting the number of ColumnarToRow & vanilla leaf node | \>= 1 | -| spark.gluten.sql.columnar.fallback.ignoreRowToColumnar | When true, the fallback policy ignores the RowToColumnar when counting fallback number. | true | -| spark.gluten.sql.columnar.fallback.preferColumnar | When true, the fallback policy prefers to use Gluten plan rather than vanilla Spark plan if the both of them contains ColumnarToRow and the vanilla Spark plan ColumnarToRow number is not smaller than Gluten plan. | true | -| spark.gluten.sql.columnar.force.hashagg | Force to use hash agg to replace sort agg. | true | -| spark.gluten.sql.columnar.vanillaReaders | Enable vanilla spark's vectorized reader. Please note it may bring perf. overhead due to extra data transition. We recommend to disable it if most queries can be fully offloaded to gluten. | false | -| spark.gluten.sql.native.bloomFilter | Enable or Disable native runtime bloom filter. | true | -| spark.gluten.sql.native.arrow.reader.enabled | Enable or Disable native arrow read CSV file format | false | -| spark.gluten.shuffleWriter.bufferSize | Set the number of buffer rows for the shuffle writer | value of spark.gluten.sql.columnar.maxBatchSize | -| spark.gluten.loadLibFromJar | Controls whether to load dynamic link library from a packed jar for gluten/cpp. Not applicable to static build and clickhouse backend. | false | -| spark.gluten.loadLibOS | When `spark.gluten.loadLibFromJar` is true. Manually specify the system os to load library, e.g., CentOS | | -| spark.gluten.loadLibOSVersion | Manually specify the system os version to load library, e.g., if `spark.gluten.loadLibOS` is CentOS, this config can be 7 | | -| spark.gluten.expression.blacklist | A black list of expression to skip transform, multiple values separated by commas. | | -| spark.gluten.sql.columnar.fallback.expressions.threshold | Fall back filter/project if the height of expression tree reaches this threshold, considering Spark codegen can bring better performance for such case. | 50 | -| spark.gluten.sql.cartesianProductTransformerEnabled | Config to enable CartesianProductExecTransformer. | true | -| spark.gluten.sql.broadcastNestedLoopJoinTransformerEnabled | Config to enable BroadcastNestedLoopJoinExecTransformer. | true | -| spark.gluten.sql.cacheWholeStageTransformerContext | When true, `WholeStageTransformer` will cache the `WholeStageTransformerContext` when executing. It is used to get substrait plan node and native plan string. | false | -| spark.gluten.sql.injectNativePlanStringToExplain | When true, Gluten will inject native plan tree to explain string inside `WholeStageTransformerContext`. | false | -| spark.gluten.sql.fallbackRegexpExpressions | When true, Gluten will fall back all regexp expressions to avoid any incompatibility risk. | false | +| Parameters | Description | Recommend Setting | +|-------------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|------------------------------------------------------| +| spark.driver.extraClassPath | To add Gluten Plugin jar file in Spark Driver | /path/to/jar_file | +| spark.executor.extraClassPath | To add Gluten Plugin jar file in Spark Executor | /path/to/jar_file | +| spark.executor.memory | To set up how much memory to be used for Spark Executor. | | +| spark.memory.offHeap.size | To set up how much memory to be used for Java OffHeap.
Please notice Gluten Plugin will leverage this setting to allocate memory space for native usage even offHeap is disabled.
The value is based on your system and it is recommended to set it larger if you are facing Out of Memory issue in Gluten Plugin | 30G | +| spark.sql.sources.useV1SourceList | Choose to use V1 source | avro | +| spark.sql.join.preferSortMergeJoin | To turn off preferSortMergeJoin in Spark | false | +| spark.plugins | To load Gluten's components by Spark's plug-in loader | org.apache.gluten.GlutenPlugin | +| spark.shuffle.manager | To turn on Gluten Columnar Shuffle Plugin | org.apache.spark.shuffle.sort.ColumnarShuffleManager | +| spark.gluten.enabled | Enable Gluten, default is true. Just an experimental property. Recommend to enable/disable Gluten through the setting for `spark.plugins`. | true | +| spark.gluten.memory.isolation | (Experimental) Enable isolated memory mode. If true, Gluten controls the maximum off-heap memory can be used by each task to X, X = executor memory / max task slots. It's recommended to set true if Gluten serves concurrent queries within a single session, since not all memory Gluten allocated is guaranteed to be spillable. In the case, the feature should be enabled to avoid OOM. Note when true, setting spark.memory.storageFraction to a lower value is suggested since storage memory is considered non-usable by Gluten. | false | +| spark.gluten.ras.enabled | Experimental: Enables RAS (relation algebra selector) during physical planning to generate more efficient query plan. Note, this feature is still in development and may not bring performance profits. | false | +| spark.gluten.sql.columnar.maxBatchSize | Number of rows to be processed in each batch. Default value is 4096. | 4096 | +| spark.gluten.sql.columnar.scanOnly | When enabled, this config will overwrite all other operators' enabling, and only Scan and Filter pushdown will be offloaded to native. | false | +| spark.gluten.sql.columnar.batchscan | Enable or Disable Columnar BatchScan, default is true | true | +| spark.gluten.sql.columnar.hashagg | Enable or Disable Columnar Hash Aggregate, default is true | true | +| spark.gluten.sql.columnar.project | Enable or Disable Columnar Project, default is true | true | +| spark.gluten.sql.columnar.filter | Enable or Disable Columnar Filter, default is true | true | +| spark.gluten.sql.columnar.sort | Enable or Disable Columnar Sort, default is true | true | +| spark.gluten.sql.columnar.window | Enable or Disable Columnar Window, default is true | true | +| spark.gluten.sql.columnar.shuffledHashJoin | Enable or Disable ShuffledHashJoin, default is true | true | +| spark.gluten.sql.columnar.forceShuffledHashJoin | Force to use ShuffledHashJoin over SortMergeJoin, default is true. For queries that can benefit from storaged patitioned join, please set it to false. | true | +| spark.gluten.sql.columnar.sortMergeJoin | Enable or Disable Columnar Sort Merge Join, default is true | true | +| spark.gluten.sql.columnar.union | Enable or Disable Columnar Union, default is true | true | +| spark.gluten.sql.columnar.expand | Enable or Disable Columnar Expand, default is true | true | +| spark.gluten.sql.columnar.generate | Enable or Disable Columnar Generate, default is true | true | +| spark.gluten.sql.columnar.limit | Enable or Disable Columnar Limit, default is true | true | +| spark.gluten.sql.columnar.tableCache | Enable or Disable Columnar Table Cache, default is false | true | +| spark.gluten.sql.columnar.broadcastExchange | Enable or Disable Columnar Broadcast Exchange, default is true | true | +| spark.gluten.sql.columnar.broadcastJoin | Enable or Disable Columnar BroadcastHashJoin, default is true | true | +| spark.gluten.sql.columnar.shuffle.sort.partitions.threshold | The threshold to determine whether to use sort-based columnar shuffle. Sort-based shuffle will be used if the number of partitions is greater than this threshold. | 100000 | +| spark.gluten.sql.columnar.shuffle.sort.columns.threshold | The threshold to determine whether to use sort-based columnar shuffle. Sort-based shuffle will be used if the number of columns is greater than this threshold. | 100000 | +| spark.gluten.sql.columnar.shuffle.codec | Set up the codec to be used for Columnar Shuffle. If this configuration is not set, will check the value of spark.io.compression.codec. By default, Gluten use software compression. Valid options for software compression are lz4, zstd. Valid options for QAT and IAA is gzip. | lz4 | +| spark.gluten.sql.columnar.shuffle.codecBackend | Enable using hardware accelerators for shuffle de/compression. Valid options are QAT and IAA. | | +| spark.gluten.sql.columnar.shuffle.compressionMode | Setting different compression mode in shuffle, Valid options are buffer and rowvector, buffer option compress each buffer of RowVector individually into one pre-allocated large buffer, rowvector option first copies each buffer of RowVector to a large buffer and then compress the entire buffer in one go. | buffer | +| spark.gluten.sql.columnar.shuffle.compression.threshold | If number of rows in a batch falls below this threshold, will copy all buffers into one buffer to compress. | 100 | +| spark.gluten.sql.columnar.shuffle.realloc.threshold | Set the threshold to dynamically adjust the size of shuffle split buffers. The size of each split buffer is recalculated for each incoming batch of data. If the new size deviates from the current partition buffer size by a factor outside the range of [1 - threshold, 1 + threshold], the split buffer will be re-allocated using the newly calculated size | 0.25 | +| spark.gluten.sql.columnar.shuffle.merge.threshold | Set the threshold control the minimum merged size. When a partition buffer is full, and the number of rows is below (`threshold * spark.gluten.sql.columnar.maxBatchSize`), it will be saved for merging. | 0.25 | +| spark.gluten.sql.columnar.numaBinding | Set up NUMABinding, default is false | true | +| spark.gluten.sql.columnar.coreRange | Set up the core range for NUMABinding, only works when numaBinding set to true.
The setting is based on the number of cores in your system. Use 72 cores as an example. | 0-17,36-53 |18-35,54-71 | +| spark.gluten.sql.columnar.wholeStage.fallback.threshold | Configure the threshold for whether whole stage will fall back in AQE supported case by counting the number of ColumnarToRow & vanilla leaf node | \>= 1 | +| spark.gluten.sql.columnar.query.fallback.threshold | Configure the threshold for whether query will fall back by counting the number of ColumnarToRow & vanilla leaf node | \>= 1 | +| spark.gluten.sql.columnar.fallback.ignoreRowToColumnar | When true, the fallback policy ignores the RowToColumnar when counting fallback number. | true | +| spark.gluten.sql.columnar.fallback.preferColumnar | When true, the fallback policy prefers to use Gluten plan rather than vanilla Spark plan if the both of them contains ColumnarToRow and the vanilla Spark plan ColumnarToRow number is not smaller than Gluten plan. | true | +| spark.gluten.sql.columnar.force.hashagg | Force to use hash agg to replace sort agg. | true | +| spark.gluten.sql.columnar.vanillaReaders | Enable vanilla spark's vectorized reader. Please note it may bring perf. overhead due to extra data transition. We recommend to disable it if most queries can be fully offloaded to gluten. | false | +| spark.gluten.sql.native.bloomFilter | Enable or Disable native runtime bloom filter. | true | +| spark.gluten.sql.native.arrow.reader.enabled | Enable or Disable native arrow read CSV file format | false | +| spark.gluten.shuffleWriter.bufferSize | Set the number of buffer rows for the shuffle writer | value of spark.gluten.sql.columnar.maxBatchSize | +| spark.gluten.loadLibFromJar | Controls whether to load dynamic link library from a packed jar for gluten/cpp. Not applicable to static build and clickhouse backend. | false | +| spark.gluten.loadLibOS | When `spark.gluten.loadLibFromJar` is true. Manually specify the system os to load library, e.g., CentOS | | +| spark.gluten.loadLibOSVersion | Manually specify the system os version to load library, e.g., if `spark.gluten.loadLibOS` is CentOS, this config can be 7 | | +| spark.gluten.expression.blacklist | A black list of expression to skip transform, multiple values separated by commas. | | +| spark.gluten.sql.columnar.fallback.expressions.threshold | Fall back filter/project if the height of expression tree reaches this threshold, considering Spark codegen can bring better performance for such case. | 50 | +| spark.gluten.sql.cartesianProductTransformerEnabled | Config to enable CartesianProductExecTransformer. | true | +| spark.gluten.sql.broadcastNestedLoopJoinTransformerEnabled | Config to enable BroadcastNestedLoopJoinExecTransformer. | true | +| spark.gluten.sql.cacheWholeStageTransformerContext | When true, `WholeStageTransformer` will cache the `WholeStageTransformerContext` when executing. It is used to get substrait plan node and native plan string. | false | +| spark.gluten.sql.injectNativePlanStringToExplain | When true, Gluten will inject native plan tree to explain string inside `WholeStageTransformerContext`. | false | +| spark.gluten.sql.fallbackRegexpExpressions | When true, Gluten will fall back all regexp expressions to avoid any incompatibility risk. | false | ## Velox Parameters @@ -94,7 +95,6 @@ The following configurations are related to Velox settings. | spark.gluten.sql.columnar.backend.velox.orc.scan.enabled | Enable velox orc scan. If disabled, vanilla spark orc scan will be used. | true | | spark.gluten.sql.complexType.scan.fallback.enabled | Force fallback for complex type scan, including struct, map, array. | true | - Additionally, you can control the configurations of gluten at thread level by local property. | Parameters | Description | Recommend Setting | From 20c8db80580bf54fa30982601f65d84e471ff8c5 Mon Sep 17 00:00:00 2001 From: Zhen Wang <643348094@qq.com> Date: Tue, 30 Jul 2024 13:21:51 +0800 Subject: [PATCH 21/61] [MINOR] Reduce unnecessary dependencies (#6608) --- gluten-data/pom.xml | 4 ++++ pom.xml | 6 ++++++ 2 files changed, 10 insertions(+) diff --git a/gluten-data/pom.xml b/gluten-data/pom.xml index c28490d77faad..a529eb9b9956e 100644 --- a/gluten-data/pom.xml +++ b/gluten-data/pom.xml @@ -119,6 +119,10 @@ io.netty netty-buffer + + commons-codec + commons-codec + diff --git a/pom.xml b/pom.xml index 4f8bd3e14f61f..81c355868159e 100644 --- a/pom.xml +++ b/pom.xml @@ -714,6 +714,12 @@ ${fasterxml.version} provided + + com.fasterxml.jackson.datatype + jackson-datatype-jsr310 + ${fasterxml.version} + provided + com.fasterxml.jackson.datatype jackson-datatype-guava From b5657d3190db75f07f4480b82bc7bc325480cdf8 Mon Sep 17 00:00:00 2001 From: Zhen Li <10524738+zhli1142015@users.noreply.github.com> Date: Tue, 30 Jul 2024 13:46:06 +0800 Subject: [PATCH 22/61] [VL] Enable collect_set, min, max for complex types (#6629) [VL] Enable collect_set, min, max for complex types. --- .../execution/VeloxAggregateFunctionsSuite.scala | 12 ++++-------- cpp/velox/substrait/SubstraitParser.cc | 1 - cpp/velox/substrait/SubstraitToVeloxPlanValidator.cc | 9 --------- 3 files changed, 4 insertions(+), 18 deletions(-) diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxAggregateFunctionsSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxAggregateFunctionsSuite.scala index 270334011da7b..fb1f0542639f7 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxAggregateFunctionsSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxAggregateFunctionsSuite.scala @@ -22,7 +22,6 @@ import org.apache.gluten.extension.columnar.validator.FallbackInjects import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.expressions.aggregate.{Final, Partial} import org.apache.spark.sql.execution.aggregate.BaseAggregateExec -import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -1141,13 +1140,10 @@ abstract class VeloxAggregateFunctionsSuite extends VeloxWholeStageTransformerSu StructField("lastUpdated", LongType, true), StructField("version", LongType, true))), true))) - val df = spark.read.schema(jsonSchema).json(Seq(jsonStr).toDS) - df.select(collect_set(col("txn"))).collect - - df.select(min(col("txn"))).collect - - df.select(max(col("txn"))).collect - + spark.read.schema(jsonSchema).json(Seq(jsonStr).toDS).createOrReplaceTempView("t1") + runQueryAndCompare("select collect_set(txn), min(txn), max(txn) from t1") { + checkGlutenOperatorMatch[HashAggregateExecTransformer] + } } test("drop redundant partial sort which has pre-project when offload sortAgg") { diff --git a/cpp/velox/substrait/SubstraitParser.cc b/cpp/velox/substrait/SubstraitParser.cc index b842914ca9334..6eb62f854e0d0 100644 --- a/cpp/velox/substrait/SubstraitParser.cc +++ b/cpp/velox/substrait/SubstraitParser.cc @@ -395,7 +395,6 @@ std::unordered_map SubstraitParser::substraitVeloxFunc {"xxhash64", "xxhash64_with_seed"}, {"modulus", "remainder"}, {"date_format", "format_datetime"}, - {"collect_set", "set_agg"}, {"negative", "unaryminus"}, {"get_array_item", "get"}}; diff --git a/cpp/velox/substrait/SubstraitToVeloxPlanValidator.cc b/cpp/velox/substrait/SubstraitToVeloxPlanValidator.cc index c229ca84fe190..c18d265986f8c 100644 --- a/cpp/velox/substrait/SubstraitToVeloxPlanValidator.cc +++ b/cpp/velox/substrait/SubstraitToVeloxPlanValidator.cc @@ -1048,15 +1048,6 @@ bool SubstraitToVeloxPlanValidator::validateAggRelFunctionType(const ::substrait LOG_VALIDATION_MSG("Validation failed for function " + funcName + " resolve type in AggregateRel."); return false; } - static const std::unordered_set notSupportComplexTypeAggFuncs = {"set_agg", "min", "max"}; - if (notSupportComplexTypeAggFuncs.find(baseFuncName) != notSupportComplexTypeAggFuncs.end() && - exec::isRawInput(funcStep)) { - auto type = binder.tryResolveType(signature->argumentTypes()[0]); - if (type->isArray() || type->isMap() || type->isRow()) { - LOG_VALIDATION_MSG("Validation failed for function " + baseFuncName + " complex type is not supported."); - return false; - } - } resolved = true; break; From 295899cd108c56dda436730d58dd716d9d8b0b28 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=AB=98=E9=98=B3=E9=98=B3?= Date: Tue, 30 Jul 2024 15:31:41 +0800 Subject: [PATCH 23/61] [VL] Enable Spark mask function (#6271) --- .../ScalarFunctionsValidateSuite.scala | 18 ++++++++++++++++++ .../gluten/expression/ExpressionNames.scala | 1 + .../sql/shims/spark34/Spark34Shims.scala | 3 ++- .../sql/shims/spark35/Spark35Shims.scala | 1 + 4 files changed, 22 insertions(+), 1 deletion(-) diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/ScalarFunctionsValidateSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/ScalarFunctionsValidateSuite.scala index 43d54fb62e4b3..13ade14b59438 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/ScalarFunctionsValidateSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/ScalarFunctionsValidateSuite.scala @@ -918,6 +918,24 @@ abstract class ScalarFunctionsValidateSuite extends FunctionsValidateTest { } } + testWithSpecifiedSparkVersion("mask", Some("3.4")) { + runQueryAndCompare("SELECT mask(c_comment) FROM customer limit 50") { + checkGlutenOperatorMatch[ProjectExecTransformer] + } + runQueryAndCompare("SELECT mask(c_comment, 'Y') FROM customer limit 50") { + checkGlutenOperatorMatch[ProjectExecTransformer] + } + runQueryAndCompare("SELECT mask(c_comment, 'Y', 'y') FROM customer limit 50") { + checkGlutenOperatorMatch[ProjectExecTransformer] + } + runQueryAndCompare("SELECT mask(c_comment, 'Y', 'y', 'o') FROM customer limit 50") { + checkGlutenOperatorMatch[ProjectExecTransformer] + } + runQueryAndCompare("SELECT mask(c_comment, 'Y', 'y', 'o', '*') FROM customer limit 50") { + checkGlutenOperatorMatch[ProjectExecTransformer] + } + } + test("bit_length") { runQueryAndCompare( "select bit_length(c_comment), bit_length(cast(c_comment as binary))" + diff --git a/shims/common/src/main/scala/org/apache/gluten/expression/ExpressionNames.scala b/shims/common/src/main/scala/org/apache/gluten/expression/ExpressionNames.scala index 7cc75405bad65..d47dbc4cc1fa8 100644 --- a/shims/common/src/main/scala/org/apache/gluten/expression/ExpressionNames.scala +++ b/shims/common/src/main/scala/org/apache/gluten/expression/ExpressionNames.scala @@ -131,6 +131,7 @@ object ExpressionNames { final val LEVENSHTEIN = "levenshtein" final val UNBASE64 = "unbase64" final val BASE64 = "base64" + final val MASK = "mask" // URL functions final val PARSE_URL = "parse_url" diff --git a/shims/spark34/src/main/scala/org/apache/gluten/sql/shims/spark34/Spark34Shims.scala b/shims/spark34/src/main/scala/org/apache/gluten/sql/shims/spark34/Spark34Shims.scala index 203256cf5fec2..cd7e4347d6b52 100644 --- a/shims/spark34/src/main/scala/org/apache/gluten/sql/shims/spark34/Spark34Shims.scala +++ b/shims/spark34/src/main/scala/org/apache/gluten/sql/shims/spark34/Spark34Shims.scala @@ -80,7 +80,8 @@ class Spark34Shims extends SparkShims { Sig[Empty2Null](ExpressionNames.EMPTY2NULL), Sig[TimestampAdd](ExpressionNames.TIMESTAMP_ADD), Sig[RoundFloor](ExpressionNames.FLOOR), - Sig[RoundCeil](ExpressionNames.CEIL) + Sig[RoundCeil](ExpressionNames.CEIL), + Sig[Mask](ExpressionNames.MASK) ) } diff --git a/shims/spark35/src/main/scala/org/apache/gluten/sql/shims/spark35/Spark35Shims.scala b/shims/spark35/src/main/scala/org/apache/gluten/sql/shims/spark35/Spark35Shims.scala index 821e0f5837d69..bb41b7e73ecad 100644 --- a/shims/spark35/src/main/scala/org/apache/gluten/sql/shims/spark35/Spark35Shims.scala +++ b/shims/spark35/src/main/scala/org/apache/gluten/sql/shims/spark35/Spark35Shims.scala @@ -78,6 +78,7 @@ class Spark35Shims extends SparkShims { Sig[Csc](ExpressionNames.CSC), Sig[KnownNullable](ExpressionNames.KNOWN_NULLABLE), Sig[Empty2Null](ExpressionNames.EMPTY2NULL), + Sig[Mask](ExpressionNames.MASK), Sig[TimestampAdd](ExpressionNames.TIMESTAMP_ADD), Sig[RoundFloor](ExpressionNames.FLOOR), Sig[RoundCeil](ExpressionNames.CEIL) From e8dd17276e593f882d39c6f5db27ee6ca403bfc1 Mon Sep 17 00:00:00 2001 From: LiuNeng <1398775315@qq.com> Date: Tue, 30 Jul 2024 16:41:16 +0800 Subject: [PATCH 24/61] [CH] Refactor off heap memory management, clean shuffle write code (#6558) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit What changes were proposed in this pull request? 将现有的通过spark管理内存并触发spill的机制改为由ClickHouse托管。shuffle spill触发由当前内存使用量和固定阈值决定。 修改原因: 统计不准确,代码运行过程中如果存在异步线程,容易出现内存使用量统计失真 无法统一管理各个算子的spill,目前已有sort ,aggregate,join,shuffle支持spill,spark task memory manager无法管理众多算子的spill触发 shuffle通过spark触发spill容易出错,因为内存分配会出现在任何时候,但不是任何时机都可以触发spill,容易导致运行错误。难以根本解决。 image 在task线程初始化时创建对应的ThreadGroup,将task线程 attach至thread group,使得在task线程创建的pipeline executor也会加入到当前的thread group。通过thread Group的内存统计来控制整个task所需的offheap内存。 在task结束时,清理相应的thread group,并更新spark统计信息. image 删除多余的配置项 spark.gluten.sql.columnar.backend.ch.shuffle.preferSpill 不再需要选择不同的模式,保留preferSpill=false的模式,对celeborn没有影响 spark.gluten.sql.columnar.backend.ch.throwIfMemoryExceed offheap内存管理切换至clickhouse MemoryTracker,不在需要该配置 spark.gluten.sql.columnar.backend.ch.flushBlockBufferBeforeEvict 保留flushBlockBufferBeforeEvict=true的行为,移除配置项 spark.gluten.sql.columnar.backend.ch.spillFirstlyBeforeStop spark.gluten.sql.columnar.backend.ch.forceExternalSortShuffle 移除冗余的external sort shuffle 及配置 新增配置 spark.gluten.sql.columnar.backend.ch.runtime_config.spill_mem_ratio=0.9 shuffle spill触发控制,默认当前task内存使用量超过90%时触发 spark.gluten.sql.columnar.backend.ch.runtime_config.extra_memory_hard_limit=0 为task内存增加额外的hard limit限制,单位为byte,当task内存使用超过hardlimit时会产生异常。如果希望task能适度超过offheap配置可增大该配置项 内存的hard_limit = soft_limit + extra_memory_hard_limit, soft_limit为spark配置的offheap size。 依然可以通过spark ui查看task的peak memory信息 image 新增performance counter 日志输出,方便对task进行性能分析。 image How was this patch tested? unit tests (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) --- .../OptimizeTableCommandOverwrites.scala | 6 +- .../OptimizeTableCommandOverwrites.scala | 6 +- .../OptimizeTableCommandOverwrites.scala | 6 +- .../apache/gluten/memory/CHThreadGroup.java | 77 ++++ .../alloc/CHManagedCHReservationListener.java | 152 ------- .../memory/alloc/CHNativeMemoryAllocator.java | 71 --- .../CHNativeMemoryAllocatorManagerImpl.java | 46 -- .../alloc/CHNativeMemoryAllocators.java | 100 ----- .../memory/alloc/CHReservationListener.java | 53 --- .../CHNativeExpressionEvaluator.java | 12 +- .../CHShuffleSplitterJniWrapper.java | 26 -- .../ExpressionEvaluatorJniWrapper.java | 6 +- .../datasources/CHDatasourceJniWrapper.java | 3 +- .../clickhouse/CHIteratorApi.scala | 3 + .../shuffle/CHColumnarShuffleWriter.scala | 35 +- .../v1/CHFormatWriterInjects.scala | 2 + .../v1/CHMergeTreeWriterInjects.scala | 5 +- .../MergeTreeFileFormatWriter.scala | 4 +- .../gluten/utils/TestExceptionUtils.java} | 9 +- .../memory/TestTaskMemoryManagerSuite.java | 115 ----- ...ouseColumnarExternalSortShuffleSuite.scala | 128 ------ ...GlutenClickHouseNativeExceptionSuite.scala | 8 +- ...sePreferSpillColumnarShuffleAQESuite.scala | 169 ------- .../GlutenClickHouseMetricsUTUtils.scala | 4 - .../GlutenClickHouseTPCDSMetricsSuite.scala | 60 +-- .../GlutenClickHouseTPCHMetricsSuite.scala | 309 ++++++------- cpp-ch/local-engine/Common/CHUtil.cpp | 13 +- cpp-ch/local-engine/Common/CHUtil.h | 2 - cpp-ch/local-engine/Common/GlutenConfig.h | 166 +++++++ cpp-ch/local-engine/Common/QueryContext.cpp | 167 +++++-- cpp-ch/local-engine/Common/QueryContext.h | 36 +- .../Join/BroadCastJoinBuilder.cpp | 18 +- .../Join/StorageJoinFromReadBuffer.cpp | 54 ++- .../Operator/GraceMergingAggregatedStep.cpp | 31 +- .../Operator/StreamingAggregatingStep.cpp | 24 +- .../Parser/AggregateRelParser.cpp | 13 +- .../Parser/SerializedPlanParser.cpp | 5 +- cpp-ch/local-engine/Parser/SortRelParser.cpp | 12 +- .../Shuffle/CachedShuffleWriter.cpp | 23 +- .../Shuffle/CachedShuffleWriter.h | 3 +- cpp-ch/local-engine/Shuffle/NativeSplitter.h | 2 +- .../local-engine/Shuffle/PartitionWriter.cpp | 372 ++++----------- cpp-ch/local-engine/Shuffle/PartitionWriter.h | 115 ++--- cpp-ch/local-engine/Shuffle/ShuffleCommon.cpp | 119 +++++ .../{ShuffleSplitter.h => ShuffleCommon.h} | 94 +--- .../local-engine/Shuffle/ShuffleSplitter.cpp | 424 ------------------ .../Shuffle/SortedPartitionDataMerger.cpp | 139 ------ .../Shuffle/SortedPartitionDataMerger.h | 93 ---- .../Mergetree/SparkMergeTreeWriter.cpp | 41 +- .../Storages/Mergetree/SparkMergeTreeWriter.h | 3 +- .../Storages/StorageMergeTreeFactory.cpp | 8 +- .../Storages/StorageMergeTreeFactory.h | 7 +- .../Storages/SubstraitSource/FormatFile.cpp | 5 +- .../SubstraitSource/ReadBufferBuilder.cpp | 14 +- .../jni/ReservationListenerWrapper.cpp | 75 ---- .../jni/ReservationListenerWrapper.h | 48 -- cpp-ch/local-engine/local_engine_jni.cpp | 110 ++--- docs/get-started/ClickHouse.md | 10 - .../CHCelebornColumnarShuffleWriter.scala | 25 -- ...eRSSColumnarExternalSortShuffleSuite.scala | 208 --------- .../org/apache/gluten/GlutenConfig.scala | 51 --- 61 files changed, 993 insertions(+), 2952 deletions(-) create mode 100644 backends-clickhouse/src/main/java/org/apache/gluten/memory/CHThreadGroup.java delete mode 100644 backends-clickhouse/src/main/java/org/apache/gluten/memory/alloc/CHManagedCHReservationListener.java delete mode 100644 backends-clickhouse/src/main/java/org/apache/gluten/memory/alloc/CHNativeMemoryAllocator.java delete mode 100644 backends-clickhouse/src/main/java/org/apache/gluten/memory/alloc/CHNativeMemoryAllocatorManagerImpl.java delete mode 100644 backends-clickhouse/src/main/java/org/apache/gluten/memory/alloc/CHNativeMemoryAllocators.java delete mode 100644 backends-clickhouse/src/main/java/org/apache/gluten/memory/alloc/CHReservationListener.java rename backends-clickhouse/src/{main/java/org/apache/gluten/memory/alloc/CHNativeMemoryAllocatorManager.java => test/java/org/apache/gluten/utils/TestExceptionUtils.java} (74%) delete mode 100644 backends-clickhouse/src/test/java/org/apache/spark/memory/TestTaskMemoryManagerSuite.java delete mode 100644 backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseColumnarExternalSortShuffleSuite.scala delete mode 100644 backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHousePreferSpillColumnarShuffleAQESuite.scala create mode 100644 cpp-ch/local-engine/Common/GlutenConfig.h create mode 100644 cpp-ch/local-engine/Shuffle/ShuffleCommon.cpp rename cpp-ch/local-engine/Shuffle/{ShuffleSplitter.h => ShuffleCommon.h} (60%) delete mode 100644 cpp-ch/local-engine/Shuffle/ShuffleSplitter.cpp delete mode 100644 cpp-ch/local-engine/Shuffle/SortedPartitionDataMerger.cpp delete mode 100644 cpp-ch/local-engine/Shuffle/SortedPartitionDataMerger.h delete mode 100644 cpp-ch/local-engine/jni/ReservationListenerWrapper.cpp delete mode 100644 cpp-ch/local-engine/jni/ReservationListenerWrapper.h delete mode 100644 gluten-celeborn/clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseRSSColumnarExternalSortShuffleSuite.scala diff --git a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/commands/OptimizeTableCommandOverwrites.scala b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/commands/OptimizeTableCommandOverwrites.scala index f7a180b6a239c..b897010d5bb38 100644 --- a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/commands/OptimizeTableCommandOverwrites.scala +++ b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/commands/OptimizeTableCommandOverwrites.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.delta.commands import org.apache.gluten.expression.ConverterUtils - +import org.apache.gluten.memory.CHThreadGroup import org.apache.spark.{TaskContext, TaskOutputFileAlreadyExistException} import org.apache.spark.internal.Logging import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage @@ -38,13 +38,11 @@ import org.apache.spark.sql.execution.datasources.v2.clickhouse.utils.CHDataSour import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType import org.apache.spark.util.{SerializableConfiguration, SystemClock, Utils} - import org.apache.hadoop.fs.{FileAlreadyExistsException, Path} import org.apache.hadoop.mapreduce.{TaskAttemptContext, TaskAttemptID, TaskID, TaskType} import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl import java.util.{Date, UUID} - import scala.collection.mutable.ArrayBuffer object OptimizeTableCommandOverwrites extends Logging { @@ -76,7 +74,7 @@ object OptimizeTableCommandOverwrites extends Logging { sparkPartitionId: Int, sparkAttemptNumber: Int ): MergeTreeWriteTaskResult = { - + CHThreadGroup.registerNewThreadGroup() val jobId = SparkHadoopWriterUtils.createJobID(new Date(description.jobIdInstant), sparkStageId) val taskId = new TaskID(jobId, TaskType.MAP, sparkPartitionId) val taskAttemptId = new TaskAttemptID(taskId, sparkAttemptNumber) diff --git a/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/commands/OptimizeTableCommandOverwrites.scala b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/commands/OptimizeTableCommandOverwrites.scala index f7a180b6a239c..b897010d5bb38 100644 --- a/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/commands/OptimizeTableCommandOverwrites.scala +++ b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/commands/OptimizeTableCommandOverwrites.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.delta.commands import org.apache.gluten.expression.ConverterUtils - +import org.apache.gluten.memory.CHThreadGroup import org.apache.spark.{TaskContext, TaskOutputFileAlreadyExistException} import org.apache.spark.internal.Logging import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage @@ -38,13 +38,11 @@ import org.apache.spark.sql.execution.datasources.v2.clickhouse.utils.CHDataSour import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType import org.apache.spark.util.{SerializableConfiguration, SystemClock, Utils} - import org.apache.hadoop.fs.{FileAlreadyExistsException, Path} import org.apache.hadoop.mapreduce.{TaskAttemptContext, TaskAttemptID, TaskID, TaskType} import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl import java.util.{Date, UUID} - import scala.collection.mutable.ArrayBuffer object OptimizeTableCommandOverwrites extends Logging { @@ -76,7 +74,7 @@ object OptimizeTableCommandOverwrites extends Logging { sparkPartitionId: Int, sparkAttemptNumber: Int ): MergeTreeWriteTaskResult = { - + CHThreadGroup.registerNewThreadGroup() val jobId = SparkHadoopWriterUtils.createJobID(new Date(description.jobIdInstant), sparkStageId) val taskId = new TaskID(jobId, TaskType.MAP, sparkPartitionId) val taskAttemptId = new TaskAttemptID(taskId, sparkAttemptNumber) diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/OptimizeTableCommandOverwrites.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/OptimizeTableCommandOverwrites.scala index 7b4c3231b8c31..ef30aaad2294f 100644 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/OptimizeTableCommandOverwrites.scala +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/OptimizeTableCommandOverwrites.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.delta.commands import org.apache.gluten.expression.ConverterUtils - +import org.apache.gluten.memory.CHThreadGroup import org.apache.spark.{TaskContext, TaskOutputFileAlreadyExistException} import org.apache.spark.internal.Logging import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage @@ -40,13 +40,11 @@ import org.apache.spark.sql.execution.datasources.v2.clickhouse.utils.CHDataSour import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType import org.apache.spark.util.{SerializableConfiguration, SystemClock, Utils} - import org.apache.hadoop.fs.{FileAlreadyExistsException, Path} import org.apache.hadoop.mapreduce.{TaskAttemptContext, TaskAttemptID, TaskID, TaskType} import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl import java.util.{Date, UUID} - import scala.collection.mutable.ArrayBuffer object OptimizeTableCommandOverwrites extends Logging { @@ -78,7 +76,7 @@ object OptimizeTableCommandOverwrites extends Logging { sparkPartitionId: Int, sparkAttemptNumber: Int ): MergeTreeWriteTaskResult = { - + CHThreadGroup.registerNewThreadGroup() val jobId = SparkHadoopWriterUtils.createJobID(new Date(description.jobIdInstant), sparkStageId) val taskId = new TaskID(jobId, TaskType.MAP, sparkPartitionId) val taskAttemptId = new TaskAttemptID(taskId, sparkAttemptNumber) diff --git a/backends-clickhouse/src/main/java/org/apache/gluten/memory/CHThreadGroup.java b/backends-clickhouse/src/main/java/org/apache/gluten/memory/CHThreadGroup.java new file mode 100644 index 0000000000000..a06c552a9f6ba --- /dev/null +++ b/backends-clickhouse/src/main/java/org/apache/gluten/memory/CHThreadGroup.java @@ -0,0 +1,77 @@ +/* + * 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.gluten.memory; + +import org.apache.spark.TaskContext; +import org.apache.spark.util.TaskResource; +import org.apache.spark.util.TaskResources; + +public class CHThreadGroup implements TaskResource { + + /** + * Register a new thread group for the current task. This method should be called at beginning of + * the task. + */ + public static void registerNewThreadGroup() { + if (TaskResources.isResourceRegistered(CHThreadGroup.class.getName())) return; + CHThreadGroup group = new CHThreadGroup(); + TaskResources.addResource(CHThreadGroup.class.getName(), group); + TaskContext.get() + .addTaskCompletionListener( + (context -> { + context.taskMetrics().incPeakExecutionMemory(group.getPeakMemory()); + })); + } + + private long thread_group_id = 0; + private long peak_memory = -1; + + private CHThreadGroup() { + thread_group_id = createThreadGroup(); + } + + public long getPeakMemory() { + if (peak_memory < 0) { + peak_memory = threadGroupPeakMemory(thread_group_id); + } + return peak_memory; + } + + @Override + public void release() throws Exception { + if (peak_memory < 0) { + peak_memory = threadGroupPeakMemory(thread_group_id); + } + releaseThreadGroup(thread_group_id); + } + + @Override + public int priority() { + return TaskResource.super.priority(); + } + + @Override + public String resourceName() { + return "CHThreadGroup"; + } + + private static native long createThreadGroup(); + + private static native long threadGroupPeakMemory(long id); + + private static native void releaseThreadGroup(long id); +} diff --git a/backends-clickhouse/src/main/java/org/apache/gluten/memory/alloc/CHManagedCHReservationListener.java b/backends-clickhouse/src/main/java/org/apache/gluten/memory/alloc/CHManagedCHReservationListener.java deleted file mode 100644 index 1f560a905858c..0000000000000 --- a/backends-clickhouse/src/main/java/org/apache/gluten/memory/alloc/CHManagedCHReservationListener.java +++ /dev/null @@ -1,152 +0,0 @@ -/* - * 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.gluten.memory.alloc; - -import org.apache.gluten.GlutenConfig; -import org.apache.gluten.memory.SimpleMemoryUsageRecorder; -import org.apache.gluten.memory.memtarget.MemoryTarget; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.concurrent.atomic.AtomicLong; - -public class CHManagedCHReservationListener implements CHReservationListener { - - private static final Logger LOG = LoggerFactory.getLogger(CHManagedCHReservationListener.class); - - private MemoryTarget target; - private final SimpleMemoryUsageRecorder usage; - private final boolean throwIfMemoryExceed = - GlutenConfig.getConf().chColumnarThrowIfMemoryExceed(); - private volatile boolean open = true; - - private final AtomicLong currentMemory = new AtomicLong(0L); - - public CHManagedCHReservationListener(MemoryTarget target, SimpleMemoryUsageRecorder usage) { - this.target = target; - this.usage = usage; - } - - @Override - public void reserveOrThrow(long size) { - if (!throwIfMemoryExceed) { - reserve(size); - return; - } - - synchronized (this) { - if (!open) { - return; - } - if (LOG.isDebugEnabled()) { - LOG.debug(String.format("reserve memory size from native: %d", size)); - } - long granted = target.borrow(size); - if (granted < size) { - target.repay(granted); - throw new UnsupportedOperationException( - "Not enough spark off-heap execution memory. " - + "Acquired: " - + size - + ", granted: " - + granted - + ". " - + "Try tweaking config option spark.memory.offHeap.size to " - + "get larger space to run this application. "); - } - currentMemory.addAndGet(size); - usage.inc(size); - } - } - - @Override - public long reserve(long size) { - synchronized (this) { - if (!open) { - return 0L; - } - if (LOG.isDebugEnabled()) { - LOG.debug(String.format("reserve memory (without exception) size from native: %d", size)); - } - long granted = target.borrow(size); - if (granted < size && (LOG.isWarnEnabled())) { - LOG.warn( - String.format( - "Not enough spark off-heap execution memory. " - + "Acquired: %d, granted: %d. Try tweaking config option " - + "spark.memory.offHeap.size to get larger space " - + "to run this application.", - size, granted)); - } - currentMemory.addAndGet(granted); - usage.inc(size); - return granted; - } - } - - @Override - public long unreserve(long size) { - synchronized (this) { - if (!open) { - return 0L; - } - long memoryToFree = size; - if ((currentMemory.get() - size) < 0L) { - if (LOG.isDebugEnabled()) { - LOG.debug( - String.format( - "The current used memory' %d will be less than 0(%d) after free %d", - currentMemory.get(), currentMemory.get() - size, size)); - } - memoryToFree = currentMemory.get(); - } - if (memoryToFree == 0L) { - return memoryToFree; - } - if (LOG.isDebugEnabled()) { - LOG.debug(String.format("unreserve memory size from native: %d", memoryToFree)); - } - target.repay(memoryToFree); - currentMemory.addAndGet(-memoryToFree); - usage.inc(-size); - return memoryToFree; - } - } - - @Override - public void inactivate() { - synchronized (this) { - // for some reasons, memory audit in the native code may not be 100% accurate - // we'll allow the inaccuracy - if (currentMemory.get() > 0) { - unreserve(currentMemory.get()); - } else if (currentMemory.get() < 0) { - reserve(currentMemory.get()); - } - currentMemory.set(0L); - - target = null; // make it gc reachable - open = false; - } - } - - @Override - public long currentMemory() { - return currentMemory.get(); - } -} diff --git a/backends-clickhouse/src/main/java/org/apache/gluten/memory/alloc/CHNativeMemoryAllocator.java b/backends-clickhouse/src/main/java/org/apache/gluten/memory/alloc/CHNativeMemoryAllocator.java deleted file mode 100644 index 0df0757c8b542..0000000000000 --- a/backends-clickhouse/src/main/java/org/apache/gluten/memory/alloc/CHNativeMemoryAllocator.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * 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.gluten.memory.alloc; - -/** - * Like {@link org.apache.gluten.vectorized.NativePlanEvaluator}, this along with {@link - * CHNativeMemoryAllocators}, as built-in toolkit for managing native memory allocations. - */ -public class CHNativeMemoryAllocator { - - private final long nativeInstanceId; - private final CHReservationListener listener; - - public CHNativeMemoryAllocator(long nativeInstanceId, CHReservationListener listener) { - this.nativeInstanceId = nativeInstanceId; - this.listener = listener; - } - - public static CHNativeMemoryAllocator getDefault() { - return new CHNativeMemoryAllocator(getDefaultAllocator(), CHReservationListener.NOOP); - } - - public static CHNativeMemoryAllocator getDefaultForUT() { - return new CHNativeMemoryAllocator( - createListenableAllocator(CHReservationListener.NOOP), CHReservationListener.NOOP); - } - - public static CHNativeMemoryAllocator createListenable(CHReservationListener listener) { - return new CHNativeMemoryAllocator(createListenableAllocator(listener), listener); - } - - public CHReservationListener listener() { - return listener; - } - - public long getNativeInstanceId() { - return this.nativeInstanceId; - } - - public long getBytesAllocated() { - if (this.nativeInstanceId == -1L) return 0; - return bytesAllocated(this.nativeInstanceId); - } - - public void close() { - if (this.nativeInstanceId == -1L) return; - releaseAllocator(this.nativeInstanceId); - } - - private static native long getDefaultAllocator(); - - private static native long createListenableAllocator(CHReservationListener listener); - - private static native void releaseAllocator(long allocatorId); - - private static native long bytesAllocated(long allocatorId); -} diff --git a/backends-clickhouse/src/main/java/org/apache/gluten/memory/alloc/CHNativeMemoryAllocatorManagerImpl.java b/backends-clickhouse/src/main/java/org/apache/gluten/memory/alloc/CHNativeMemoryAllocatorManagerImpl.java deleted file mode 100644 index 2c1c2547fa818..0000000000000 --- a/backends-clickhouse/src/main/java/org/apache/gluten/memory/alloc/CHNativeMemoryAllocatorManagerImpl.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * 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.gluten.memory.alloc; - -public class CHNativeMemoryAllocatorManagerImpl implements CHNativeMemoryAllocatorManager { - private final CHNativeMemoryAllocator managed; - - public CHNativeMemoryAllocatorManagerImpl(CHNativeMemoryAllocator managed) { - this.managed = managed; - } - - @Override - public void release() { - managed.close(); - managed.listener().inactivate(); - } - - @Override - public CHNativeMemoryAllocator getManaged() { - return managed; - } - - @Override - public int priority() { - return 0; // lowest priority - } - - @Override - public String resourceName() { - return "CHNativeMemoryAllocatorManager_" + managed.getNativeInstanceId(); - } -} diff --git a/backends-clickhouse/src/main/java/org/apache/gluten/memory/alloc/CHNativeMemoryAllocators.java b/backends-clickhouse/src/main/java/org/apache/gluten/memory/alloc/CHNativeMemoryAllocators.java deleted file mode 100644 index 0f30972fcd44d..0000000000000 --- a/backends-clickhouse/src/main/java/org/apache/gluten/memory/alloc/CHNativeMemoryAllocators.java +++ /dev/null @@ -1,100 +0,0 @@ -/* - * 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.gluten.memory.alloc; - -import org.apache.gluten.memory.SimpleMemoryUsageRecorder; -import org.apache.gluten.memory.memtarget.MemoryTargets; -import org.apache.gluten.memory.memtarget.Spiller; -import org.apache.gluten.memory.memtarget.Spillers; - -import org.apache.spark.memory.TaskMemoryManager; -import org.apache.spark.util.TaskResources; - -import java.util.Collections; - -/** - * Built-in toolkit for managing native memory allocations. To use the facility, one should import - * Gluten's C++ library then create the c++ instance using following example code: - * - *

```c++ auto* allocator = reinterpret_cast(allocator_id); ``` - * - *

The ID "allocator_id" can be retrieved from Java API {@link - * CHNativeMemoryAllocator#getNativeInstanceId()}. - * - *

FIXME: to export the native APIs in a standard way - */ -public abstract class CHNativeMemoryAllocators { - private CHNativeMemoryAllocators() {} - - private static final CHNativeMemoryAllocator GLOBAL = CHNativeMemoryAllocator.getDefault(); - - private static CHNativeMemoryAllocatorManager createNativeMemoryAllocatorManager( - String name, - TaskMemoryManager taskMemoryManager, - Spiller spiller, - SimpleMemoryUsageRecorder usage) { - - CHManagedCHReservationListener rl = - new CHManagedCHReservationListener( - MemoryTargets.newConsumer(taskMemoryManager, name, spiller, Collections.emptyMap()), - usage); - return new CHNativeMemoryAllocatorManagerImpl(CHNativeMemoryAllocator.createListenable(rl)); - } - - public static CHNativeMemoryAllocator contextInstance() { - if (!TaskResources.inSparkTask()) { - return globalInstance(); - } - - final String id = CHNativeMemoryAllocatorManager.class.toString(); - if (!TaskResources.isResourceRegistered(id)) { - final CHNativeMemoryAllocatorManager manager = - createNativeMemoryAllocatorManager( - "ContextInstance", - TaskResources.getLocalTaskContext().taskMemoryManager(), - Spillers.NOOP, - TaskResources.getSharedUsage()); - TaskResources.addResource(id, manager); - } - return ((CHNativeMemoryAllocatorManager) TaskResources.getResource(id)).getManaged(); - } - - public static CHNativeMemoryAllocator contextInstanceForUT() { - return CHNativeMemoryAllocator.getDefaultForUT(); - } - - public static CHNativeMemoryAllocator createSpillable(String name, Spiller spiller) { - if (!TaskResources.inSparkTask()) { - throw new IllegalStateException("spiller must be used in a Spark task"); - } - - final CHNativeMemoryAllocatorManager manager = - createNativeMemoryAllocatorManager( - name, - TaskResources.getLocalTaskContext().taskMemoryManager(), - spiller, - TaskResources.getSharedUsage()); - TaskResources.addAnonymousResource(manager); - // force add memory consumer to task memory manager, will release by inactivate - manager.getManaged().listener().reserve(1); - return manager.getManaged(); - } - - public static CHNativeMemoryAllocator globalInstance() { - return GLOBAL; - } -} diff --git a/backends-clickhouse/src/main/java/org/apache/gluten/memory/alloc/CHReservationListener.java b/backends-clickhouse/src/main/java/org/apache/gluten/memory/alloc/CHReservationListener.java deleted file mode 100644 index 926c4426d3d5d..0000000000000 --- a/backends-clickhouse/src/main/java/org/apache/gluten/memory/alloc/CHReservationListener.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * 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.gluten.memory.alloc; - -public interface CHReservationListener { - CHReservationListener NOOP = - new CHReservationListener() { - @Override - public void reserveOrThrow(long size) {} - - @Override - public long reserve(long size) { - return 0L; - } - - @Override - public long unreserve(long size) { - return 0L; - } - - @Override - public void inactivate() {} - - @Override - public long currentMemory() { - return 0L; - } - }; - - long reserve(long size); - - void reserveOrThrow(long size); - - long unreserve(long size); - - void inactivate(); - - long currentMemory(); -} diff --git a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHNativeExpressionEvaluator.java b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHNativeExpressionEvaluator.java index 01f38cb3b90be..adcf827eaf167 100644 --- a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHNativeExpressionEvaluator.java +++ b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHNativeExpressionEvaluator.java @@ -18,7 +18,7 @@ import org.apache.gluten.GlutenConfig; import org.apache.gluten.backendsapi.BackendsApiManager; -import org.apache.gluten.memory.alloc.CHNativeMemoryAllocators; +import org.apache.gluten.memory.CHThreadGroup; import org.apache.gluten.substrait.expression.ExpressionBuilder; import org.apache.gluten.substrait.expression.StringMapNode; import org.apache.gluten.substrait.extensions.AdvancedExtensionNode; @@ -81,9 +81,7 @@ private static Map getNativeBackendConf() { public static void injectWriteFilesTempPath(String path, String fileName) { ExpressionEvaluatorJniWrapper.injectWriteFilesTempPath( - CHNativeMemoryAllocators.contextInstance().getNativeInstanceId(), - path.getBytes(StandardCharsets.UTF_8), - fileName.getBytes(StandardCharsets.UTF_8)); + path.getBytes(StandardCharsets.UTF_8), fileName.getBytes(StandardCharsets.UTF_8)); } // Used by WholeStageTransform to create the native computing pipeline and @@ -93,9 +91,9 @@ public static BatchIterator createKernelWithBatchIterator( byte[][] splitInfo, List iterList, boolean materializeInput) { + CHThreadGroup.registerNewThreadGroup(); long handle = nativeCreateKernelWithIterator( - CHNativeMemoryAllocators.contextInstance().getNativeInstanceId(), wsPlan, splitInfo, iterList.toArray(new GeneralInIterator[0]), @@ -106,10 +104,10 @@ public static BatchIterator createKernelWithBatchIterator( // Only for UT. public static BatchIterator createKernelWithBatchIterator( - long allocId, byte[] wsPlan, byte[][] splitInfo, List iterList) { + byte[] wsPlan, byte[][] splitInfo, List iterList) { + CHThreadGroup.registerNewThreadGroup(); long handle = nativeCreateKernelWithIterator( - allocId, wsPlan, splitInfo, iterList.toArray(new GeneralInIterator[0]), diff --git a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHShuffleSplitterJniWrapper.java b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHShuffleSplitterJniWrapper.java index 815bf472c027a..864cc4eb70ace 100644 --- a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHShuffleSplitterJniWrapper.java +++ b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHShuffleSplitterJniWrapper.java @@ -30,14 +30,9 @@ public long make( String dataFile, String localDirs, int subDirsPerLocalDir, - boolean preferSpill, long spillThreshold, String hashAlgorithm, - boolean throwIfMemoryExceed, - boolean flushBlockBufferBeforeEvict, long maxSortBufferSize, - boolean spillFirstlyBeforeStop, - boolean forceExternalSort, boolean forceMemorySort) { return nativeMake( part.getShortName(), @@ -51,14 +46,9 @@ public long make( dataFile, localDirs, subDirsPerLocalDir, - preferSpill, spillThreshold, hashAlgorithm, - throwIfMemoryExceed, - flushBlockBufferBeforeEvict, maxSortBufferSize, - spillFirstlyBeforeStop, - forceExternalSort, forceMemorySort); } @@ -71,9 +61,6 @@ public long makeForRSS( long spillThreshold, String hashAlgorithm, Object pusher, - boolean throwIfMemoryExceed, - boolean flushBlockBufferBeforeEvict, - boolean forceExternalSort, boolean forceMemorySort) { return nativeMakeForRSS( part.getShortName(), @@ -87,9 +74,6 @@ public long makeForRSS( spillThreshold, hashAlgorithm, pusher, - throwIfMemoryExceed, - flushBlockBufferBeforeEvict, - forceExternalSort, forceMemorySort); } @@ -105,14 +89,9 @@ public native long nativeMake( String dataFile, String localDirs, int subDirsPerLocalDir, - boolean preferSpill, long spillThreshold, String hashAlgorithm, - boolean throwIfMemoryExceed, - boolean flushBlockBufferBeforeEvict, long maxSortBufferSize, - boolean spillFirstlyBeforeStop, - boolean forceSort, boolean forceMemorySort); public native long nativeMakeForRSS( @@ -127,15 +106,10 @@ public native long nativeMakeForRSS( long spillThreshold, String hashAlgorithm, Object pusher, - boolean throwIfMemoryExceed, - boolean flushBlockBufferBeforeEvict, - boolean forceSort, boolean forceMemorySort); public native void split(long splitterId, long block); - public native long evict(long splitterId); - public native CHSplitResult stop(long splitterId) throws IOException; public native void close(long splitterId); diff --git a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/ExpressionEvaluatorJniWrapper.java b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/ExpressionEvaluatorJniWrapper.java index a5a474d2a2521..e73b293d618e6 100644 --- a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/ExpressionEvaluatorJniWrapper.java +++ b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/ExpressionEvaluatorJniWrapper.java @@ -32,11 +32,9 @@ public class ExpressionEvaluatorJniWrapper { /** * Create a native compute kernel and return a columnar result iterator. * - * @param allocatorId allocator id * @return iterator instance id */ public static native long nativeCreateKernelWithIterator( - long allocatorId, byte[] wsPlan, byte[][] splitInfo, GeneralInIterator[] batchItr, @@ -46,9 +44,7 @@ public static native long nativeCreateKernelWithIterator( /** * Set the temp path for writing files. * - * @param allocatorId allocator id for current task attempt(or thread) * @param path the temp path for writing files */ - public static native void injectWriteFilesTempPath( - long allocatorId, byte[] path, byte[] filename); + public static native void injectWriteFilesTempPath(byte[] path, byte[] filename); } diff --git a/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/CHDatasourceJniWrapper.java b/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/CHDatasourceJniWrapper.java index c041ee352c421..2bb3d44e0ff16 100644 --- a/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/CHDatasourceJniWrapper.java +++ b/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/CHDatasourceJniWrapper.java @@ -28,8 +28,7 @@ public native long nativeInitMergeTreeWriterWrapper( String taskId, String partition_dir, String bucket_dir, - byte[] confArray, - long allocId); + byte[] confArray); public native String nativeMergeMTParts( byte[] plan, diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala index d54eb59036d86..7519580b9cb74 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala @@ -20,6 +20,7 @@ import org.apache.gluten.GlutenNumaBindingInfo import org.apache.gluten.backendsapi.IteratorApi import org.apache.gluten.execution._ import org.apache.gluten.expression.ConverterUtils +import org.apache.gluten.memory.CHThreadGroup import org.apache.gluten.metrics.{IMetrics, NativeMetrics} import org.apache.gluten.substrait.plan.PlanNode import org.apache.gluten.substrait.rel._ @@ -82,6 +83,7 @@ class CHIteratorApi extends IteratorApi with Logging with LogLevelUtil { listIterator, materializeInput ) + } private def createCloseIterator( @@ -293,6 +295,7 @@ class CHIteratorApi extends IteratorApi with Logging with LogLevelUtil { } override def injectWriteFilesTempPath(path: String, fileName: String): Unit = { + CHThreadGroup.registerNewThreadGroup() CHNativeExpressionEvaluator.injectWriteFilesTempPath(path, fileName) } } diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/CHColumnarShuffleWriter.scala b/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/CHColumnarShuffleWriter.scala index c113f8d4dd319..db9bba5f170a3 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/CHColumnarShuffleWriter.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/CHColumnarShuffleWriter.scala @@ -18,8 +18,7 @@ package org.apache.spark.shuffle import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.clickhouse.CHBackendSettings -import org.apache.gluten.memory.alloc.CHNativeMemoryAllocators -import org.apache.gluten.memory.memtarget.{MemoryTarget, Spiller, Spillers} +import org.apache.gluten.memory.CHThreadGroup import org.apache.gluten.vectorized._ import org.apache.spark.SparkEnv @@ -54,13 +53,7 @@ class CHColumnarShuffleWriter[K, V]( private val splitSize = GlutenConfig.getConf.maxBatchSize private val customizedCompressCodec = GlutenShuffleUtils.getCompressionCodec(conf).toUpperCase(Locale.ROOT) - private val preferSpill = GlutenConfig.getConf.chColumnarShufflePreferSpill - private val throwIfMemoryExceed = GlutenConfig.getConf.chColumnarThrowIfMemoryExceed - private val flushBlockBufferBeforeEvict = - GlutenConfig.getConf.chColumnarFlushBlockBufferBeforeEvict private val maxSortBufferSize = GlutenConfig.getConf.chColumnarMaxSortBufferSize - private val spillFirstlyBeforeStop = GlutenConfig.getConf.chColumnarSpillFirstlyBeforeStop - private val forceExternalSortShuffle = GlutenConfig.getConf.chColumnarForceExternalSortShuffle private val forceMemorySortShuffle = GlutenConfig.getConf.chColumnarForceMemorySortShuffle private val spillThreshold = GlutenConfig.getConf.chColumnarShuffleSpillThreshold private val jniWrapper = new CHShuffleSplitterJniWrapper @@ -81,6 +74,7 @@ class CHColumnarShuffleWriter[K, V]( @throws[IOException] override def write(records: Iterator[Product2[K, V]]): Unit = { + CHThreadGroup.registerNewThreadGroup() internalCHWrite(records) } @@ -108,36 +102,11 @@ class CHColumnarShuffleWriter[K, V]( dataTmp.getAbsolutePath, localDirs, subDirsPerLocalDir, - preferSpill, spillThreshold, CHBackendSettings.shuffleHashAlgorithm, - throwIfMemoryExceed, - flushBlockBufferBeforeEvict, maxSortBufferSize, - spillFirstlyBeforeStop, - forceExternalSortShuffle, forceMemorySortShuffle ) - CHNativeMemoryAllocators.createSpillable( - "ShuffleWriter", - new Spiller() { - override def spill(self: MemoryTarget, phase: Spiller.Phase, size: Long): Long = { - if (!Spillers.PHASE_SET_SPILL_ONLY.contains(phase)) { - return 0L; - } - if (nativeSplitter == 0) { - throw new IllegalStateException( - "Fatal: spill() called before a shuffle writer " + - "is created. This behavior should be optimized by moving memory " + - "allocations from make() to split()") - } - logError(s"Gluten shuffle writer: Trying to spill $size bytes of data") - val spilled = splitterJniWrapper.evict(nativeSplitter); - logError(s"Gluten shuffle writer: Spilled $spilled / $size bytes of data") - spilled - } - } - ) } while (records.hasNext) { val cb = records.next()._2.asInstanceOf[ColumnarBatch] diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHFormatWriterInjects.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHFormatWriterInjects.scala index 9d4c26e5a47b4..06d5b152716d3 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHFormatWriterInjects.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHFormatWriterInjects.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.datasources.v1 import org.apache.gluten.execution.datasource.GlutenRowSplitter +import org.apache.gluten.memory.CHThreadGroup import org.apache.gluten.vectorized.CHColumnVector import org.apache.spark.sql.SparkSession @@ -37,6 +38,7 @@ trait CHFormatWriterInjects extends GlutenFormatWriterInjectsBase { nativeConf: java.util.Map[String, String]): OutputWriter = { val originPath = path val datasourceJniWrapper = new CHDatasourceJniWrapper(); + CHThreadGroup.registerNewThreadGroup() val instance = datasourceJniWrapper.nativeInitFileWriterWrapper( path, diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHMergeTreeWriterInjects.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHMergeTreeWriterInjects.scala index 237d5a46d69f9..d203deacc8104 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHMergeTreeWriterInjects.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHMergeTreeWriterInjects.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.datasources.v1 import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.expression.ConverterUtils -import org.apache.gluten.memory.alloc.CHNativeMemoryAllocators import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.gluten.substrait.`type`.ColumnTypeNode import org.apache.gluten.substrait.SubstraitContext @@ -97,7 +96,6 @@ class CHMergeTreeWriterInjects extends GlutenFormatWriterInjectsBase { // use table schema instead of data schema SparkShimLoader.getSparkShims.attributesFromStruct(tableSchema) ) - val allocId = CHNativeMemoryAllocators.contextInstance.getNativeInstanceId val datasourceJniWrapper = new CHDatasourceJniWrapper() val instance = datasourceJniWrapper.nativeInitMergeTreeWriterWrapper( @@ -107,8 +105,7 @@ class CHMergeTreeWriterInjects extends GlutenFormatWriterInjectsBase { context.getTaskAttemptID.getTaskID.getId.toString, context.getConfiguration.get("mapreduce.task.gluten.mergetree.partition.dir"), context.getConfiguration.get("mapreduce.task.gluten.mergetree.bucketid.str"), - buildNativeConf(nativeConf), - allocId + buildNativeConf(nativeConf) ) new MergeTreeOutputWriter(database, tableName, datasourceJniWrapper, instance, path) diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/clickhouse/MergeTreeFileFormatWriter.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/clickhouse/MergeTreeFileFormatWriter.scala index ad2f3851627c0..506bdd03b4f16 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/clickhouse/MergeTreeFileFormatWriter.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/clickhouse/MergeTreeFileFormatWriter.scala @@ -16,6 +16,8 @@ */ package org.apache.spark.sql.execution.datasources.v1.clickhouse +import org.apache.gluten.memory.CHThreadGroup + import org.apache.spark.{SparkException, TaskContext, TaskOutputFileAlreadyExistException} import org.apache.spark.internal.Logging import org.apache.spark.internal.io.{FileCommitProtocol, SparkHadoopWriterUtils} @@ -263,7 +265,7 @@ object MergeTreeFileFormatWriter extends Logging { iterator: Iterator[InternalRow], concurrentOutputWriterSpec: Option[ConcurrentOutputWriterSpec] ): MergeTreeWriteTaskResult = { - + CHThreadGroup.registerNewThreadGroup(); val jobId = SparkHadoopWriterUtils.createJobID(new Date(jobIdInstant), sparkStageId) val taskId = new TaskID(jobId, TaskType.MAP, sparkPartitionId) val taskAttemptId = new TaskAttemptID(taskId, sparkAttemptNumber) diff --git a/backends-clickhouse/src/main/java/org/apache/gluten/memory/alloc/CHNativeMemoryAllocatorManager.java b/backends-clickhouse/src/test/java/org/apache/gluten/utils/TestExceptionUtils.java similarity index 74% rename from backends-clickhouse/src/main/java/org/apache/gluten/memory/alloc/CHNativeMemoryAllocatorManager.java rename to backends-clickhouse/src/test/java/org/apache/gluten/utils/TestExceptionUtils.java index 6645748e56c2c..74e2141203b03 100644 --- a/backends-clickhouse/src/main/java/org/apache/gluten/memory/alloc/CHNativeMemoryAllocatorManager.java +++ b/backends-clickhouse/src/test/java/org/apache/gluten/utils/TestExceptionUtils.java @@ -14,11 +14,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.memory.alloc; +package org.apache.gluten.utils; -import org.apache.spark.util.TaskResource; - -/** Resource manager implementation that manages a {@link CHNativeMemoryAllocator}. */ -public interface CHNativeMemoryAllocatorManager extends TaskResource { - CHNativeMemoryAllocator getManaged(); +public class TestExceptionUtils { + public static native void generateNativeException(); } diff --git a/backends-clickhouse/src/test/java/org/apache/spark/memory/TestTaskMemoryManagerSuite.java b/backends-clickhouse/src/test/java/org/apache/spark/memory/TestTaskMemoryManagerSuite.java deleted file mode 100644 index 905ffacde023d..0000000000000 --- a/backends-clickhouse/src/test/java/org/apache/spark/memory/TestTaskMemoryManagerSuite.java +++ /dev/null @@ -1,115 +0,0 @@ -/* - * 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.spark.memory; - -import org.apache.gluten.memory.SimpleMemoryUsageRecorder; -import org.apache.gluten.memory.alloc.CHManagedCHReservationListener; -import org.apache.gluten.memory.alloc.CHNativeMemoryAllocator; -import org.apache.gluten.memory.alloc.CHNativeMemoryAllocatorManagerImpl; -import org.apache.gluten.memory.memtarget.MemoryTargets; -import org.apache.gluten.memory.memtarget.Spillers; - -import org.apache.spark.SparkConf; -import org.apache.spark.internal.config.package$; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.util.Collections; - -public class TestTaskMemoryManagerSuite { - static { - // for skip loading lib in NativeMemoryAllocator - System.setProperty("spark.sql.testkey", "true"); - } - - protected TaskMemoryManager taskMemoryManager; - protected CHManagedCHReservationListener listener; - protected CHNativeMemoryAllocatorManagerImpl manager; - - @Before - public void initMemoryManager() { - final SparkConf conf = - new SparkConf() - .set(package$.MODULE$.MEMORY_OFFHEAP_ENABLED(), true) - .set(package$.MODULE$.MEMORY_OFFHEAP_SIZE(), 1000L); - taskMemoryManager = new TaskMemoryManager(new UnifiedMemoryManager(conf, 1000L, 500L, 1), 0); - - listener = - new CHManagedCHReservationListener( - MemoryTargets.newConsumer( - taskMemoryManager, "test", Spillers.NOOP, Collections.emptyMap()), - new SimpleMemoryUsageRecorder()); - - manager = new CHNativeMemoryAllocatorManagerImpl(new CHNativeMemoryAllocator(-1L, listener)); - } - - @After - public void destroyMemoryManager() { - taskMemoryManager = null; - listener = null; - manager = null; - } - - @Test - public void testCHNativeMemoryManager() { - listener.reserveOrThrow(100L); - Assert.assertEquals(100L, taskMemoryManager.getMemoryConsumptionForThisTask()); - - listener.unreserve(100L); - Assert.assertEquals(0L, taskMemoryManager.getMemoryConsumptionForThisTask()); - } - - @Test - public void testMemoryFreeLessThanMalloc() { - listener.reserveOrThrow(100L); - Assert.assertEquals(100L, taskMemoryManager.getMemoryConsumptionForThisTask()); - - listener.unreserve(200L); - Assert.assertEquals(0L, taskMemoryManager.getMemoryConsumptionForThisTask()); - } - - @Test - public void testMemoryLeak() { - listener.reserveOrThrow(100L); - Assert.assertEquals(100L, taskMemoryManager.getMemoryConsumptionForThisTask()); - - listener.unreserve(100L); - Assert.assertEquals(0L, taskMemoryManager.getMemoryConsumptionForThisTask()); - - listener.reserveOrThrow(100L); - Assert.assertEquals(100L, taskMemoryManager.getMemoryConsumptionForThisTask()); - - listener.reserveOrThrow(100L); - Assert.assertEquals(200L, taskMemoryManager.getMemoryConsumptionForThisTask()); - - try { - manager.release(); - } catch (Exception e) { - Assert.assertTrue(e instanceof UnsupportedOperationException); - } - } - - @Test(expected = UnsupportedOperationException.class) - public void testAcquireLessMemory() { - listener.reserveOrThrow(100L); - Assert.assertEquals(100L, taskMemoryManager.getMemoryConsumptionForThisTask()); - - listener.reserveOrThrow(1000L); - } -} diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseColumnarExternalSortShuffleSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseColumnarExternalSortShuffleSuite.scala deleted file mode 100644 index be36cd998485d..0000000000000 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseColumnarExternalSortShuffleSuite.scala +++ /dev/null @@ -1,128 +0,0 @@ -/* - * 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.gluten.execution - -import org.apache.spark.SparkConf -import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper - -class GlutenClickHouseColumnarExternalSortShuffleSuite - extends GlutenClickHouseTPCHAbstractSuite - with AdaptiveSparkPlanHelper { - - override protected val tablesPath: String = basePath + "/tpch-data-ch" - override protected val tpchQueries: String = rootPath + "queries/tpch-queries-ch" - override protected val queriesResults: String = rootPath + "mergetree-queries-output" - - /** Run Gluten + ClickHouse Backend with ColumnarShuffleManager */ - override protected def sparkConf: SparkConf = { - super.sparkConf - .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") - .set("spark.io.compression.codec", "LZ4") - .set("spark.sql.shuffle.partitions", "5") - .set("spark.sql.autoBroadcastJoinThreshold", "10MB") - .set("spark.sql.adaptive.enabled", "true") - .set("spark.gluten.sql.columnar.backend.ch.forceExternalSortShuffle", "true") - } - - test("TPCH Q1") { - runTPCHQuery(1) { df => } - } - - test("TPCH Q2") { - runTPCHQuery(2) { df => } - } - - test("TPCH Q3") { - runTPCHQuery(3) { df => } - } - - test("TPCH Q4") { - runTPCHQuery(4) { df => } - } - - test("TPCH Q5") { - runTPCHQuery(5) { df => } - } - - test("TPCH Q6") { - runTPCHQuery(6) { df => } - } - - test("TPCH Q7") { - runTPCHQuery(7) { df => } - } - - test("TPCH Q8") { - runTPCHQuery(8) { df => } - } - - test("TPCH Q9") { - runTPCHQuery(9) { df => } - } - - test("TPCH Q10") { - runTPCHQuery(10) { df => } - } - - test("TPCH Q11") { - runTPCHQuery(11) { df => } - } - - test("TPCH Q12") { - runTPCHQuery(12) { df => } - } - - test("TPCH Q13") { - runTPCHQuery(13) { df => } - } - - test("TPCH Q14") { - runTPCHQuery(14) { df => } - } - - test("TPCH Q15") { - runTPCHQuery(15) { df => } - } - - test("TPCH Q16") { - runTPCHQuery(16, noFallBack = false) { df => } - } - - test("TPCH Q17") { - runTPCHQuery(17) { df => } - } - - test("TPCH Q18") { - runTPCHQuery(18) { df => } - } - - test("TPCH Q19") { - runTPCHQuery(19) { df => } - } - - test("TPCH Q20") { - runTPCHQuery(20) { df => } - } - - test("TPCH Q21") { - runTPCHQuery(21, noFallBack = false) { df => } - } - - test("TPCH Q22") { - runTPCHQuery(22) { df => } - } -} diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeExceptionSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeExceptionSuite.scala index a0fac50598d8c..cac1a8c5b3464 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeExceptionSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeExceptionSuite.scala @@ -17,8 +17,7 @@ package org.apache.gluten.execution import org.apache.gluten.GlutenConfig -import org.apache.gluten.memory.alloc.{CHNativeMemoryAllocator, CHReservationListener} -import org.apache.gluten.utils.UTSystemParameters +import org.apache.gluten.utils.{TestExceptionUtils, UTSystemParameters} import org.apache.spark.SparkConf @@ -31,12 +30,11 @@ class GlutenClickHouseNativeExceptionSuite extends GlutenClickHouseWholeStageTra test("native exception caught by jvm") { try { - val x = new CHNativeMemoryAllocator(100, CHReservationListener.NOOP) - x.close() // this will incur a native exception + TestExceptionUtils.generateNativeException() assert(false) } catch { case e: Exception => - assert(e.getMessage.contains("allocator 100 not found")) + assert(e.getMessage.contains("test native exception")) } } } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHousePreferSpillColumnarShuffleAQESuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHousePreferSpillColumnarShuffleAQESuite.scala deleted file mode 100644 index 1884f850718ad..0000000000000 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHousePreferSpillColumnarShuffleAQESuite.scala +++ /dev/null @@ -1,169 +0,0 @@ -/* - * 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.gluten.execution - -import org.apache.spark.SparkConf -import org.apache.spark.sql.execution.CoalescedPartitionSpec -import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AdaptiveSparkPlanHelper, AQEShuffleReadExec} - -class GlutenClickHousePreferSpillColumnarShuffleAQESuite - extends GlutenClickHouseTPCHAbstractSuite - with AdaptiveSparkPlanHelper { - - override protected val tablesPath: String = basePath + "/tpch-data-ch" - override protected val tpchQueries: String = rootPath + "queries/tpch-queries-ch" - override protected val queriesResults: String = rootPath + "mergetree-queries-output" - - /** Run Gluten + ClickHouse Backend with ColumnarShuffleManager */ - override protected def sparkConf: SparkConf = { - super.sparkConf - .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") - .set("spark.io.compression.codec", "LZ4") - .set("spark.sql.shuffle.partitions", "5") - .set("spark.sql.autoBroadcastJoinThreshold", "10MB") - .set("spark.sql.adaptive.enabled", "true") - .set("spark.gluten.sql.columnar.backend.ch.shuffle.preferSpill", "true") - } - - test("TPCH Q1") { - runTPCHQuery(1) { - df => - assert(df.queryExecution.executedPlan.isInstanceOf[AdaptiveSparkPlanExec]) - - val colCustomShuffleReaderExecs = collect(df.queryExecution.executedPlan) { - case csr: AQEShuffleReadExec => csr - } - assert(colCustomShuffleReaderExecs.size == 2) - val coalescedPartitionSpec0 = colCustomShuffleReaderExecs(0) - .partitionSpecs(0) - .asInstanceOf[CoalescedPartitionSpec] - assert(coalescedPartitionSpec0.startReducerIndex == 0) - assert(coalescedPartitionSpec0.endReducerIndex == 5) - val coalescedPartitionSpec1 = colCustomShuffleReaderExecs(1) - .partitionSpecs(0) - .asInstanceOf[CoalescedPartitionSpec] - assert(coalescedPartitionSpec1.startReducerIndex == 0) - assert(coalescedPartitionSpec1.endReducerIndex == 5) - } - } - - test("TPCH Q2") { - runTPCHQuery(2) { df => } - } - - test("TPCH Q3") { - runTPCHQuery(3) { df => } - } - - test("TPCH Q4") { - runTPCHQuery(4) { df => } - } - - test("TPCH Q5") { - runTPCHQuery(5) { df => } - } - - test("TPCH Q6") { - runTPCHQuery(6) { df => } - } - - test("TPCH Q7") { - runTPCHQuery(7) { df => } - } - - test("TPCH Q8") { - runTPCHQuery(8) { df => } - } - - test("TPCH Q9") { - runTPCHQuery(9) { df => } - } - - test("TPCH Q10") { - runTPCHQuery(10) { df => } - } - - test("TPCH Q11") { - runTPCHQuery(11) { - df => - assert(df.queryExecution.executedPlan.isInstanceOf[AdaptiveSparkPlanExec]) - val adaptiveSparkPlanExec = collectWithSubqueries(df.queryExecution.executedPlan) { - case adaptive: AdaptiveSparkPlanExec => adaptive - } - assert(adaptiveSparkPlanExec.size == 2) - } - } - - test("TPCH Q12") { - runTPCHQuery(12) { df => } - } - - test("TPCH Q13") { - runTPCHQuery(13) { df => } - } - - test("TPCH Q14") { - runTPCHQuery(14) { df => } - } - - test("TPCH Q15") { - runTPCHQuery(15) { - df => - assert(df.queryExecution.executedPlan.isInstanceOf[AdaptiveSparkPlanExec]) - val adaptiveSparkPlanExec = collectWithSubqueries(df.queryExecution.executedPlan) { - case adaptive: AdaptiveSparkPlanExec => adaptive - } - assert(adaptiveSparkPlanExec.size == 2) - } - } - - test("TPCH Q16") { - runTPCHQuery(16, noFallBack = false) { df => } - } - - test("TPCH Q17") { - runTPCHQuery(17) { df => } - } - - test("TPCH Q18") { - runTPCHQuery(18) { df => } - } - - test("TPCH Q19") { - runTPCHQuery(19) { df => } - } - - test("TPCH Q20") { - runTPCHQuery(20) { df => } - } - - test("TPCH Q21") { - runTPCHQuery(21, noFallBack = false) { df => } - } - - test("TPCH Q22") { - runTPCHQuery(22) { - df => - assert(df.queryExecution.executedPlan.isInstanceOf[AdaptiveSparkPlanExec]) - val adaptiveSparkPlanExec = collectWithSubqueries(df.queryExecution.executedPlan) { - case adaptive: AdaptiveSparkPlanExec => adaptive - } - assert(adaptiveSparkPlanExec.size == 3) - assert(adaptiveSparkPlanExec(1) == adaptiveSparkPlanExec(2)) - } - } -} diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseMetricsUTUtils.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseMetricsUTUtils.scala index 3253e04bb36cc..801b60dda7f20 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseMetricsUTUtils.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseMetricsUTUtils.scala @@ -17,7 +17,6 @@ package org.apache.gluten.execution.metrics import org.apache.gluten.execution.WholeStageTransformer -import org.apache.gluten.memory.alloc.CHNativeMemoryAllocators import org.apache.gluten.metrics.{MetricsUtil, NativeMetrics} import org.apache.gluten.utils.SubstraitPlanPrinterUtil import org.apache.gluten.vectorized.{CHNativeExpressionEvaluator, GeneralInIterator} @@ -45,9 +44,7 @@ object GlutenClickHouseMetricsUTUtils { SubstraitPlanPrinterUtil.jsonToSubstraitPlan( substraitPlanJsonStr.replaceAll("basePath", basePath.substring(1))) - val mockMemoryAllocator = CHNativeMemoryAllocators.contextInstanceForUT() val resIter = CHNativeExpressionEvaluator.createKernelWithBatchIterator( - mockMemoryAllocator.getNativeInstanceId, substraitPlan.toByteArray, new Array[Array[Byte]](0), inBatchIters) @@ -75,7 +72,6 @@ object GlutenClickHouseMetricsUTUtils { iter.foreach(_.toString) resIter.close() - mockMemoryAllocator.close() nativeMetricsList.toSeq } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCDSMetricsSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCDSMetricsSuite.scala index be671575aad60..79017eab0d51a 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCDSMetricsSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCDSMetricsSuite.scala @@ -23,6 +23,7 @@ import org.apache.gluten.vectorized.GeneralInIterator import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.execution.InputIteratorTransformer +import org.apache.spark.util.TaskResources import scala.collection.JavaConverters._ @@ -42,35 +43,36 @@ class GlutenClickHouseTPCDSMetricsSuite extends GlutenClickHouseTPCDSAbstractSui } test("test tpcds q47 metrics") { - // Test getting metrics - val inBatchIters = new java.util.ArrayList[GeneralInIterator]( - Array(0).map(iter => new ColumnarNativeIterator(Iterator.empty.asJava)).toSeq.asJava) - val outputAttributes = new java.util.ArrayList[Attribute](0) - - val nativeMetricsList = GlutenClickHouseMetricsUTUtils - .executeSubstraitPlan( - substraitPlansDatPath + "/tpcds-q47-wholestage-9.json", - basePath, - inBatchIters, - outputAttributes - ) - - assert(nativeMetricsList.size == 1) - val nativeMetricsData = nativeMetricsList(0) - assert(nativeMetricsData.metricsDataList.size() == 7) - - assert(nativeMetricsData.metricsDataList.get(1).getName.equals("kSort")) - assert( - nativeMetricsData.metricsDataList - .get(1) - .getSteps - .get(0) - .getProcessors - .get(0) - .getOutputRows == 0) - assert(nativeMetricsData.metricsDataList.get(2).getName.equals("kWindow")) - assert(nativeMetricsData.metricsDataList.get(4).getName.equals("kWindow")) - + TaskResources.runUnsafe({ + // Test getting metrics + val inBatchIters = new java.util.ArrayList[GeneralInIterator]( + Array(0).map(iter => new ColumnarNativeIterator(Iterator.empty.asJava)).toSeq.asJava) + val outputAttributes = new java.util.ArrayList[Attribute](0) + + val nativeMetricsList = GlutenClickHouseMetricsUTUtils + .executeSubstraitPlan( + substraitPlansDatPath + "/tpcds-q47-wholestage-9.json", + basePath, + inBatchIters, + outputAttributes + ) + + assert(nativeMetricsList.size == 1) + val nativeMetricsData = nativeMetricsList(0) + assert(nativeMetricsData.metricsDataList.size() == 7) + + assert(nativeMetricsData.metricsDataList.get(1).getName.equals("kSort")) + assert( + nativeMetricsData.metricsDataList + .get(1) + .getSteps + .get(0) + .getProcessors + .get(0) + .getOutputRows == 0) + assert(nativeMetricsData.metricsDataList.get(2).getName.equals("kWindow")) + assert(nativeMetricsData.metricsDataList.get(4).getName.equals("kWindow")) + }) // Test metrics update val df = GlutenClickHouseMetricsUTUtils.getTPCDSQueryExecution(spark, "q47", tpcdsQueries) val allWholeStageTransformers = df.queryExecution.executedPlan.collect { diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCHMetricsSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCHMetricsSuite.scala index 1b3df81667a0b..4b5a5b328cb3f 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCHMetricsSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCHMetricsSuite.scala @@ -23,6 +23,7 @@ import org.apache.gluten.vectorized.GeneralInIterator import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.execution.InputIteratorTransformer +import org.apache.spark.util.TaskResources import scala.collection.JavaConverters._ @@ -152,57 +153,59 @@ class GlutenClickHouseTPCHMetricsSuite extends GlutenClickHouseTPCHAbstractSuite } test("test tpch wholestage execute") { - val inBatchIters = new java.util.ArrayList[GeneralInIterator](0) - val outputAttributes = new java.util.ArrayList[Attribute](0) - val nativeMetricsList = GlutenClickHouseMetricsUTUtils - .executeSubstraitPlan( - substraitPlansDatPath + "/tpch-q4-wholestage-2.json", - basePath, - inBatchIters, - outputAttributes - ) - - assert(nativeMetricsList.size == 1) - val nativeMetricsData = nativeMetricsList(0) - assert(nativeMetricsData.metricsDataList.size() == 3) - - assert(nativeMetricsData.metricsDataList.get(0).getName.equals("kRead")) - assert( - nativeMetricsData.metricsDataList - .get(0) - .getSteps - .get(0) - .getProcessors - .get(0) - .getOutputRows == 600572) - - assert(nativeMetricsData.metricsDataList.get(1).getName.equals("kFilter")) - assert( - nativeMetricsData.metricsDataList - .get(1) - .getSteps - .get(0) - .getProcessors - .get(0) - .getInputRows == 600572) - assert( - nativeMetricsData.metricsDataList - .get(1) - .getSteps - .get(0) - .getProcessors - .get(0) - .getOutputRows == 379809) - - assert(nativeMetricsData.metricsDataList.get(2).getName.equals("kProject")) - assert( - nativeMetricsData.metricsDataList - .get(2) - .getSteps - .get(0) - .getProcessors - .get(0) - .getOutputRows == 379809) + TaskResources.runUnsafe { + val inBatchIters = new java.util.ArrayList[GeneralInIterator](0) + val outputAttributes = new java.util.ArrayList[Attribute](0) + val nativeMetricsList = GlutenClickHouseMetricsUTUtils + .executeSubstraitPlan( + substraitPlansDatPath + "/tpch-q4-wholestage-2.json", + basePath, + inBatchIters, + outputAttributes + ) + + assert(nativeMetricsList.size == 1) + val nativeMetricsData = nativeMetricsList(0) + assert(nativeMetricsData.metricsDataList.size() == 3) + + assert(nativeMetricsData.metricsDataList.get(0).getName.equals("kRead")) + assert( + nativeMetricsData.metricsDataList + .get(0) + .getSteps + .get(0) + .getProcessors + .get(0) + .getOutputRows == 600572) + + assert(nativeMetricsData.metricsDataList.get(1).getName.equals("kFilter")) + assert( + nativeMetricsData.metricsDataList + .get(1) + .getSteps + .get(0) + .getProcessors + .get(0) + .getInputRows == 600572) + assert( + nativeMetricsData.metricsDataList + .get(1) + .getSteps + .get(0) + .getProcessors + .get(0) + .getOutputRows == 379809) + + assert(nativeMetricsData.metricsDataList.get(2).getName.equals("kProject")) + assert( + nativeMetricsData.metricsDataList + .get(2) + .getSteps + .get(0) + .getProcessors + .get(0) + .getOutputRows == 379809) + } } test("Check TPCH Q2 metrics updater") { @@ -310,106 +313,108 @@ class GlutenClickHouseTPCHMetricsSuite extends GlutenClickHouseTPCHAbstractSuite } test("GLUTEN-1754: test agg func covar_samp, covar_pop final stage execute") { - val inBatchIters = new java.util.ArrayList[GeneralInIterator](0) - val outputAttributes = new java.util.ArrayList[Attribute](0) - val nativeMetricsList = GlutenClickHouseMetricsUTUtils - .executeSubstraitPlan( - substraitPlansDatPath + "/covar_samp-covar_pop-partial-agg-stage.json", - basePath, - inBatchIters, - outputAttributes - ) - - assert(nativeMetricsList.size == 1) - val nativeMetricsData = nativeMetricsList(0) - assert(nativeMetricsData.metricsDataList.size() == 5) - - assert(nativeMetricsData.metricsDataList.get(0).getName.equals("kRead")) - assert( - nativeMetricsData.metricsDataList - .get(0) - .getSteps - .get(0) - .getProcessors - .get(0) - .getOutputRows == 600572) - - assert(nativeMetricsData.metricsDataList.get(1).getName.equals("kFilter")) - assert( - nativeMetricsData.metricsDataList - .get(1) - .getSteps - .get(0) - .getProcessors - .get(0) - .getInputRows == 600572) - assert( - nativeMetricsData.metricsDataList - .get(1) - .getSteps - .get(0) - .getProcessors - .get(0) - .getOutputRows == 591673) - - assert(nativeMetricsData.metricsDataList.get(2).getName.equals("kProject")) - - assert(nativeMetricsData.metricsDataList.get(3).getName.equals("kProject")) - assert(nativeMetricsData.metricsDataList.get(4).getName.equals("kAggregate")) - assert( - nativeMetricsData.metricsDataList - .get(4) - .getSteps - .get(0) - .getProcessors - .get(0) - .getInputRows == 591673) - assert( - nativeMetricsData.metricsDataList - .get(4) - .getSteps - .get(0) - .getProcessors - .get(0) - .getOutputRows == 4) - - assert( - nativeMetricsData.metricsDataList - .get(4) - .getSteps - .get(0) - .getProcessors - .get(0) - .getOutputRows == 4) - - val inBatchItersFinal = new java.util.ArrayList[GeneralInIterator]( - Array(0).map(iter => new ColumnarNativeIterator(Iterator.empty.asJava)).toSeq.asJava) - val outputAttributesFinal = new java.util.ArrayList[Attribute](0) - - val nativeMetricsListFinal = GlutenClickHouseMetricsUTUtils - .executeSubstraitPlan( - substraitPlansDatPath + "/covar_samp-covar_pop-final-agg-stage.json", - basePath, - inBatchItersFinal, - outputAttributesFinal - ) - - assert(nativeMetricsListFinal.size == 1) - val nativeMetricsDataFinal = nativeMetricsListFinal(0) - assert(nativeMetricsDataFinal.metricsDataList.size() == 3) - - assert(nativeMetricsDataFinal.metricsDataList.get(0).getName.equals("kRead")) - assert(nativeMetricsDataFinal.metricsDataList.get(1).getName.equals("kAggregate")) - assert(nativeMetricsDataFinal.metricsDataList.get(1).getSteps.size() == 2) - assert( - nativeMetricsDataFinal.metricsDataList - .get(1) - .getSteps - .get(0) - .getName - .equals("GraceMergingAggregatedStep")) - assert( - nativeMetricsDataFinal.metricsDataList.get(1).getSteps.get(1).getName.equals("Expression")) - assert(nativeMetricsDataFinal.metricsDataList.get(2).getName.equals("kProject")) + TaskResources.runUnsafe { + val inBatchIters = new java.util.ArrayList[GeneralInIterator](0) + val outputAttributes = new java.util.ArrayList[Attribute](0) + val nativeMetricsList = GlutenClickHouseMetricsUTUtils + .executeSubstraitPlan( + substraitPlansDatPath + "/covar_samp-covar_pop-partial-agg-stage.json", + basePath, + inBatchIters, + outputAttributes + ) + + assert(nativeMetricsList.size == 1) + val nativeMetricsData = nativeMetricsList(0) + assert(nativeMetricsData.metricsDataList.size() == 5) + + assert(nativeMetricsData.metricsDataList.get(0).getName.equals("kRead")) + assert( + nativeMetricsData.metricsDataList + .get(0) + .getSteps + .get(0) + .getProcessors + .get(0) + .getOutputRows == 600572) + + assert(nativeMetricsData.metricsDataList.get(1).getName.equals("kFilter")) + assert( + nativeMetricsData.metricsDataList + .get(1) + .getSteps + .get(0) + .getProcessors + .get(0) + .getInputRows == 600572) + assert( + nativeMetricsData.metricsDataList + .get(1) + .getSteps + .get(0) + .getProcessors + .get(0) + .getOutputRows == 591673) + + assert(nativeMetricsData.metricsDataList.get(2).getName.equals("kProject")) + + assert(nativeMetricsData.metricsDataList.get(3).getName.equals("kProject")) + assert(nativeMetricsData.metricsDataList.get(4).getName.equals("kAggregate")) + assert( + nativeMetricsData.metricsDataList + .get(4) + .getSteps + .get(0) + .getProcessors + .get(0) + .getInputRows == 591673) + assert( + nativeMetricsData.metricsDataList + .get(4) + .getSteps + .get(0) + .getProcessors + .get(0) + .getOutputRows == 4) + + assert( + nativeMetricsData.metricsDataList + .get(4) + .getSteps + .get(0) + .getProcessors + .get(0) + .getOutputRows == 4) + + val inBatchItersFinal = new java.util.ArrayList[GeneralInIterator]( + Array(0).map(iter => new ColumnarNativeIterator(Iterator.empty.asJava)).toSeq.asJava) + val outputAttributesFinal = new java.util.ArrayList[Attribute](0) + + val nativeMetricsListFinal = GlutenClickHouseMetricsUTUtils + .executeSubstraitPlan( + substraitPlansDatPath + "/covar_samp-covar_pop-final-agg-stage.json", + basePath, + inBatchItersFinal, + outputAttributesFinal + ) + + assert(nativeMetricsListFinal.size == 1) + val nativeMetricsDataFinal = nativeMetricsListFinal(0) + assert(nativeMetricsDataFinal.metricsDataList.size() == 3) + + assert(nativeMetricsDataFinal.metricsDataList.get(0).getName.equals("kRead")) + assert(nativeMetricsDataFinal.metricsDataList.get(1).getName.equals("kAggregate")) + assert(nativeMetricsDataFinal.metricsDataList.get(1).getSteps.size() == 2) + assert( + nativeMetricsDataFinal.metricsDataList + .get(1) + .getSteps + .get(0) + .getName + .equals("GraceMergingAggregatedStep")) + assert( + nativeMetricsDataFinal.metricsDataList.get(1).getSteps.get(1).getName.equals("Expression")) + assert(nativeMetricsDataFinal.metricsDataList.get(2).getName.equals("kProject")) + } } } diff --git a/cpp-ch/local-engine/Common/CHUtil.cpp b/cpp-ch/local-engine/Common/CHUtil.cpp index 3a699b50e302e..a606a06b5c151 100644 --- a/cpp-ch/local-engine/Common/CHUtil.cpp +++ b/cpp-ch/local-engine/Common/CHUtil.cpp @@ -627,7 +627,7 @@ DB::Context::ConfigurationPtr BackendInitializerUtil::initConfig(std::mapsetString(CH_TASK_MEMORY, backend_conf_map.at(GLUTEN_TASK_OFFHEAP)); + config->setString(MemoryConfig::CH_TASK_MEMORY, backend_conf_map.at(GLUTEN_TASK_OFFHEAP)); } const bool use_current_directory_as_tmp = config->getBool("use_current_directory_as_tmp", false); @@ -1050,17 +1050,6 @@ String DateTimeUtil::convertTimeZone(const String & time_zone) return res; } -UInt64 MemoryUtil::getCurrentMemoryUsage(size_t depth) -{ - Int64 current_memory_usage = 0; - auto * current_mem_tracker = DB::CurrentThread::getMemoryTracker(); - for (size_t i = 0; i < depth && current_mem_tracker; ++i) - current_mem_tracker = current_mem_tracker->getParent(); - if (current_mem_tracker) - current_memory_usage = current_mem_tracker->get(); - return current_memory_usage < 0 ? 0 : current_memory_usage; -} - UInt64 MemoryUtil::getMemoryRSS() { long rss = 0L; diff --git a/cpp-ch/local-engine/Common/CHUtil.h b/cpp-ch/local-engine/Common/CHUtil.h index b45c6ab3c4d21..05b730552fa53 100644 --- a/cpp-ch/local-engine/Common/CHUtil.h +++ b/cpp-ch/local-engine/Common/CHUtil.h @@ -188,7 +188,6 @@ class BackendInitializerUtil inline static const std::string SPARK_SESSION_TIME_ZONE = "spark.sql.session.timeZone"; inline static const String GLUTEN_TASK_OFFHEAP = "spark.gluten.memory.task.offHeap.size.in.bytes"; - inline static const String CH_TASK_MEMORY = "off_heap_per_task"; /// On yarn mode, native writing on hdfs cluster takes yarn container user as the user passed to libhdfs3, which /// will cause permission issue because yarn container user is not the owner of the hdfs dir to be written. @@ -252,7 +251,6 @@ class DateTimeUtil class MemoryUtil { public: - static UInt64 getCurrentMemoryUsage(size_t depth = 1); static UInt64 getMemoryRSS(); }; diff --git a/cpp-ch/local-engine/Common/GlutenConfig.h b/cpp-ch/local-engine/Common/GlutenConfig.h new file mode 100644 index 0000000000000..782df7f5413d4 --- /dev/null +++ b/cpp-ch/local-engine/Common/GlutenConfig.h @@ -0,0 +1,166 @@ +/* + * 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. + */ + +#pragma once + +#include +#include +#include + +namespace local_engine +{ +struct MemoryConfig +{ + inline static const String EXTRA_MEMORY_HARD_LIMIT = "extra_memory_hard_limit"; + inline static const String CH_TASK_MEMORY = "off_heap_per_task"; + inline static const String SPILL_MEM_RATIO = "spill_mem_ratio"; + + size_t extra_memory_hard_limit = 0; + size_t off_heap_per_task = 0; + double spill_mem_ratio = 0.9; + + static MemoryConfig loadFromContext(DB::ContextPtr context) + { + MemoryConfig config; + config.extra_memory_hard_limit = context->getConfigRef().getUInt64(EXTRA_MEMORY_HARD_LIMIT, 0); + config.off_heap_per_task = context->getConfigRef().getUInt64(CH_TASK_MEMORY, 0); + config.spill_mem_ratio = context->getConfigRef().getUInt64(SPILL_MEM_RATIO, 0.9); + return config; + } +}; + +struct GraceMergingAggregateConfig +{ + inline static const String MAX_GRACE_AGGREGATE_MERGING_BUCKETS = "max_grace_aggregate_merging_buckets"; + inline static const String THROW_ON_OVERFLOW_GRACE_AGGREGATE_MERGING_BUCKETS = "throw_on_overflow_grace_aggregate_merging_buckets"; + inline static const String AGGREGATED_KEYS_BEFORE_EXTEND_GRACE_AGGREGATE_MERGING_BUCKETS = "aggregated_keys_before_extend_grace_aggregate_merging_buckets"; + inline static const String MAX_PENDING_FLUSH_BLOCKS_PER_GRACE_AGGREGATE_MERGING_BUCKET = "max_pending_flush_blocks_per_grace_aggregate_merging_bucket"; + inline static const String MAX_ALLOWED_MEMORY_USAGE_RATIO_FOR_AGGREGATE_MERGING = "max_allowed_memory_usage_ratio_for_aggregate_merging"; + + size_t max_grace_aggregate_merging_buckets = 32; + bool throw_on_overflow_grace_aggregate_merging_buckets = false; + size_t aggregated_keys_before_extend_grace_aggregate_merging_buckets = 8192; + size_t max_pending_flush_blocks_per_grace_aggregate_merging_bucket = 1_MiB; + double max_allowed_memory_usage_ratio_for_aggregate_merging = 0.9; + + static GraceMergingAggregateConfig loadFromContext(DB::ContextPtr context) + { + GraceMergingAggregateConfig config; + config.max_grace_aggregate_merging_buckets = context->getConfigRef().getUInt64(MAX_GRACE_AGGREGATE_MERGING_BUCKETS, 32); + config.throw_on_overflow_grace_aggregate_merging_buckets = context->getConfigRef().getBool(THROW_ON_OVERFLOW_GRACE_AGGREGATE_MERGING_BUCKETS, false); + config.aggregated_keys_before_extend_grace_aggregate_merging_buckets = context->getConfigRef().getUInt64(AGGREGATED_KEYS_BEFORE_EXTEND_GRACE_AGGREGATE_MERGING_BUCKETS, 8192); + config.max_pending_flush_blocks_per_grace_aggregate_merging_bucket = context->getConfigRef().getUInt64(MAX_PENDING_FLUSH_BLOCKS_PER_GRACE_AGGREGATE_MERGING_BUCKET, 1_MiB); + config.max_allowed_memory_usage_ratio_for_aggregate_merging = context->getConfigRef().getDouble(MAX_ALLOWED_MEMORY_USAGE_RATIO_FOR_AGGREGATE_MERGING, 0.9); + return config; + } +}; + +struct StreamingAggregateConfig +{ + inline static const String AGGREGATED_KEYS_BEFORE_STREAMING_AGGREGATING_EVICT = "aggregated_keys_before_streaming_aggregating_evict"; + inline static const String MAX_MEMORY_USAGE_RATIO_FOR_STREAMING_AGGREGATING = "max_memory_usage_ratio_for_streaming_aggregating"; + inline static const String HIGH_CARDINALITY_THRESHOLD_FOR_STREAMING_AGGREGATING = "high_cardinality_threshold_for_streaming_aggregating"; + inline static const String ENABLE_STREAMING_AGGREGATING = "enable_streaming_aggregating"; + + size_t aggregated_keys_before_streaming_aggregating_evict = 1024; + double max_memory_usage_ratio_for_streaming_aggregating = 0.9; + double high_cardinality_threshold_for_streaming_aggregating = 0.8; + bool enable_streaming_aggregating = true; + + static StreamingAggregateConfig loadFromContext(DB::ContextPtr context) + { + StreamingAggregateConfig config; + config.aggregated_keys_before_streaming_aggregating_evict = context->getConfigRef().getUInt64(AGGREGATED_KEYS_BEFORE_STREAMING_AGGREGATING_EVICT, 1024); + config.max_memory_usage_ratio_for_streaming_aggregating = context->getConfigRef().getDouble(MAX_MEMORY_USAGE_RATIO_FOR_STREAMING_AGGREGATING, 0.9); + config.high_cardinality_threshold_for_streaming_aggregating = context->getConfigRef().getDouble(HIGH_CARDINALITY_THRESHOLD_FOR_STREAMING_AGGREGATING, 0.8); + config.enable_streaming_aggregating = context->getConfigRef().getBool(ENABLE_STREAMING_AGGREGATING, true); + return config; + } +}; + +struct ExecutorConfig +{ + inline static const String DUMP_PIPELINE = "dump_pipeline"; + inline static const String USE_LOCAL_FORMAT = "use_local_format"; + + bool dump_pipeline = false; + bool use_local_format = false; + + static ExecutorConfig loadFromContext(DB::ContextPtr context) + { + ExecutorConfig config; + config.dump_pipeline = context->getConfigRef().getBool(DUMP_PIPELINE, false); + config.use_local_format = context->getConfigRef().getBool(USE_LOCAL_FORMAT, false); + return config; + } +}; + +struct HdfsConfig +{ + inline static const String HDFS_ASYNC = "hdfs.enable_async_io"; + + bool hdfs_async = true; + + static HdfsConfig loadFromContext(DB::ContextPtr context) + { + HdfsConfig config; + config.hdfs_async = context->getConfigRef().getBool(HDFS_ASYNC, true); + return config; + } +}; + +struct S3Config +{ + inline static const String S3_LOCAL_CACHE_ENABLE = "s3.local_cache.enabled"; + inline static const String S3_LOCAL_CACHE_MAX_SIZE = "s3.local_cache.max_size"; + inline static const String S3_LOCAL_CACHE_CACHE_PATH = "s3.local_cache.cache_path"; + inline static const String S3_GCS_ISSUE_COMPOSE_REQUEST = "s3.gcs_issue_compose_request"; + + bool s3_local_cache_enabled = false; + size_t s3_local_cache_max_size = 100_GiB; + String s3_local_cache_cache_path = ""; + bool s3_gcs_issue_compose_request = false; + + static S3Config loadFromContext(DB::ContextPtr context) + { + S3Config config; + config.s3_local_cache_enabled = context->getConfigRef().getBool(S3_LOCAL_CACHE_ENABLE, false); + config.s3_local_cache_max_size = context->getConfigRef().getUInt64(S3_LOCAL_CACHE_MAX_SIZE, 100_GiB); + config.s3_local_cache_cache_path = context->getConfigRef().getString(S3_LOCAL_CACHE_CACHE_PATH, ""); + config.s3_gcs_issue_compose_request = context->getConfigRef().getBool(S3_GCS_ISSUE_COMPOSE_REQUEST, false); + return config; + } +}; + +struct MergeTreeConfig +{ + inline static const String TABLE_PART_METADATA_CACHE_MAX_COUNT = "table_part_metadata_cache_max_count"; + inline static const String TABLE_METADATA_CACHE_MAX_COUNT = "table_metadata_cache_max_count"; + + size_t table_part_metadata_cache_max_count = 1000; + size_t table_metadata_cache_max_count = 100; + + static MergeTreeConfig loadFromContext(DB::ContextPtr context) + { + MergeTreeConfig config; + config.table_part_metadata_cache_max_count = context->getConfigRef().getUInt64(TABLE_PART_METADATA_CACHE_MAX_COUNT, 1000); + config.table_metadata_cache_max_count = context->getConfigRef().getUInt64(TABLE_METADATA_CACHE_MAX_COUNT, 100); + return config; + } +}; +} + diff --git a/cpp-ch/local-engine/Common/QueryContext.cpp b/cpp-ch/local-engine/Common/QueryContext.cpp index f4d39c612430e..68934adad3671 100644 --- a/cpp-ch/local-engine/Common/QueryContext.cpp +++ b/cpp-ch/local-engine/Common/QueryContext.cpp @@ -15,33 +15,47 @@ * limitations under the License. */ #include "QueryContext.h" + +#include + #include #include -#include #include #include +#include +#include +#include +#include +#include namespace DB { namespace ErrorCodes { - extern const int LOGICAL_ERROR; +extern const int LOGICAL_ERROR; } } namespace local_engine { using namespace DB; -thread_local std::shared_ptr query_scope; -thread_local std::shared_ptr thread_status; -ConcurrentMap allocator_map; -int64_t initializeQuery(ReservationListenerWrapperPtr listener) +struct QueryContext +{ + std::shared_ptr thread_status; + std::shared_ptr thread_group; + ContextMutablePtr query_context; +}; + +std::unordered_map> query_map; +std::mutex query_map_mutex; + +int64_t QueryContextManager::initializeQuery() { - if (thread_status) return -1; - auto query_context = Context::createCopy(SerializedPlanParser::global_context); - query_context->makeQueryContext(); + std::shared_ptr query_context = std::make_shared(); + query_context->query_context = Context::createCopy(SerializedPlanParser::global_context); + query_context->query_context->makeQueryContext(); // empty input will trigger random query id to be set // FileCache will check if query id is set to decide whether to skip cache or not @@ -49,56 +63,115 @@ int64_t initializeQuery(ReservationListenerWrapperPtr listener) // // Notice: // this generated random query id a qualified global queryid for the spark query - query_context->setCurrentQueryId(""); - - auto allocator_context = std::make_shared(); - allocator_context->thread_status = std::make_shared(true); - allocator_context->query_scope = std::make_shared(query_context); - allocator_context->group = std::make_shared(query_context); - allocator_context->query_context = query_context; - allocator_context->listener = listener; - thread_status = allocator_context->thread_status; - query_scope = allocator_context->query_scope; - auto allocator_id = reinterpret_cast(allocator_context.get()); - CurrentMemoryTracker::before_alloc = [listener](Int64 size, bool throw_if_memory_exceed) -> void + query_context->query_context->setCurrentQueryId(toString(UUIDHelpers::generateV4())); + auto config = MemoryConfig::loadFromContext(query_context->query_context); + query_context->thread_status = std::make_shared(false); + query_context->thread_group = std::make_shared(query_context->query_context); + CurrentThread::attachToGroup(query_context->thread_group); + auto memory_limit = config.off_heap_per_task; + + query_context->thread_group->memory_tracker.setSoftLimit(memory_limit); + query_context->thread_group->memory_tracker.setHardLimit(memory_limit + config.extra_memory_hard_limit); + std::lock_guard lock_guard(query_map_mutex); + int64_t id = reinterpret_cast(query_context->thread_group.get()); + query_map.emplace(id, query_context); + return id; +} + +DB::ContextMutablePtr QueryContextManager::currentQueryContext() +{ + if (!CurrentThread::getGroup()) { - if (throw_if_memory_exceed) - listener->reserveOrThrow(size); - else - listener->reserve(size); - }; - CurrentMemoryTracker::before_free = [listener](Int64 size) -> void { listener->tryFree(size); }; - CurrentMemoryTracker::current_memory = [listener]() -> Int64 { return listener->currentMemory(); }; - allocator_map.insert(allocator_id, allocator_context); - return allocator_id; + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Thread group not found."); + } + std::lock_guard lock_guard(query_map_mutex); + int64_t id = reinterpret_cast(CurrentThread::getGroup().get()); + return query_map[id]->query_context; } -void releaseAllocator(int64_t allocator_id) +void QueryContextManager::logCurrentPerformanceCounters(ProfileEvents::Counters & counters) { - if (!allocator_map.get(allocator_id)) + if (!CurrentThread::getGroup()) + { + return; + } + if (logger->information()) { - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "allocator {} not found", allocator_id); + std::ostringstream msg; + msg << "\n---------------------Task Performance Counters-----------------------------\n"; + for (ProfileEvents::Event event = ProfileEvents::Event(0); event < counters.num_counters; event++) + { + const auto * name = ProfileEvents::getName(event); + const auto * doc = ProfileEvents::getDocumentation(event); + auto & count = counters[event]; + if (count == 0) + continue; + msg << std::setw(50) << std::setfill(' ') << std::left << name << "|" + << std::setw(20) << std::setfill(' ') << std::left << count.load() + << " | (" << doc << ")\n"; + } + LOG_INFO(logger, "{}", msg.str()); } - auto status = allocator_map.get(allocator_id)->thread_status; - status->detachFromGroup(); - auto listener = allocator_map.get(allocator_id)->listener; - if (status->untracked_memory < 0) - listener->free(-status->untracked_memory); - else if (status->untracked_memory > 0) - listener->reserve(status->untracked_memory); - allocator_map.erase(allocator_id); - thread_status.reset(); - query_scope.reset(); } -NativeAllocatorContextPtr getAllocator(int64_t allocator) +size_t QueryContextManager::currentPeakMemory(int64_t id) { - return allocator_map.get(allocator); + std::lock_guard lock_guard(query_map_mutex); + if (!query_map.contains(id)) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "context released {}", id); + return query_map[id]->thread_group->memory_tracker.getPeak(); } -int64_t allocatorMemoryUsage(int64_t allocator_id) +void QueryContextManager::finalizeQuery(int64_t id) { - return allocator_map.get(allocator_id)->thread_status->memory_tracker.get(); + if (!CurrentThread::getGroup()) + { + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Thread group not found."); + } + std::shared_ptr context; + { + std::lock_guard lock_guard(query_map_mutex); + context = query_map[id]; + } + auto query_context = context->thread_status->getQueryContext(); + if (!query_context) + { + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "query context not found"); + } + context->thread_status->flushUntrackedMemory(); + context->thread_status->finalizePerformanceCounters(); + LOG_INFO(logger, "Task finished, peak memory usage: {} bytes", currentPeakMemory(id)); + + if (currentThreadGroupMemoryUsage() > 1_MiB) + { + LOG_WARNING(logger, "{} bytes memory didn't release, There may be a memory leak!", currentThreadGroupMemoryUsage()); + } + logCurrentPerformanceCounters(context->thread_group->performance_counters); + context->thread_status->detachFromGroup(); + context->thread_group.reset(); + context->thread_status.reset(); + query_context.reset(); + { + std::lock_guard lock_guard(query_map_mutex); + query_map.erase(id); + } } +size_t currentThreadGroupMemoryUsage() +{ + if (!CurrentThread::getGroup()) + { + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Thread group not found, please call initializeQuery first."); + } + return CurrentThread::getGroup()->memory_tracker.get(); +} + +double currentThreadGroupMemoryUsageRatio() +{ + if (!CurrentThread::getGroup()) + { + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Thread group not found, please call initializeQuery first."); + } + return static_cast(CurrentThread::getGroup()->memory_tracker.get()) / CurrentThread::getGroup()->memory_tracker.getSoftLimit(); } +} \ No newline at end of file diff --git a/cpp-ch/local-engine/Common/QueryContext.h b/cpp-ch/local-engine/Common/QueryContext.h index 77d522dc9337a..0fbf4977321f1 100644 --- a/cpp-ch/local-engine/Common/QueryContext.h +++ b/cpp-ch/local-engine/Common/QueryContext.h @@ -15,30 +15,30 @@ * limitations under the License. */ #pragma once -#include #include -#include -#include #include namespace local_engine { -int64_t initializeQuery(ReservationListenerWrapperPtr listener); - -void releaseAllocator(int64_t allocator_id); - -int64_t allocatorMemoryUsage(int64_t allocator_id); - -struct NativeAllocatorContext +class QueryContextManager { - std::shared_ptr query_scope; - std::shared_ptr thread_status; - DB::ContextMutablePtr query_context; - std::shared_ptr group; - ReservationListenerWrapperPtr listener; -}; +public: + static QueryContextManager & instance() + { + static QueryContextManager instance; + return instance; + } + int64_t initializeQuery(); + DB::ContextMutablePtr currentQueryContext(); + void logCurrentPerformanceCounters(ProfileEvents::Counters& counters); + size_t currentPeakMemory(int64_t id); + void finalizeQuery(int64_t id); -using NativeAllocatorContextPtr = std::shared_ptr; +private: + QueryContextManager() = default; + LoggerPtr logger = getLogger("QueryContextManager"); +}; -NativeAllocatorContextPtr getAllocator(int64_t allocator); +size_t currentThreadGroupMemoryUsage(); +double currentThreadGroupMemoryUsageRatio(); } diff --git a/cpp-ch/local-engine/Join/BroadCastJoinBuilder.cpp b/cpp-ch/local-engine/Join/BroadCastJoinBuilder.cpp index c21cc8ba35246..f47f423df89b3 100644 --- a/cpp-ch/local-engine/Join/BroadCastJoinBuilder.cpp +++ b/cpp-ch/local-engine/Join/BroadCastJoinBuilder.cpp @@ -70,11 +70,13 @@ DB::Block resetBuildTableBlockName(Block & block, bool only_one = false) void cleanBuildHashTable(const std::string & hash_table_id, jlong instance) { - /// Thread status holds raw pointer on query context, thus it always must be destroyed - /// It always called by no thread_status. We need create first. - /// Otherwise global tracker will not free bhj memory. - DB::ThreadStatus thread_status; - SharedPointerWrapper::dispose(instance); + auto clean_join = [&] + { + SharedPointerWrapper::dispose(instance); + }; + /// Record memory usage in Total Memory Tracker + ThreadFromGlobalPoolNoTracingContextPropagation thread(clean_join); + thread.join(); LOG_DEBUG(&Poco::Logger::get("BroadCastJoinBuilder"), "Broadcast hash table {} is cleaned", hash_table_id); } @@ -122,6 +124,7 @@ std::shared_ptr buildJoin( header = resetBuildTableBlockName(header); Blocks data; + auto collect_data = [&] { bool header_empty = header.getNamesAndTypesList().empty(); bool only_one_column = header_empty; @@ -157,7 +160,10 @@ std::shared_ptr buildJoin( info.update(final_block); data.emplace_back(std::move(final_block)); } - } + }; + /// Record memory usage in Total Memory Tracker + ThreadFromGlobalPoolNoTracingContextPropagation thread(collect_data); + thread.join(); ColumnsDescription columns_description(header.getNamesAndTypesList()); diff --git a/cpp-ch/local-engine/Join/StorageJoinFromReadBuffer.cpp b/cpp-ch/local-engine/Join/StorageJoinFromReadBuffer.cpp index 2f5afd434b416..1d87b5f57a094 100644 --- a/cpp-ch/local-engine/Join/StorageJoinFromReadBuffer.cpp +++ b/cpp-ch/local-engine/Join/StorageJoinFromReadBuffer.cpp @@ -100,9 +100,16 @@ StorageJoinFromReadBuffer::StorageJoinFromReadBuffer( void StorageJoinFromReadBuffer::buildJoin(Blocks & data, const Block header, std::shared_ptr analyzed_join) { - join = std::make_shared(analyzed_join, header, overwrite, row_count); - for (Block block : data) - join->addBlockToJoin(std::move(block), true); + + auto build_join = [&] + { + join = std::make_shared(analyzed_join, header, overwrite, row_count); + for (Block block : data) + join->addBlockToJoin(std::move(block), true); + }; + /// Record memory usage in Total Memory Tracker + ThreadFromGlobalPoolNoTracingContextPropagation thread(build_join); + thread.join(); } void StorageJoinFromReadBuffer::collectAllInputs(Blocks & data, const DB::Block) @@ -113,28 +120,35 @@ void StorageJoinFromReadBuffer::collectAllInputs(Blocks & data, const DB::Block) void StorageJoinFromReadBuffer::buildJoinLazily(DB::Block header, std::shared_ptr analyzed_join) { + auto build_join = [&] { - std::shared_lock lock(join_mutex); + { + std::shared_lock lock(join_mutex); + if (join) + return; + } + std::unique_lock lock(join_mutex); if (join) return; - } - std::unique_lock lock(join_mutex); - if (join) - return; - join = std::make_shared(analyzed_join, header, overwrite, row_count); - while(!input_blocks.empty()) - { - auto & block = *input_blocks.begin(); - DB::ColumnsWithTypeAndName columns; - for (size_t i = 0; i < block.columns(); ++i) + join = std::make_shared(analyzed_join, header, overwrite, row_count); + while(!input_blocks.empty()) { - const auto & column = block.getByPosition(i); - columns.emplace_back(BlockUtil::convertColumnAsNecessary(column, header.getByPosition(i))); + auto & block = *input_blocks.begin(); + DB::ColumnsWithTypeAndName columns; + for (size_t i = 0; i < block.columns(); ++i) + { + const auto & column = block.getByPosition(i); + columns.emplace_back(BlockUtil::convertColumnAsNecessary(column, header.getByPosition(i))); + } + DB::Block final_block(columns); + join->addBlockToJoin(final_block, true); + input_blocks.pop_front(); } - DB::Block final_block(columns); - join->addBlockToJoin(final_block, true); - input_blocks.pop_front(); - } + }; + + /// Record memory usage in Total Memory Tracker + ThreadFromGlobalPoolNoTracingContextPropagation thread(build_join); + thread.join(); } diff --git a/cpp-ch/local-engine/Operator/GraceMergingAggregatedStep.cpp b/cpp-ch/local-engine/Operator/GraceMergingAggregatedStep.cpp index a9a2df276a594..82b498e58ff90 100644 --- a/cpp-ch/local-engine/Operator/GraceMergingAggregatedStep.cpp +++ b/cpp-ch/local-engine/Operator/GraceMergingAggregatedStep.cpp @@ -22,6 +22,8 @@ #include #include #include +#include +#include namespace DB { @@ -114,12 +116,13 @@ GraceMergingAggregatedTransform::GraceMergingAggregatedTransform(const DB::Block , no_pre_aggregated(no_pre_aggregated_) , tmp_data_disk(std::make_unique(context_->getTempDataOnDisk())) { - max_buckets = context->getConfigRef().getUInt64("max_grace_aggregate_merging_buckets", 32); - throw_on_overflow_buckets = context->getConfigRef().getBool("throw_on_overflow_grace_aggregate_merging_buckets", false); - aggregated_keys_before_extend_buckets = context->getConfigRef().getUInt64("aggregated_keys_before_extend_grace_aggregate_merging_buckets", 8196); + auto config = GraceMergingAggregateConfig::loadFromContext(context); + max_buckets = config.max_grace_aggregate_merging_buckets; + throw_on_overflow_buckets = config.throw_on_overflow_grace_aggregate_merging_buckets; + aggregated_keys_before_extend_buckets = config.aggregated_keys_before_extend_grace_aggregate_merging_buckets; aggregated_keys_before_extend_buckets = PODArrayUtil::adjustMemoryEfficientSize(aggregated_keys_before_extend_buckets); - max_pending_flush_blocks_per_bucket = context->getConfigRef().getUInt64("max_pending_flush_blocks_per_grace_aggregate_merging_bucket", 1024 * 1024); - max_allowed_memory_usage_ratio = context->getConfigRef().getDouble("max_allowed_memory_usage_ratio_for_aggregate_merging", 0.9); + max_pending_flush_blocks_per_bucket = config.max_pending_flush_blocks_per_grace_aggregate_merging_bucket; + max_allowed_memory_usage_ratio = config.max_allowed_memory_usage_ratio_for_aggregate_merging; // bucket 0 is for in-memory data, it's just a placeholder. buckets.emplace(0, BufferFileStream()); @@ -160,7 +163,7 @@ GraceMergingAggregatedTransform::Status GraceMergingAggregatedTransform::prepare "Output one chunk. rows: {}, bytes: {}, current memory usage: {}", output_chunk.getNumRows(), ReadableSize(output_chunk.bytes()), - ReadableSize(MemoryUtil::getCurrentMemoryUsage())); + ReadableSize(currentThreadGroupMemoryUsage())); total_output_rows += output_chunk.getNumRows(); total_output_blocks++; output.push(std::move(output_chunk)); @@ -189,7 +192,7 @@ GraceMergingAggregatedTransform::Status GraceMergingAggregatedTransform::prepare "Input one new chunk. rows: {}, bytes: {}, current memory usage: {}", input_chunk.getNumRows(), ReadableSize(input_chunk.bytes()), - ReadableSize(MemoryUtil::getCurrentMemoryUsage())); + ReadableSize(currentThreadGroupMemoryUsage())); total_input_rows += input_chunk.getNumRows(); total_input_blocks++; has_input = true; @@ -277,7 +280,7 @@ bool GraceMergingAggregatedTransform::extendBuckets() void GraceMergingAggregatedTransform::rehashDataVariants() { - auto before_memoery_usage = MemoryUtil::getCurrentMemoryUsage(); + auto before_memoery_usage = currentThreadGroupMemoryUsage(); auto converter = currentDataVariantToBlockConverter(false); checkAndSetupCurrentDataVariants(); @@ -318,7 +321,7 @@ void GraceMergingAggregatedTransform::rehashDataVariants() current_bucket_index, getBucketsNum(), ReadableSize(before_memoery_usage), - ReadableSize(MemoryUtil::getCurrentMemoryUsage())); + ReadableSize(currentThreadGroupMemoryUsage())); }; DB::Blocks GraceMergingAggregatedTransform::scatterBlock(const DB::Block & block) @@ -539,7 +542,7 @@ void GraceMergingAggregatedTransform::mergeOneBlock(const DB::Block &block, bool block.info.bucket_num, current_bucket_index, getBucketsNum(), - ReadableSize(MemoryUtil::getCurrentMemoryUsage())); + ReadableSize(currentThreadGroupMemoryUsage())); /// the block could be one read from disk. block.info.bucket_num stores the number of buckets when it was scattered. /// so if the buckets number is not changed since it was scattered, we don't need to scatter it again. @@ -590,11 +593,13 @@ bool GraceMergingAggregatedTransform::isMemoryOverflow() /// More greedy memory usage strategy. if (!current_data_variants) return false; - if (!context->getSettingsRef().max_memory_usage) + + auto memory_soft_limit = DB::CurrentThread::getGroup()->memory_tracker.getSoftLimit(); + if (!memory_soft_limit) return false; - auto max_mem_used = static_cast(context->getSettingsRef().max_memory_usage * max_allowed_memory_usage_ratio); + auto max_mem_used = static_cast(memory_soft_limit * max_allowed_memory_usage_ratio); auto current_result_rows = current_data_variants->size(); - auto current_mem_used = MemoryUtil::getCurrentMemoryUsage(); + auto current_mem_used = currentThreadGroupMemoryUsage(); if (per_key_memory_usage > 0) { if (current_mem_used + per_key_memory_usage * current_result_rows >= max_mem_used) diff --git a/cpp-ch/local-engine/Operator/StreamingAggregatingStep.cpp b/cpp-ch/local-engine/Operator/StreamingAggregatingStep.cpp index 65d77f8e968f0..2235f4cbe45f5 100644 --- a/cpp-ch/local-engine/Operator/StreamingAggregatingStep.cpp +++ b/cpp-ch/local-engine/Operator/StreamingAggregatingStep.cpp @@ -19,8 +19,9 @@ #include #include #include -#include #include +#include +#include #include namespace DB @@ -41,10 +42,11 @@ StreamingAggregatingTransform::StreamingAggregatingTransform(DB::ContextPtr cont , aggregate_columns(params_->params.aggregates_size) , params(params_) { - aggregated_keys_before_evict = context->getConfigRef().getUInt64("aggregated_keys_before_streaming_aggregating_evict", 1024); + auto config = StreamingAggregateConfig::loadFromContext(context); + aggregated_keys_before_evict = config.aggregated_keys_before_streaming_aggregating_evict; aggregated_keys_before_evict = PODArrayUtil::adjustMemoryEfficientSize(aggregated_keys_before_evict); - max_allowed_memory_usage_ratio = context->getConfigRef().getDouble("max_memory_usage_ratio_for_streaming_aggregating", 0.9); - high_cardinality_threshold = context->getConfigRef().getDouble("high_cardinality_threshold_for_streaming_aggregating", 0.8); + max_allowed_memory_usage_ratio = config.max_memory_usage_ratio_for_streaming_aggregating; + high_cardinality_threshold = config.high_cardinality_threshold_for_streaming_aggregating; } StreamingAggregatingTransform::~StreamingAggregatingTransform() @@ -60,7 +62,7 @@ StreamingAggregatingTransform::~StreamingAggregatingTransform() total_clear_data_variants_num, total_aggregate_time, total_convert_data_variants_time, - ReadableSize(MemoryUtil::getCurrentMemoryUsage())); + ReadableSize(currentThreadGroupMemoryUsage())); } StreamingAggregatingTransform::Status StreamingAggregatingTransform::prepare() @@ -82,7 +84,7 @@ StreamingAggregatingTransform::Status StreamingAggregatingTransform::prepare() "Output one chunk. rows: {}, bytes: {}, current memory usage: {}", output_chunk.getNumRows(), ReadableSize(output_chunk.bytes()), - ReadableSize(MemoryUtil::getCurrentMemoryUsage())); + ReadableSize(currentThreadGroupMemoryUsage())); total_output_rows += output_chunk.getNumRows(); total_output_blocks++; if (!output_chunk.getNumRows()) @@ -125,7 +127,7 @@ StreamingAggregatingTransform::Status StreamingAggregatingTransform::prepare() "Input one new chunk. rows: {}, bytes: {}, current memory usage: {}", input_chunk.getNumRows(), ReadableSize(input_chunk.bytes()), - ReadableSize(MemoryUtil::getCurrentMemoryUsage())); + ReadableSize(currentThreadGroupMemoryUsage())); total_input_rows += input_chunk.getNumRows(); total_input_blocks++; has_input = true; @@ -136,10 +138,10 @@ bool StreamingAggregatingTransform::needEvict() { if (input_finished) return true; - if (!context->getSettingsRef().max_memory_usage) + auto memory_soft_limit = DB::CurrentThread::getGroup()->memory_tracker.getSoftLimit(); + if (!memory_soft_limit) return false; - - auto max_mem_used = static_cast(context->getSettingsRef().max_memory_usage * max_allowed_memory_usage_ratio); + auto max_mem_used = static_cast(memory_soft_limit * max_allowed_memory_usage_ratio); auto current_result_rows = data_variants->size(); /// avoid evict empty or too small aggregated results. if (current_result_rows < aggregated_keys_before_evict) @@ -150,7 +152,7 @@ bool StreamingAggregatingTransform::needEvict() if (static_cast(total_output_rows)/total_input_rows > high_cardinality_threshold) return true; - auto current_mem_used = MemoryUtil::getCurrentMemoryUsage(); + auto current_mem_used = currentThreadGroupMemoryUsage(); if (per_key_memory_usage > 0) { /// When we know each key memory usage, we can take a more greedy memory usage strategy diff --git a/cpp-ch/local-engine/Parser/AggregateRelParser.cpp b/cpp-ch/local-engine/Parser/AggregateRelParser.cpp index 0857995571d45..532b4114b8f07 100644 --- a/cpp-ch/local-engine/Parser/AggregateRelParser.cpp +++ b/cpp-ch/local-engine/Parser/AggregateRelParser.cpp @@ -29,6 +29,7 @@ #include #include #include +#include namespace DB { @@ -287,8 +288,8 @@ void AggregateRelParser::addMergingAggregatedStep() settings.max_threads, PODArrayUtil::adjustMemoryEfficientSize(settings.max_block_size), settings.min_hit_rate_to_use_consecutive_keys_optimization); - bool enable_streaming_aggregating = getContext()->getConfigRef().getBool("enable_streaming_aggregating", true); - if (enable_streaming_aggregating) + auto config = StreamingAggregateConfig::loadFromContext(getContext()); + if (config.enable_streaming_aggregating) { params.group_by_two_level_threshold = settings.group_by_two_level_threshold; auto merging_step = std::make_unique(getContext(), plan->getCurrentDataStream(), params, false); @@ -319,8 +320,8 @@ void AggregateRelParser::addCompleteModeAggregatedStep() AggregateDescriptions aggregate_descriptions; buildAggregateDescriptions(aggregate_descriptions); const auto & settings = getContext()->getSettingsRef(); - bool enable_streaming_aggregating = getContext()->getConfigRef().getBool("enable_streaming_aggregating", true); - if (enable_streaming_aggregating) + auto config = StreamingAggregateConfig::loadFromContext(getContext()); + if (config.enable_streaming_aggregating) { Aggregator::Params params( grouping_keys, @@ -397,9 +398,9 @@ void AggregateRelParser::addAggregatingStep() AggregateDescriptions aggregate_descriptions; buildAggregateDescriptions(aggregate_descriptions); const auto & settings = getContext()->getSettingsRef(); - bool enable_streaming_aggregating = getContext()->getConfigRef().getBool("enable_streaming_aggregating", true); - if (enable_streaming_aggregating) + auto config = StreamingAggregateConfig::loadFromContext(getContext()); + if (config.enable_streaming_aggregating) { // Disable spilling to disk. // If group_by_two_level_threshold_bytes != 0, `Aggregator` will use memory usage as a condition to convert diff --git a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp index 151e24c6da456..5aaf006a362e8 100644 --- a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp +++ b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp @@ -82,6 +82,7 @@ #include #include #include +#include #include #include #include @@ -1358,8 +1359,8 @@ std::unique_ptr SerializedPlanParser::createExecutor(DB::QueryPla logger, "clickhouse plan [optimization={}]:\n{}", settings.query_plan_enable_optimizations, PlanUtil::explainPlan(*query_plan)); LOG_DEBUG(logger, "clickhouse pipeline:\n{}", QueryPipelineUtil::explainPipeline(pipeline)); - bool dump_pipeline = context->getConfigRef().getBool("dump_pipeline", false); - return std::make_unique(std::move(query_plan), std::move(pipeline), dump_pipeline); + auto config = ExecutorConfig::loadFromContext(context); + return std::make_unique(std::move(query_plan), std::move(pipeline), config.dump_pipeline); } SerializedPlanParser::SerializedPlanParser(const ContextPtr & context_) : context(context_) diff --git a/cpp-ch/local-engine/Parser/SortRelParser.cpp b/cpp-ch/local-engine/Parser/SortRelParser.cpp index ea29e72d1324c..8fb97d6da5dd3 100644 --- a/cpp-ch/local-engine/Parser/SortRelParser.cpp +++ b/cpp-ch/local-engine/Parser/SortRelParser.cpp @@ -15,10 +15,12 @@ * limitations under the License. */ #include "SortRelParser.h" + +#include #include #include -#include #include +#include namespace DB { @@ -41,11 +43,11 @@ SortRelParser::parse(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, st const auto & sort_rel = rel.sort(); auto sort_descr = parseSortDescription(sort_rel.sorts(), query_plan->getCurrentDataStream().header); SortingStep::Settings settings(*getContext()); - size_t offheap_per_task = getContext()->getConfigRef().getUInt64("off_heap_per_task"); - double spill_mem_ratio = getContext()->getConfigRef().getDouble("spill_mem_ratio", 0.9); - settings.worth_external_sort = [offheap_per_task, spill_mem_ratio]() -> bool + auto config = MemoryConfig::loadFromContext(getContext()); + double spill_mem_ratio = config.spill_mem_ratio; + settings.worth_external_sort = [spill_mem_ratio]() -> bool { - return CurrentMemoryTracker::current_memory() > offheap_per_task * spill_mem_ratio; + return currentThreadGroupMemoryUsageRatio() > spill_mem_ratio; }; auto sorting_step = std::make_unique( query_plan->getCurrentDataStream(), sort_descr, limit, settings, false); diff --git a/cpp-ch/local-engine/Shuffle/CachedShuffleWriter.cpp b/cpp-ch/local-engine/Shuffle/CachedShuffleWriter.cpp index fd6f6fd81b5d1..1ab95abcca48d 100644 --- a/cpp-ch/local-engine/Shuffle/CachedShuffleWriter.cpp +++ b/cpp-ch/local-engine/Shuffle/CachedShuffleWriter.cpp @@ -133,26 +133,20 @@ void CachedShuffleWriter::lazyInitPartitionWriter(Block & input_sample) if (partition_writer) return; -// auto avg_row_size = input_sample.allocatedBytes() / input_sample.rows(); -// auto overhead_memory = std::max(avg_row_size, input_sample.columns() * 16) * options.split_size * options.partition_num; -// auto use_sort_shuffle = overhead_memory > options.spill_threshold * 0.5 || options.partition_num >= 300; - auto use_external_sort_shuffle = options.force_external_sort; - auto use_memory_sort_shuffle = options.force_mermory_sort; - sort_shuffle = use_memory_sort_shuffle || use_external_sort_shuffle; + auto avg_row_size = input_sample.allocatedBytes() / input_sample.rows(); + auto overhead_memory = std::max(avg_row_size, input_sample.columns() * 16) * options.split_size * options.partition_num; + auto use_sort_shuffle = overhead_memory > options.spill_threshold * 0.5 || options.partition_num >= 300; + sort_shuffle = use_sort_shuffle || options.force_memory_sort; if (celeborn_client) { - if (use_external_sort_shuffle) - partition_writer = std::make_unique(this, std::move(celeborn_client)); - else if (use_memory_sort_shuffle) + if (sort_shuffle) partition_writer = std::make_unique(this, std::move(celeborn_client)); else partition_writer = std::make_unique(this, std::move(celeborn_client)); } else { - if (use_external_sort_shuffle) - partition_writer = std::make_unique(this); - else if (use_memory_sort_shuffle) + if (sort_shuffle) partition_writer = std::make_unique(this); else partition_writer = std::make_unique(this); @@ -169,9 +163,4 @@ SplitResult CachedShuffleWriter::stop() return split_result; } -size_t CachedShuffleWriter::evictPartitions() -{ - if (!partition_writer) return 0; - return partition_writer->evictPartitions(true, options.flush_block_buffer_before_evict); -} } \ No newline at end of file diff --git a/cpp-ch/local-engine/Shuffle/CachedShuffleWriter.h b/cpp-ch/local-engine/Shuffle/CachedShuffleWriter.h index e6395c8e47128..6de22f35d9bff 100644 --- a/cpp-ch/local-engine/Shuffle/CachedShuffleWriter.h +++ b/cpp-ch/local-engine/Shuffle/CachedShuffleWriter.h @@ -17,7 +17,7 @@ #pragma once #include #include -#include +#include #include #include #include @@ -46,7 +46,6 @@ class CachedShuffleWriter : public ShuffleWriterBase ~CachedShuffleWriter() override = default; void split(DB::Block & block) override; - size_t evictPartitions() override; SplitResult stop() override; private: diff --git a/cpp-ch/local-engine/Shuffle/NativeSplitter.h b/cpp-ch/local-engine/Shuffle/NativeSplitter.h index 201beb98a6ce9..71d63b61da78e 100644 --- a/cpp-ch/local-engine/Shuffle/NativeSplitter.h +++ b/cpp-ch/local-engine/Shuffle/NativeSplitter.h @@ -26,7 +26,7 @@ #include #include #include -#include +#include #include #include diff --git a/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp b/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp index d02c79e0a5d6d..a2ef0888aeff5 100644 --- a/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp +++ b/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp @@ -18,21 +18,19 @@ #include #include #include -#include #include #include #include #include #include #include -#include #include #include #include #include -#include #include #include +#include #include #include @@ -51,15 +49,32 @@ namespace local_engine { static const String PARTITION_COLUMN_NAME = "partition"; -void PartitionWriter::write(const PartitionInfo & partition_info, DB::Block & block) +int64_t searchLastPartitionIdIndex(ColumnPtr column, size_t start, size_t partition_id) { - /// PartitionWriter::write is alwasy the top frame who occupies evicting_or_writing - if (evicting_or_writing) - throw Exception(ErrorCodes::LOGICAL_ERROR, "PartitionWriter::write is invoked with evicting_or_writing being occupied"); + const auto & int64_column = checkAndGetColumn(*column); + int64_t low = start, high = int64_column.size() - 1; + while (low <= high) + { + int64_t mid = low + (high - low) / 2; + if (int64_column.get64(mid) > partition_id) + high = mid - 1; + else + low = mid + 1; + if (int64_column.get64(high) == partition_id) + return high; + } + return -1; +} - evicting_or_writing = true; - SCOPE_EXIT({ evicting_or_writing = false; }); +bool PartitionWriter::worthToSpill(size_t cache_size) const +{ + return (options->spill_threshold > 0 && cache_size >= options->spill_threshold) || + currentThreadGroupMemoryUsageRatio() > settings.spill_mem_ratio; +} +void PartitionWriter::write(const PartitionInfo & partition_info, DB::Block & block) +{ + /// PartitionWriter::write is alwasy the top frame who occupies evicting_or_writing Stopwatch watch; size_t current_cached_bytes = bytes(); for (size_t partition_id = 0; partition_id < partition_info.partition_num; ++partition_id) @@ -79,60 +94,48 @@ void PartitionWriter::write(const PartitionInfo & partition_info, DB::Block & bl current_cached_bytes += block_buffer->bytes(); /// Only works for celeborn partitiion writer - if (supportsEvictSinglePartition() && options->spill_threshold > 0 && current_cached_bytes >= options->spill_threshold) + if (supportsEvictSinglePartition() && worthToSpill(current_cached_bytes)) { - /// If flush_block_buffer_before_evict is disabled, evict partitions from (last_partition_id+1)%partition_num to partition_id directly without flush, - /// Otherwise flush partition block buffer if it's size is no less than average rows, then evict partitions as above. - if (!options->flush_block_buffer_before_evict) + /// Calculate average rows of each partition block buffer + size_t avg_size = 0; + size_t cnt = 0; + for (size_t i = (last_partition_id + 1) % options->partition_num; i != (partition_id + 1) % options->partition_num; + i = (i + 1) % options->partition_num) { - for (size_t i = (last_partition_id + 1) % options->partition_num; i != (partition_id + 1) % options->partition_num; - i = (i + 1) % options->partition_num) - unsafeEvictSinglePartition(false, false, i); + avg_size += partition_block_buffer[i]->size(); + ++cnt; } - else - { - /// Calculate average rows of each partition block buffer - size_t avg_size = 0; - size_t cnt = 0; - for (size_t i = (last_partition_id + 1) % options->partition_num; i != (partition_id + 1) % options->partition_num; - i = (i + 1) % options->partition_num) - { - avg_size += partition_block_buffer[i]->size(); - ++cnt; - } - avg_size /= cnt; + avg_size /= cnt; - for (size_t i = (last_partition_id + 1) % options->partition_num; i != (partition_id + 1) % options->partition_num; - i = (i + 1) % options->partition_num) - { - bool flush_block_buffer = partition_block_buffer[i]->size() >= avg_size; - current_cached_bytes -= flush_block_buffer ? partition_block_buffer[i]->bytes() + partition_buffer[i]->bytes() - : partition_buffer[i]->bytes(); - unsafeEvictSinglePartition(false, flush_block_buffer, i); - } - // std::cout << "current cached bytes after evict partitions is " << current_cached_bytes << " partition from " - // << (last_partition_id + 1) % options->partition_num << " to " << partition_id << " average size:" << avg_size - // << std::endl; + for (size_t i = (last_partition_id + 1) % options->partition_num; i != (partition_id + 1) % options->partition_num; + i = (i + 1) % options->partition_num) + { + bool flush_block_buffer = partition_block_buffer[i]->size() >= avg_size; + current_cached_bytes -= flush_block_buffer ? partition_block_buffer[i]->bytes() + partition_buffer[i]->bytes() + : partition_buffer[i]->bytes(); + evictSinglePartition(i); } - + // std::cout << "current cached bytes after evict partitions is " << current_cached_bytes << " partition from " + // << (last_partition_id + 1) % options->partition_num << " to " << partition_id << " average size:" << avg_size + // << std::endl; last_partition_id = partition_id; } } /// Only works for local partition writer - if (!supportsEvictSinglePartition() && options->spill_threshold && CurrentMemoryTracker::current_memory() >= options->spill_threshold) - unsafeEvictPartitions(false, options->flush_block_buffer_before_evict); + if (!supportsEvictSinglePartition() && worthToSpill(current_cached_bytes)) + evictPartitions(); shuffle_writer->split_result.total_split_time += watch.elapsedNanoseconds(); } -size_t LocalPartitionWriter::unsafeEvictPartitions(bool for_memory_spill, bool flush_block_buffer) +size_t LocalPartitionWriter::evictPartitions() { size_t res = 0; size_t spilled_bytes = 0; - auto spill_to_file = [this, for_memory_spill, flush_block_buffer, &res, &spilled_bytes]() + auto spill_to_file = [this, &res, &spilled_bytes]() { auto file = getNextSpillFile(); WriteBufferFromFile output(file, shuffle_writer->options.io_buffer_size); @@ -148,12 +151,9 @@ size_t LocalPartitionWriter::unsafeEvictPartitions(bool for_memory_spill, bool f { auto & buffer = partition_buffer[partition_id]; - if (flush_block_buffer) - { - auto & block_buffer = partition_block_buffer[partition_id]; - if (!block_buffer->empty()) - buffer->addBlock(block_buffer->releaseColumns()); - } + auto & block_buffer = partition_block_buffer[partition_id]; + if (!block_buffer->empty()) + buffer->addBlock(block_buffer->releaseColumns()); if (buffer->empty()) continue; @@ -177,24 +177,16 @@ size_t LocalPartitionWriter::unsafeEvictPartitions(bool for_memory_spill, bool f }; Stopwatch spill_time_watch; - if (for_memory_spill && options->throw_if_memory_exceed) - { - // escape memory track from current thread status; add untracked memory limit for create thread object, avoid trigger memory spill again - IgnoreMemoryTracker ignore(settings.spill_memory_overhead); - spill_to_file(); - } - else - { - spill_to_file(); - } + spill_to_file(); shuffle_writer->split_result.total_spill_time += spill_time_watch.elapsedNanoseconds(); shuffle_writer->split_result.total_bytes_spilled += spilled_bytes; + LOG_INFO(logger, "spill shuffle data {} bytes, use spill time {} ms", spilled_bytes, spill_time_watch.elapsedMilliseconds()); return res; } String Spillable::getNextSpillFile() { - auto file_name = std::to_string(split_options.shuffle_id) + "_" + std::to_string(split_options.map_id) + "_" + std::to_string(spill_infos.size()); + auto file_name = std::to_string(static_cast(split_options.shuffle_id)) + "_" + std::to_string(static_cast(split_options.map_id)) + "_" + std::to_string(spill_infos.size()); std::hash hasher; auto hash = hasher(file_name); auto dir_id = hash % split_options.local_dirs_list.size(); @@ -304,32 +296,28 @@ void SortBasedPartitionWriter::write(const PartitionInfo & info, DB::Block & blo current_accumulated_bytes += accumulated_blocks.back().allocatedBytes(); current_accumulated_rows += accumulated_blocks.back().getNumRows(); shuffle_writer->split_result.total_write_time += write_time_watch.elapsedNanoseconds(); - if (options->spill_threshold && CurrentMemoryTracker::current_memory() >= options->spill_threshold) - unsafeEvictPartitions(false, false); + if (worthToSpill(current_accumulated_bytes)) + evictPartitions(); } -LocalPartitionWriter::LocalPartitionWriter(CachedShuffleWriter * shuffle_writer_) : PartitionWriter(shuffle_writer_), Spillable(shuffle_writer_->options) +LocalPartitionWriter::LocalPartitionWriter(CachedShuffleWriter * shuffle_writer_) : PartitionWriter(shuffle_writer_, getLogger("LocalPartitionWriter")), Spillable(shuffle_writer_->options) { } -void LocalPartitionWriter::unsafeStop() +void LocalPartitionWriter::stop() { WriteBufferFromFile output(options->data_file, options->io_buffer_size); auto offsets = mergeSpills(shuffle_writer, output, {partition_block_buffer, partition_buffer}); shuffle_writer->split_result.partition_lengths = offsets; } -void PartitionWriterSettings::loadFromContext(DB::ContextPtr context) -{ - spill_memory_overhead = context->getConfigRef().getUInt64("spill_memory_overhead", 50 << 20); -} - -PartitionWriter::PartitionWriter(CachedShuffleWriter * shuffle_writer_) +PartitionWriter::PartitionWriter(CachedShuffleWriter * shuffle_writer_, LoggerPtr logger_) : shuffle_writer(shuffle_writer_) , options(&shuffle_writer->options) , partition_block_buffer(options->partition_num) , partition_buffer(options->partition_num) , last_partition_id(options->partition_num - 1) + , logger(logger_) { for (size_t partition_id = 0; partition_id < options->partition_num; ++partition_id) { @@ -339,26 +327,6 @@ PartitionWriter::PartitionWriter(CachedShuffleWriter * shuffle_writer_) settings.loadFromContext(SerializedPlanParser::global_context); } -size_t PartitionWriter::evictPartitions(bool for_memory_spill, bool flush_block_buffer) -{ - if (evicting_or_writing) - return 0; - - evicting_or_writing = true; - SCOPE_EXIT({ evicting_or_writing = false; }); - return unsafeEvictPartitions(for_memory_spill, flush_block_buffer); -} - -void PartitionWriter::stop() -{ - if (evicting_or_writing) - throw Exception(ErrorCodes::LOGICAL_ERROR, "PartitionWriter::stop is invoked with evicting_or_writing being occupied"); - - evicting_or_writing = true; - SCOPE_EXIT({ evicting_or_writing = false; }); - return unsafeStop(); -} - size_t PartitionWriter::bytes() const { size_t bytes = 0; @@ -372,7 +340,8 @@ size_t PartitionWriter::bytes() const return bytes; } -size_t MemorySortLocalPartitionWriter::unsafeEvictPartitions(bool for_memory_spill, bool /*flush_block_buffer*/) + +size_t MemorySortLocalPartitionWriter::evictPartitions() { size_t res = 0; size_t spilled_bytes = 0; @@ -456,34 +425,26 @@ size_t MemorySortLocalPartitionWriter::unsafeEvictPartitions(bool for_memory_spi }; Stopwatch spill_time_watch; - if (for_memory_spill && options->throw_if_memory_exceed) - { - // escape memory track from current thread status; add untracked memory limit for create thread object, avoid trigger memory spill again - IgnoreMemoryTracker ignore(settings.spill_memory_overhead); - spill_to_file(); - } - else - { - spill_to_file(); - } + spill_to_file(); shuffle_writer->split_result.total_spill_time += spill_time_watch.elapsedNanoseconds(); shuffle_writer->split_result.total_bytes_spilled += spilled_bytes; + LOG_INFO(logger, "spill shuffle data {} bytes, use spill time {} ms", spilled_bytes, spill_time_watch.elapsedMilliseconds()); return res; } -void MemorySortLocalPartitionWriter::unsafeStop() +void MemorySortLocalPartitionWriter::stop() { - unsafeEvictPartitions(false, false); + evictPartitions(); WriteBufferFromFile output(options->data_file, options->io_buffer_size); auto offsets = mergeSpills(shuffle_writer, output); shuffle_writer->split_result.partition_lengths = offsets; } -size_t MemorySortCelebornPartitionWriter::unsafeEvictPartitions(bool for_memory_spill, bool flush_block_buffer) +size_t MemorySortCelebornPartitionWriter::evictPartitions() { size_t res = 0; size_t spilled_bytes = 0; - auto spill_to_celeborn = [this, for_memory_spill, flush_block_buffer, &res, &spilled_bytes]() + auto spill_to_celeborn = [this, &res, &spilled_bytes]() { Stopwatch serialization_time_watch; @@ -553,202 +514,49 @@ size_t MemorySortCelebornPartitionWriter::unsafeEvictPartitions(bool for_memory_ shuffle_writer->split_result.total_compress_time += compressed_output.getCompressTime(); shuffle_writer->split_result.total_io_time += compressed_output.getWriteTime(); - shuffle_writer->split_result.total_serialize_time += serialization_time_watch.elapsedNanoseconds(); }; Stopwatch spill_time_watch; - if (for_memory_spill && options->throw_if_memory_exceed) - { - // escape memory track from current thread status; add untracked memory limit for create thread object, avoid trigger memory spill again - IgnoreMemoryTracker ignore(settings.spill_memory_overhead); - spill_to_celeborn(); - } - else - { - spill_to_celeborn(); - } - + spill_to_celeborn(); shuffle_writer->split_result.total_spill_time += spill_time_watch.elapsedNanoseconds(); shuffle_writer->split_result.total_bytes_spilled += spilled_bytes; + LOG_INFO(logger, "spill shuffle data {} bytes, use spill time {} ms", spilled_bytes, spill_time_watch.elapsedMilliseconds()); return res; } -void MemorySortCelebornPartitionWriter::unsafeStop() -{ - unsafeEvictPartitions(false, false); -} - -size_t ExternalSortLocalPartitionWriter::unsafeEvictPartitions(bool, bool) +void MemorySortCelebornPartitionWriter::stop() { - // escape memory track - IgnoreMemoryTracker ignore(settings.spill_memory_overhead); - if (accumulated_blocks.empty()) - return 0; - if (max_merge_block_bytes) - { - max_merge_block_size = std::max(max_merge_block_bytes / (current_accumulated_bytes / current_accumulated_rows), 128UL); - } - Stopwatch watch; - MergeSorter sorter(sort_header, std::move(accumulated_blocks), sort_description, max_merge_block_size, 0); - streams.emplace_back(&tmp_data->createStream(sort_header)); - while (auto data = sorter.read()) - { - Block serialized_block = sort_header.cloneWithColumns(data.detachColumns()); - streams.back()->write(serialized_block); - } - streams.back()->finishWriting(); - auto result = current_accumulated_bytes; - current_accumulated_bytes = 0; - current_accumulated_rows = 0; - shuffle_writer->split_result.total_spill_time += watch.elapsedNanoseconds(); - return result; + evictPartitions(); } -std::queue ExternalSortLocalPartitionWriter::mergeDataInMemory() -{ - if (accumulated_blocks.empty()) - return {}; - std::queue result; - MergeSorter sorter(sort_header, std::move(accumulated_blocks), sort_description, max_merge_block_size, 0); - while (auto data = sorter.read()) - { - Block serialized_block = sort_header.cloneWithColumns(data.detachColumns()); - result.push(serialized_block); - } - return result; -} - -ExternalSortLocalPartitionWriter::MergeContext ExternalSortLocalPartitionWriter::prepareMerge() -{ - MergeContext context; - if (options->spill_firstly_before_stop) - unsafeEvictPartitions(false, false); - auto num_input = accumulated_blocks.empty() ? streams.size() : streams.size() + 1; - std::unique_ptr algorithm = std::make_unique( - sort_header, num_input, sort_description, max_merge_block_size, 0, SortingQueueStrategy::Batch); - context.codec = CompressionCodecFactory::instance().get(boost::to_upper_copy(shuffle_writer->options.compress_method), {}); - auto sorted_memory_data = mergeDataInMemory(); - context.merger = std::make_unique(std::move(algorithm), streams, sorted_memory_data, output_header); - return context; -} - -void ExternalSortLocalPartitionWriter::unsafeStop() -{ - // escape memory track - IgnoreMemoryTracker ignore(settings.spill_memory_overhead); - Stopwatch write_time_watch; - // no data to write - if (streams.empty() && accumulated_blocks.empty()) - return; - auto context = prepareMerge(); - WriteBufferFromFile output(options->data_file, options->io_buffer_size); - CompressedWriteBuffer compressed_output(output, context.codec, shuffle_writer->options.io_buffer_size); - NativeWriter native_writer(compressed_output, output_header); - - std::vector partition_length(shuffle_writer->options.partition_num, 0); - size_t current_file_size = 0; - size_t current_partition_raw_size = 0; - size_t current_partition_id = 0; - auto finish_partition_if_needed = [&]() - { - if (!partition_length[current_partition_id]) - { - compressed_output.sync(); - shuffle_writer->split_result.raw_partition_lengths[current_partition_id] = current_partition_raw_size; - partition_length[current_partition_id] = output.count() - current_file_size; - current_file_size = output.count(); - current_partition_id++; - current_partition_raw_size = 0; - } - }; - while (!context.merger->isFinished()) - { - auto result = context.merger->next(); - if (result.empty) - break; - for (auto & item : result.blocks) - { - while (item.second - current_partition_id > 1) - finish_partition_if_needed(); - current_partition_raw_size += native_writer.write(item.first); - } - } - while (shuffle_writer->options.partition_num - current_partition_id > 0) - finish_partition_if_needed(); - shuffle_writer->split_result.partition_lengths = partition_length; - shuffle_writer->split_result.total_write_time += write_time_watch.elapsedNanoseconds(); - shuffle_writer->split_result.total_compress_time += compressed_output.getCompressTime(); - shuffle_writer->split_result.total_io_time += compressed_output.getWriteTime(); -} - -void ExternalSortCelebornPartitionWriter::unsafeStop() -{ - // escape memory track - IgnoreMemoryTracker ignore(settings.spill_memory_overhead); - Stopwatch write_time_watch; - // no data to write - if (streams.empty() && accumulated_blocks.empty()) - return; - auto context = prepareMerge(); - - WriteBufferFromOwnString output; - CompressedWriteBuffer compressed_output(output, context.codec, shuffle_writer->options.io_buffer_size); - NativeWriter native_writer(compressed_output, output_header); - std::vector partition_length(shuffle_writer->options.partition_num, 0); - - while (!context.merger->isFinished()) - { - auto result = context.merger->next(); - if (result.empty) - break; - for (auto & item : result.blocks) - { - shuffle_writer->split_result.raw_partition_lengths[item.second] += native_writer.write(item.first); - compressed_output.sync(); - partition_length[item.second] += output.count(); - Stopwatch push_time; - celeborn_client->pushPartitionData(item.second, output.str().data(), output.str().size()); - shuffle_writer->split_result.total_io_time += push_time.elapsedNanoseconds(); - output.restart(); - } - } - - shuffle_writer->split_result.partition_lengths = partition_length; - shuffle_writer->split_result.total_write_time += write_time_watch.elapsedNanoseconds(); - shuffle_writer->split_result.total_compress_time += compressed_output.getCompressTime(); - shuffle_writer->split_result.total_io_time += compressed_output.getWriteTime(); -} CelebornPartitionWriter::CelebornPartitionWriter(CachedShuffleWriter * shuffleWriter, std::unique_ptr celeborn_client_) - : PartitionWriter(shuffleWriter), celeborn_client(std::move(celeborn_client_)) + : PartitionWriter(shuffleWriter, getLogger("CelebornPartitionWriter")), celeborn_client(std::move(celeborn_client_)) { } -size_t CelebornPartitionWriter::unsafeEvictPartitions(bool for_memory_spill, bool flush_block_buffer) +size_t CelebornPartitionWriter::evictPartitions() { size_t res = 0; for (size_t partition_id = 0; partition_id < options->partition_num; ++partition_id) - res += unsafeEvictSinglePartition(for_memory_spill, flush_block_buffer, partition_id); + res += evictSinglePartition(partition_id); return res; } -size_t CelebornPartitionWriter::unsafeEvictSinglePartition(bool for_memory_spill, bool flush_block_buffer, size_t partition_id) +size_t CelebornPartitionWriter::evictSinglePartition(size_t partition_id) { size_t res = 0; size_t spilled_bytes = 0; - auto spill_to_celeborn = [this, for_memory_spill, flush_block_buffer, partition_id, &res, &spilled_bytes]() + auto spill_to_celeborn = [this,partition_id, &res, &spilled_bytes]() { Stopwatch serialization_time_watch; auto & buffer = partition_buffer[partition_id]; - if (flush_block_buffer) + auto & block_buffer = partition_block_buffer[partition_id]; + if (!block_buffer->empty()) { - auto & block_buffer = partition_block_buffer[partition_id]; - if (!block_buffer->empty()) - { - // std::cout << "flush block buffer for partition:" << partition_id << " rows:" << block_buffer->size() << std::endl; - buffer->addBlock(block_buffer->releaseColumns()); - } + // std::cout << "flush block buffer for partition:" << partition_id << " rows:" << block_buffer->size() << std::endl; + buffer->addBlock(block_buffer->releaseColumns()); } /// Skip empty buffer @@ -781,26 +589,16 @@ size_t CelebornPartitionWriter::unsafeEvictSinglePartition(bool for_memory_spill }; Stopwatch spill_time_watch; - if (for_memory_spill && options->throw_if_memory_exceed) - { - // escape memory track from current thread status; add untracked memory limit for create thread object, avoid trigger memory spill again - IgnoreMemoryTracker ignore(settings.spill_memory_overhead); - spill_to_celeborn(); - } - else - { - spill_to_celeborn(); - } - + spill_to_celeborn(); shuffle_writer->split_result.total_spill_time += spill_time_watch.elapsedNanoseconds(); shuffle_writer->split_result.total_bytes_spilled += spilled_bytes; + LOG_INFO(logger, "spill shuffle data {} bytes, use spill time {} ms", spilled_bytes, spill_time_watch.elapsedMilliseconds()); return res; } -void CelebornPartitionWriter::unsafeStop() +void CelebornPartitionWriter::stop() { - unsafeEvictPartitions(false, true); - + evictPartitions(); for (const auto & length : shuffle_writer->split_result.partition_lengths) shuffle_writer->split_result.total_bytes_written += length; } @@ -831,4 +629,6 @@ size_t Partition::spill(NativeWriter & writer) return written_bytes; } + + } diff --git a/cpp-ch/local-engine/Shuffle/PartitionWriter.h b/cpp-ch/local-engine/Shuffle/PartitionWriter.h index e2c10b0cbd464..0c3c0be50f2d6 100644 --- a/cpp-ch/local-engine/Shuffle/PartitionWriter.h +++ b/cpp-ch/local-engine/Shuffle/PartitionWriter.h @@ -18,13 +18,13 @@ #include #include #include +#include #include #include -#include #include #include #include -#include +#include #include @@ -60,53 +60,43 @@ class Partition size_t cached_bytes = 0; }; -struct PartitionWriterSettings -{ - uint64_t spill_memory_overhead = 0; - - void loadFromContext(DB::ContextPtr context); -}; - class CachedShuffleWriter; using PartitionPtr = std::shared_ptr; class PartitionWriter : boost::noncopyable { public: - explicit PartitionWriter(CachedShuffleWriter * shuffle_writer_); + explicit PartitionWriter(CachedShuffleWriter * shuffle_writer_, LoggerPtr logger_); virtual ~PartitionWriter() = default; virtual String getName() const = 0; virtual void write(const PartitionInfo & info, DB::Block & block); - size_t evictPartitions(bool for_memory_spill = false, bool flush_block_buffer = false); - void stop(); + virtual void stop() = 0; protected: + virtual size_t evictPartitions() = 0; + size_t bytes() const; - virtual size_t unsafeEvictPartitions(bool for_memory_spill, bool flush_block_buffer = false) = 0; + virtual bool worthToSpill(size_t cache_size) const; virtual bool supportsEvictSinglePartition() const { return false; } - virtual size_t unsafeEvictSinglePartition(bool for_memory_spill, bool flush_block_buffer, size_t partition_id) + virtual size_t evictSinglePartition(size_t partition_id) { throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Evict single partition is not supported for {}", getName()); } - virtual void unsafeStop() = 0; - CachedShuffleWriter * shuffle_writer; const SplitOptions * options; - PartitionWriterSettings settings; + MemoryConfig settings; std::vector partition_block_buffer; std::vector partition_buffer; - /// Make sure memory spill doesn't happen while write/stop are executed. - bool evicting_or_writing{false}; - /// Only valid in celeborn partition writer size_t last_partition_id; + LoggerPtr logger = nullptr; }; class Spillable @@ -138,21 +128,21 @@ class LocalPartitionWriter : public PartitionWriter, public Spillable String getName() const override { return "LocalPartitionWriter"; } -protected: - size_t unsafeEvictPartitions(bool for_memory_spill, bool flush_block_buffer) override; - void unsafeStop() override; + size_t evictPartitions() override; + void stop() override; + }; class SortBasedPartitionWriter : public PartitionWriter { -public: - explicit SortBasedPartitionWriter(CachedShuffleWriter * shuffle_writer_) : PartitionWriter(shuffle_writer_) +protected: + explicit SortBasedPartitionWriter(CachedShuffleWriter * shuffle_writer_, LoggerPtr logger) : PartitionWriter(shuffle_writer_, logger) { max_merge_block_size = options->split_size; max_sort_buffer_size = options->max_sort_buffer_size; max_merge_block_bytes = SerializedPlanParser::global_context->getSettings().prefer_external_sort_block_bytes; } - +public: String getName() const override { return "SortBasedPartitionWriter"; } void write(const PartitionInfo & info, DB::Block & block) override; size_t adaptiveBlockSize() @@ -181,80 +171,32 @@ class MemorySortLocalPartitionWriter : public SortBasedPartitionWriter, public S { public: explicit MemorySortLocalPartitionWriter(CachedShuffleWriter* shuffle_writer_) - : SortBasedPartitionWriter(shuffle_writer_), Spillable(shuffle_writer_->options) + : SortBasedPartitionWriter(shuffle_writer_, getLogger("MemorySortLocalPartitionWriter")), Spillable(shuffle_writer_->options) { } ~MemorySortLocalPartitionWriter() override = default; String getName() const override { return "MemorySortLocalPartitionWriter"; } -protected: - size_t unsafeEvictPartitions(bool for_memory_spill, bool flush_block_buffer) override; - void unsafeStop() override; + size_t evictPartitions() override; + void stop() override; }; class MemorySortCelebornPartitionWriter : public SortBasedPartitionWriter { public: explicit MemorySortCelebornPartitionWriter(CachedShuffleWriter* shuffle_writer_, std::unique_ptr celeborn_client_) - : SortBasedPartitionWriter(shuffle_writer_), celeborn_client(std::move(celeborn_client_)) + : SortBasedPartitionWriter(shuffle_writer_, getLogger("MemorySortCelebornPartitionWriter")), celeborn_client(std::move(celeborn_client_)) { } + String getName() const override { return "MemorySortCelebornPartitionWriter"; } ~MemorySortCelebornPartitionWriter() override = default; -protected: - size_t unsafeEvictPartitions(bool for_memory_spill, bool flush_block_buffer) override; - void unsafeStop() override; - -private: - std::unique_ptr celeborn_client; -}; - -class SortedPartitionDataMerger; - -class ExternalSortLocalPartitionWriter : public SortBasedPartitionWriter -{ -public: - struct MergeContext - { - CompressionCodecPtr codec; - std::unique_ptr merger; - }; - - explicit ExternalSortLocalPartitionWriter(CachedShuffleWriter * shuffle_writer_) : SortBasedPartitionWriter(shuffle_writer_) - { - max_merge_block_size = options->split_size; - max_sort_buffer_size = options->max_sort_buffer_size; - max_merge_block_bytes = SerializedPlanParser::global_context->getSettings().prefer_external_sort_block_bytes; - tmp_data = std::make_unique(SerializedPlanParser::global_context->getTempDataOnDisk()); - } - - ~ExternalSortLocalPartitionWriter() override = default; - - String getName() const override { return "ExternalSortLocalPartitionWriter"; } + void stop() override; protected: - size_t unsafeEvictPartitions(bool for_memory_spill, bool flush_block_buffer) override; - /// Prepare for data merging, spill the remaining memory data,and create a merger object. - MergeContext prepareMerge(); - void unsafeStop() override; - std::queue mergeDataInMemory(); - - TemporaryDataOnDiskPtr tmp_data; - std::vector streams; -}; - -class ExternalSortCelebornPartitionWriter : public ExternalSortLocalPartitionWriter -{ -public: - explicit ExternalSortCelebornPartitionWriter(CachedShuffleWriter * shuffle_writer_, std::unique_ptr celeborn_client_) - : ExternalSortLocalPartitionWriter(shuffle_writer_), celeborn_client(std::move(celeborn_client_)) - { - } -protected: - void unsafeStop() override; - + size_t evictPartitions() override; private: std::unique_ptr celeborn_client; }; @@ -266,15 +208,12 @@ class CelebornPartitionWriter : public PartitionWriter ~CelebornPartitionWriter() override = default; String getName() const override { return "CelebornPartitionWriter"; } - + void stop() override; protected: - size_t unsafeEvictPartitions(bool for_memory_spill, bool flush_block_buffer) override; - + size_t evictPartitions() override; bool supportsEvictSinglePartition() const override { return true; } - size_t unsafeEvictSinglePartition(bool for_memory_spill, bool flush_block_buffer, size_t partition_id) override; - - void unsafeStop() override; - + size_t evictSinglePartition(size_t partition_id) override; +private: std::unique_ptr celeborn_client; }; } diff --git a/cpp-ch/local-engine/Shuffle/ShuffleCommon.cpp b/cpp-ch/local-engine/Shuffle/ShuffleCommon.cpp new file mode 100644 index 0000000000000..e0d8c0e84eaa4 --- /dev/null +++ b/cpp-ch/local-engine/Shuffle/ShuffleCommon.cpp @@ -0,0 +1,119 @@ +/* + * 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. + */ +#include +#include +#include +#include + +namespace local_engine +{ +void ColumnsBuffer::add(DB::Block & block, int start, int end) +{ + if (!header) + header = block.cloneEmpty(); + + if (accumulated_columns.empty()) + { + accumulated_columns.reserve(block.columns()); + for (size_t i = 0; i < block.columns(); i++) + { + auto column = block.getColumns()[i]->cloneEmpty(); + column->reserve(prefer_buffer_size); + accumulated_columns.emplace_back(std::move(column)); + } + } + + assert(!accumulated_columns.empty()); + for (size_t i = 0; i < block.columns(); ++i) + { + if (!accumulated_columns[i]->onlyNull()) + { + accumulated_columns[i]->insertRangeFrom(*block.getByPosition(i).column, start, end - start); + } + else + { + accumulated_columns[i]->insertMany(DB::Field(), end - start); + } + } +} + +void ColumnsBuffer::appendSelective( + size_t column_idx, + const DB::Block & source, + const DB::IColumn::Selector & selector, + size_t from, + size_t length) +{ + if (!header) + header = source.cloneEmpty(); + + if (accumulated_columns.empty()) + { + accumulated_columns.reserve(source.columns()); + for (size_t i = 0; i < source.columns(); i++) + { + auto column = source.getColumns()[i]->convertToFullIfNeeded()->cloneEmpty(); + column->reserve(prefer_buffer_size); + accumulated_columns.emplace_back(std::move(column)); + } + } + + if (!accumulated_columns[column_idx]->onlyNull()) + { + accumulated_columns[column_idx]->insertRangeSelective( + *source.getByPosition(column_idx).column->convertToFullIfNeeded(), + selector, + from, + length); + } + else + { + accumulated_columns[column_idx]->insertMany(DB::Field(), length); + } +} + +size_t ColumnsBuffer::size() const +{ + return accumulated_columns.empty() ? 0 : accumulated_columns[0]->size(); +} + +bool ColumnsBuffer::empty() const +{ + return accumulated_columns.empty() ? true : accumulated_columns[0]->empty(); +} + +DB::Block ColumnsBuffer::releaseColumns() +{ + DB::Columns columns(std::make_move_iterator(accumulated_columns.begin()), std::make_move_iterator(accumulated_columns.end())); + accumulated_columns.clear(); + + if (columns.empty()) + return header.cloneEmpty(); + else + return header.cloneWithColumns(columns); +} + +DB::Block ColumnsBuffer::getHeader() +{ + return header; +} + +ColumnsBuffer::ColumnsBuffer(size_t prefer_buffer_size_) + : prefer_buffer_size(prefer_buffer_size_) +{ +} +} \ No newline at end of file diff --git a/cpp-ch/local-engine/Shuffle/ShuffleSplitter.h b/cpp-ch/local-engine/Shuffle/ShuffleCommon.h similarity index 60% rename from cpp-ch/local-engine/Shuffle/ShuffleSplitter.h rename to cpp-ch/local-engine/Shuffle/ShuffleCommon.h index 75edea325c67b..d398362aa4b64 100644 --- a/cpp-ch/local-engine/Shuffle/ShuffleSplitter.h +++ b/cpp-ch/local-engine/Shuffle/ShuffleCommon.h @@ -47,14 +47,8 @@ struct SplitOptions int compress_level; size_t spill_threshold = 300 * 1024 * 1024; std::string hash_algorithm; - bool throw_if_memory_exceed = true; - /// Whether to flush partition_block_buffer in PartitionWriter before evict. - bool flush_block_buffer_before_evict = false; size_t max_sort_buffer_size = 1_GiB; - // Whether to spill firstly before stop external sort shuffle. - bool spill_firstly_before_stop = true; - bool force_external_sort = false; - bool force_mermory_sort = false; + bool force_memory_sort = false; }; class ColumnsBuffer @@ -118,92 +112,6 @@ struct SplitResult } }; -class ShuffleSplitter; -using ShuffleSplitterPtr = std::unique_ptr; -class ShuffleSplitter : public ShuffleWriterBase -{ -public: - inline const static std::vector compress_methods = {"", "ZSTD", "LZ4"}; - - static ShuffleSplitterPtr create(const std::string & short_name, const SplitOptions & options_); - - explicit ShuffleSplitter(const SplitOptions & options); - virtual ~ShuffleSplitter() override - { - if (!stopped) - stop(); - } - - void split(DB::Block & block) override; - virtual void computeAndCountPartitionId(DB::Block &) { } - std::vector getPartitionLength() const { return split_result.partition_lengths; } - void writeIndexFile(); - SplitResult stop() override; - -private: - void init(); - void initOutputIfNeeded(DB::Block & block); - void splitBlockByPartition(DB::Block & block); - void spillPartition(size_t partition_id); - std::string getPartitionTempFile(size_t partition_id); - void mergePartitionFiles(); - std::unique_ptr getPartitionWriteBuffer(size_t partition_id); - -protected: - bool stopped = false; - PartitionInfo partition_info; - std::vector partition_buffer; - std::vector> partition_outputs; - std::vector> partition_write_buffers; - std::vector> partition_cached_write_buffers; - std::vector compressed_buffers; - std::vector output_columns_indicies; - DB::Block output_header; - SplitOptions options; - SplitResult split_result; -}; - -class RoundRobinSplitter : public ShuffleSplitter -{ -public: - static ShuffleSplitterPtr create(const SplitOptions & options); - - explicit RoundRobinSplitter(const SplitOptions & options_); - virtual ~RoundRobinSplitter() override = default; - - void computeAndCountPartitionId(DB::Block & block) override; - -private: - std::unique_ptr selector_builder; -}; - -class HashSplitter : public ShuffleSplitter -{ -public: - static ShuffleSplitterPtr create(const SplitOptions & options); - - explicit HashSplitter(SplitOptions options_); - virtual ~HashSplitter() override = default; - - void computeAndCountPartitionId(DB::Block & block) override; - -private: - std::unique_ptr selector_builder; -}; - -class RangeSplitter : public ShuffleSplitter -{ -public: - static ShuffleSplitterPtr create(const SplitOptions & options); - - explicit RangeSplitter(const SplitOptions & options_); - virtual ~RangeSplitter() override = default; - - void computeAndCountPartitionId(DB::Block & block) override; - -private: - std::unique_ptr selector_builder; -}; struct SplitterHolder { std::unique_ptr splitter; diff --git a/cpp-ch/local-engine/Shuffle/ShuffleSplitter.cpp b/cpp-ch/local-engine/Shuffle/ShuffleSplitter.cpp deleted file mode 100644 index 9baf3c4692c81..0000000000000 --- a/cpp-ch/local-engine/Shuffle/ShuffleSplitter.cpp +++ /dev/null @@ -1,424 +0,0 @@ -/* - * 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. - */ -#include "ShuffleSplitter.h" -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace local_engine -{ - -void ShuffleSplitter::split(DB::Block & block) -{ - if (block.rows() == 0) - { - return; - } - initOutputIfNeeded(block); - computeAndCountPartitionId(block); - Stopwatch split_time_watch; - block = convertAggregateStateInBlock(block); - split_result.total_split_time += split_time_watch.elapsedNanoseconds(); - splitBlockByPartition(block); -} - -SplitResult ShuffleSplitter::stop() -{ - // spill all buffers - Stopwatch watch; - for (size_t i = 0; i < options.partition_num; i++) - { - spillPartition(i); - partition_outputs[i]->flush(); - partition_write_buffers[i]->sync(); - } - for (auto * item : compressed_buffers) - { - if (item) - { - split_result.total_compress_time += item->getCompressTime(); - split_result.total_io_time += item->getWriteTime(); - } - } - split_result.total_serialize_time = split_result.total_spill_time - split_result.total_compress_time - split_result.total_io_time; - partition_outputs.clear(); - partition_cached_write_buffers.clear(); - partition_write_buffers.clear(); - mergePartitionFiles(); - split_result.total_write_time += watch.elapsedNanoseconds(); - stopped = true; - return split_result; -} - -void ShuffleSplitter::initOutputIfNeeded(Block & block) -{ - if (output_header.columns() == 0) [[unlikely]] - { - output_header = block.cloneEmpty(); - if (output_columns_indicies.empty()) - { - output_header = block.cloneEmpty(); - for (size_t i = 0; i < block.columns(); ++i) - { - output_columns_indicies.push_back(i); - } - } - else - { - ColumnsWithTypeAndName cols; - for (const auto & index : output_columns_indicies) - { - cols.push_back(block.getByPosition(index)); - } - output_header = DB::Block(cols); - } - } -} - -void ShuffleSplitter::splitBlockByPartition(DB::Block & block) -{ - Stopwatch split_time_watch; - DB::Block out_block; - for (size_t col = 0; col < output_header.columns(); ++col) - { - out_block.insert(block.getByPosition(output_columns_indicies[col])); - } - for (size_t col = 0; col < output_header.columns(); ++col) - { - for (size_t j = 0; j < partition_info.partition_num; ++j) - { - size_t from = partition_info.partition_start_points[j]; - size_t length = partition_info.partition_start_points[j + 1] - from; - if (length == 0) - continue; // no data for this partition continue; - partition_buffer[j]->appendSelective(col, out_block, partition_info.partition_selector, from, length); - } - } - split_result.total_split_time += split_time_watch.elapsedNanoseconds(); - - for (size_t i = 0; i < options.partition_num; ++i) - { - auto & buffer = partition_buffer[i]; - if (buffer->size() >= options.split_size) - { - spillPartition(i); - } - } -} - -ShuffleSplitter::ShuffleSplitter(const SplitOptions & options_) : options(options_) -{ - init(); -} - -void ShuffleSplitter::init() -{ - partition_buffer.resize(options.partition_num); - partition_outputs.resize(options.partition_num); - partition_write_buffers.resize(options.partition_num); - partition_cached_write_buffers.resize(options.partition_num); - split_result.partition_lengths.resize(options.partition_num); - split_result.raw_partition_lengths.resize(options.partition_num); - for (size_t partition_i = 0; partition_i < options.partition_num; ++partition_i) - { - partition_buffer[partition_i] = std::make_shared(options.split_size); - split_result.partition_lengths[partition_i] = 0; - split_result.raw_partition_lengths[partition_i] = 0; - } -} - -void ShuffleSplitter::spillPartition(size_t partition_id) -{ - Stopwatch watch; - if (!partition_outputs[partition_id]) - { - partition_write_buffers[partition_id] = getPartitionWriteBuffer(partition_id); - partition_outputs[partition_id] - = std::make_unique(*partition_write_buffers[partition_id], output_header); - } - DB::Block result = partition_buffer[partition_id]->releaseColumns(); - if (result.rows() > 0) - { - partition_outputs[partition_id]->write(result); - } - split_result.total_spill_time += watch.elapsedNanoseconds(); - split_result.total_bytes_spilled += result.bytes(); -} - -void ShuffleSplitter::mergePartitionFiles() -{ - Stopwatch merge_io_time; - DB::WriteBufferFromFile data_write_buffer = DB::WriteBufferFromFile(options.data_file); - std::string buffer; - size_t buffer_size = options.io_buffer_size; - buffer.reserve(buffer_size); - for (size_t i = 0; i < options.partition_num; ++i) - { - auto file = getPartitionTempFile(i); - DB::ReadBufferFromFile reader = DB::ReadBufferFromFile(file, options.io_buffer_size); - while (reader.next()) - { - auto bytes = reader.readBig(buffer.data(), buffer_size); - data_write_buffer.write(buffer.data(), bytes); - split_result.partition_lengths[i] += bytes; - split_result.total_bytes_written += bytes; - } - reader.close(); - std::filesystem::remove(file); - } - split_result.total_io_time += merge_io_time.elapsedNanoseconds(); - data_write_buffer.close(); -} - - -ShuffleSplitterPtr ShuffleSplitter::create(const std::string & short_name, const SplitOptions & options_) -{ - if (short_name == "rr") - return RoundRobinSplitter::create(options_); - else if (short_name == "hash") - return HashSplitter::create(options_); - else if (short_name == "single") - { - SplitOptions options = options_; - options.partition_num = 1; - return RoundRobinSplitter::create(options); - } - else if (short_name == "range") - return RangeSplitter::create(options_); - else - throw std::runtime_error("unsupported splitter " + short_name); -} - -std::string ShuffleSplitter::getPartitionTempFile(size_t partition_id) -{ - auto file_name = std::to_string(options.shuffle_id) + "_" + std::to_string(options.map_id) + "_" + std::to_string(partition_id); - std::hash hasher; - auto hash = hasher(file_name); - auto dir_id = hash % options.local_dirs_list.size(); - auto sub_dir_id = (hash / options.local_dirs_list.size()) % options.num_sub_dirs; - - std::string dir = std::filesystem::path(options.local_dirs_list[dir_id]) / std::format("{:02x}", sub_dir_id); - if (!std::filesystem::exists(dir)) - std::filesystem::create_directories(dir); - return std::filesystem::path(dir) / file_name; -} - -std::unique_ptr ShuffleSplitter::getPartitionWriteBuffer(size_t partition_id) -{ - auto file = getPartitionTempFile(partition_id); - if (partition_cached_write_buffers[partition_id] == nullptr) - partition_cached_write_buffers[partition_id] - = std::make_unique(file, options.io_buffer_size, O_CREAT | O_WRONLY | O_APPEND); - if (!options.compress_method.empty() - && std::find(compress_methods.begin(), compress_methods.end(), options.compress_method) != compress_methods.end()) - { - auto codec = DB::CompressionCodecFactory::instance().get(boost::to_upper_copy(options.compress_method), {}); - auto compressed = std::make_unique(*partition_cached_write_buffers[partition_id], codec); - compressed_buffers.emplace_back(compressed.get()); - return compressed; - } - else - { - return std::move(partition_cached_write_buffers[partition_id]); - } -} - -void ShuffleSplitter::writeIndexFile() -{ - auto index_file = options.data_file + ".index"; - auto writer = std::make_unique(index_file, options.io_buffer_size, O_CREAT | O_WRONLY | O_TRUNC); - for (auto len : split_result.partition_lengths) - { - DB::writeIntText(len, *writer); - DB::writeChar('\n', *writer); - } -} - -void ColumnsBuffer::add(DB::Block & block, int start, int end) -{ - if (!header) - header = block.cloneEmpty(); - - if (accumulated_columns.empty()) - { - accumulated_columns.reserve(block.columns()); - for (size_t i = 0; i < block.columns(); i++) - { - auto column = block.getColumns()[i]->cloneEmpty(); - column->reserve(prefer_buffer_size); - accumulated_columns.emplace_back(std::move(column)); - } - } - - assert(!accumulated_columns.empty()); - for (size_t i = 0; i < block.columns(); ++i) - { - if (!accumulated_columns[i]->onlyNull()) - { - accumulated_columns[i]->insertRangeFrom(*block.getByPosition(i).column, start, end - start); - } - else - { - accumulated_columns[i]->insertMany(DB::Field(), end - start); - } - } -} - -void ColumnsBuffer::appendSelective( - size_t column_idx, const DB::Block & source, const DB::IColumn::Selector & selector, size_t from, size_t length) -{ - if (!header) - header = source.cloneEmpty(); - - if (accumulated_columns.empty()) - { - accumulated_columns.reserve(source.columns()); - for (size_t i = 0; i < source.columns(); i++) - { - auto column = source.getColumns()[i]->convertToFullIfNeeded()->cloneEmpty(); - column->reserve(prefer_buffer_size); - accumulated_columns.emplace_back(std::move(column)); - } - } - - if (!accumulated_columns[column_idx]->onlyNull()) - { - accumulated_columns[column_idx]->insertRangeSelective( - *source.getByPosition(column_idx).column->convertToFullIfNeeded(), selector, from, length); - } - else - { - accumulated_columns[column_idx]->insertMany(DB::Field(), length); - } -} - -size_t ColumnsBuffer::size() const -{ - return accumulated_columns.empty() ? 0 : accumulated_columns[0]->size(); -} - -bool ColumnsBuffer::empty() const -{ - return accumulated_columns.empty() ? true : accumulated_columns[0]->empty(); -} - -DB::Block ColumnsBuffer::releaseColumns() -{ - DB::Columns columns(std::make_move_iterator(accumulated_columns.begin()), std::make_move_iterator(accumulated_columns.end())); - accumulated_columns.clear(); - - if (columns.empty()) - return header.cloneEmpty(); - else - return header.cloneWithColumns(columns); -} - -DB::Block ColumnsBuffer::getHeader() -{ - return header; -} - -ColumnsBuffer::ColumnsBuffer(size_t prefer_buffer_size_) : prefer_buffer_size(prefer_buffer_size_) -{ -} - -RoundRobinSplitter::RoundRobinSplitter(const SplitOptions & options_) : ShuffleSplitter(options_) -{ - Poco::StringTokenizer output_column_tokenizer(options_.out_exprs, ","); - for (auto iter = output_column_tokenizer.begin(); iter != output_column_tokenizer.end(); ++iter) - { - output_columns_indicies.push_back(std::stoi(*iter)); - } - selector_builder = std::make_unique(options.partition_num); -} - -void RoundRobinSplitter::computeAndCountPartitionId(DB::Block & block) -{ - Stopwatch watch; - partition_info = selector_builder->build(block); - split_result.total_compute_pid_time += watch.elapsedNanoseconds(); -} - -ShuffleSplitterPtr RoundRobinSplitter::create(const SplitOptions & options_) -{ - return std::make_unique(options_); -} - -HashSplitter::HashSplitter(SplitOptions options_) : ShuffleSplitter(options_) -{ - Poco::StringTokenizer exprs_list(options_.hash_exprs, ","); - std::vector hash_fields; - for (auto iter = exprs_list.begin(); iter != exprs_list.end(); ++iter) - { - hash_fields.push_back(std::stoi(*iter)); - } - - Poco::StringTokenizer output_column_tokenizer(options_.out_exprs, ","); - for (auto iter = output_column_tokenizer.begin(); iter != output_column_tokenizer.end(); ++iter) - { - output_columns_indicies.push_back(std::stoi(*iter)); - } - - selector_builder = std::make_unique(options.partition_num, hash_fields, options_.hash_algorithm); -} - -std::unique_ptr HashSplitter::create(const SplitOptions & options_) -{ - return std::make_unique(options_); -} - -void HashSplitter::computeAndCountPartitionId(DB::Block & block) -{ - Stopwatch watch; - partition_info = selector_builder->build(block); - split_result.total_compute_pid_time += watch.elapsedNanoseconds(); -} - -ShuffleSplitterPtr RangeSplitter::create(const SplitOptions & options_) -{ - return std::make_unique(options_); -} - -RangeSplitter::RangeSplitter(const SplitOptions & options_) : ShuffleSplitter(options_) -{ - Poco::StringTokenizer output_column_tokenizer(options_.out_exprs, ","); - for (auto iter = output_column_tokenizer.begin(); iter != output_column_tokenizer.end(); ++iter) - { - output_columns_indicies.push_back(std::stoi(*iter)); - } - selector_builder = std::make_unique(options.hash_exprs, options.partition_num); -} - -void RangeSplitter::computeAndCountPartitionId(DB::Block & block) -{ - Stopwatch watch; - partition_info = selector_builder->build(block); - split_result.total_compute_pid_time += watch.elapsedNanoseconds(); -} -} diff --git a/cpp-ch/local-engine/Shuffle/SortedPartitionDataMerger.cpp b/cpp-ch/local-engine/Shuffle/SortedPartitionDataMerger.cpp deleted file mode 100644 index 9c4ae6bf4680f..0000000000000 --- a/cpp-ch/local-engine/Shuffle/SortedPartitionDataMerger.cpp +++ /dev/null @@ -1,139 +0,0 @@ -/* - * 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. - */ - -#include "SortedPartitionDataMerger.h" -using namespace DB; -namespace local_engine -{ -SortedPartitionDataMerger::SortedPartitionDataMerger( - std::unique_ptr algorithm, - const std::vector & streams, - std::queue & extra_blocks_in_memory_, - const Block & output_header_) -{ - merging_algorithm = std::move(algorithm); - IMergingAlgorithm::Inputs initial_inputs; - bool use_in_memory_data = !extra_blocks_in_memory_.empty(); - for (auto * stream : streams) - { - Block data = stream->read(); - IMergingAlgorithm::Input input; - input.set({data.getColumns(), data.rows()}); - initial_inputs.emplace_back(std::move(input)); - } - if (use_in_memory_data) - { - IMergingAlgorithm::Input input; - const auto & data = extra_blocks_in_memory_.front(); - input.set({data.getColumns(), data.rows()}); - initial_inputs.emplace_back(std::move(input)); - extra_blocks_in_memory_.pop(); - } - for (int i = 0; i < streams.size(); ++i) - sources.emplace_back(std::make_shared(streams[i], i)); - if (use_in_memory_data) - sources.emplace_back(std::make_shared(extra_blocks_in_memory_, sources.size())); - output_header = output_header_; - merging_algorithm->initialize(std::move(initial_inputs)); -} - -int64_t searchLastPartitionIdIndex(ColumnPtr column, size_t start, size_t partition_id) -{ - const auto & int64_column = checkAndGetColumn(*column); - int64_t low = start, high = int64_column.size() - 1; - while (low <= high) - { - int64_t mid = low + (high - low) / 2; - if (int64_column.get64(mid) > partition_id) - high = mid - 1; - else - low = mid + 1; - if (int64_column.get64(high) == partition_id) - return high; - } - return -1; -} - -SortedPartitionDataMerger::Result SortedPartitionDataMerger::next() -{ - if (finished) - return Result{.empty = true}; - Chunk chunk; - while (true) - { - auto result = merging_algorithm->merge(); - - if (result.required_source >= 0) - { - auto stream = sources[result.required_source]; - auto block = stream->next(); - if (block) - { - IMergingAlgorithm::Input input; - input.set({block.getColumns(), block.rows()}); - merging_algorithm->consume(input, stream->getPartitionId()); - } - } - if (result.chunk.getNumRows() > 0) - { - chunk = std::move(result.chunk); - break; - } - if (result.is_finished) - { - finished = true; - if (chunk.getNumRows() == 0) - return Result{.empty = true}; - break; - } - } - Result partitions; - size_t row_idx = 0; - Columns result_columns; - result_columns.reserve(chunk.getColumns().size() - 1); - for (size_t i = 0; i < chunk.getColumns().size() - 1; ++i) - result_columns.push_back(chunk.getColumns()[i]); - while (row_idx < chunk.getNumRows()) - { - auto idx = searchLastPartitionIdIndex(chunk.getColumns().back(), row_idx, current_partition_id); - if (idx >= 0) - { - if (row_idx == 0 && idx == chunk.getNumRows() - 1) - { - partitions.blocks.emplace_back(output_header.cloneWithColumns(result_columns), current_partition_id); - break; - } - else - { - Columns cut_columns; - cut_columns.reserve(result_columns.size()); - for (auto & result_column : result_columns) - cut_columns.push_back(result_column->cut(row_idx, idx - row_idx + 1)); - partitions.blocks.emplace_back(output_header.cloneWithColumns(cut_columns), current_partition_id); - row_idx = idx + 1; - if (idx != chunk.getNumRows() - 1) - current_partition_id++; - } - } - else - { - current_partition_id++; - } - } - return partitions; -} -} \ No newline at end of file diff --git a/cpp-ch/local-engine/Shuffle/SortedPartitionDataMerger.h b/cpp-ch/local-engine/Shuffle/SortedPartitionDataMerger.h deleted file mode 100644 index e38f58647e963..0000000000000 --- a/cpp-ch/local-engine/Shuffle/SortedPartitionDataMerger.h +++ /dev/null @@ -1,93 +0,0 @@ -/* - * 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. - */ -#pragma once -#include -#include -#include - - -namespace local_engine -{ - -int64_t searchLastPartitionIdIndex(DB::ColumnPtr column, size_t start, size_t partition_id); -class SortedPartitionDataMerger; -using SortedPartitionDataMergerPtr = std::unique_ptr; -class SortedPartitionDataMerger -{ -public: - struct Result - { - bool empty = false; - std::vector> blocks; - }; - - class SortedData - { - public: - SortedData(DB::TemporaryFileStream * stream, size_t partitionId) : stream(stream), partition_id(partitionId) { } - SortedData(const std::queue & blocksInMemory, size_t partitionId) - : blocks_in_memory(blocksInMemory), partition_id(partitionId) - { - } - DB::Block next() - { - if (stream) - { - auto data = stream->read(); - end = !data; - return data; - } - if (!blocks_in_memory.empty()) - { - auto block = blocks_in_memory.front(); - blocks_in_memory.pop(); - return block; - } - return {}; - } - bool isEnd() const - { - if (stream) - return stream->isEof(); - return blocks_in_memory.empty() || end; - } - size_t getPartitionId() const { return partition_id; } - - private: - DB::TemporaryFileStream * stream = nullptr; - std::queue blocks_in_memory; - size_t partition_id; - bool end = false; - }; - - SortedPartitionDataMerger( - std::unique_ptr algorithm, - const std::vector & streams, - std::queue & extra_blocks_in_memory, - const DB::Block & output_header); - Result next(); - bool isFinished() const { return finished; } - -private: - std::unique_ptr merging_algorithm; - std::vector> sources; - DB::Block output_header; - bool finished = false; - size_t current_partition_id = 0; -}; - -} \ No newline at end of file diff --git a/cpp-ch/local-engine/Storages/Mergetree/SparkMergeTreeWriter.cpp b/cpp-ch/local-engine/Storages/Mergetree/SparkMergeTreeWriter.cpp index 403b845147fa4..da92eeba83ce4 100644 --- a/cpp-ch/local-engine/Storages/Mergetree/SparkMergeTreeWriter.cpp +++ b/cpp-ch/local-engine/Storages/Mergetree/SparkMergeTreeWriter.cpp @@ -26,6 +26,7 @@ #include #include #include +#include namespace CurrentMetrics @@ -149,7 +150,6 @@ bool SparkMergeTreeWriter::blockToPart(Block & block) new_parts.emplace_back(writeTempPartAndFinalize(item, metadata_snapshot).part); part_num++; - manualFreeMemory(before_write_memory); /// Reset earlier to free memory item.block.clear(); item.partition.clear(); @@ -158,36 +158,6 @@ bool SparkMergeTreeWriter::blockToPart(Block & block) return true; } -void SparkMergeTreeWriter::manualFreeMemory(size_t before_write_memory) -{ - // If mergetree disk is not local fs, like remote fs s3 or hdfs, - // it may alloc memory in current thread, and free on global thread. - // Now, wo have not idea to clear global memory by used spark thread tracker. - // So we manually correct the memory usage. - if (isRemoteStorage && insert_without_local_storage) - return; - - auto disk = storage->getStoragePolicy()->getAnyDisk(); - std::lock_guard lock(memory_mutex); - auto * memory_tracker = CurrentThread::getMemoryTracker(); - if (memory_tracker && CurrentMemoryTracker::before_free) - { - CurrentThread::flushUntrackedMemory(); - const size_t ch_alloc = memory_tracker->get(); - if (disk->getName().contains("s3") && context->getSettings().s3_allow_parallel_part_upload && ch_alloc > before_write_memory) - { - const size_t diff_ch_alloc = before_write_memory - ch_alloc; - memory_tracker->adjustWithUntrackedMemory(diff_ch_alloc); - } - - const size_t spark_alloc = CurrentMemoryTracker::current_memory(); - const size_t diff_alloc = spark_alloc - memory_tracker->get(); - - if (diff_alloc > 0) - CurrentMemoryTracker::before_free(diff_alloc); - } -} - void SparkMergeTreeWriter::finalize() { chunkToPart(squashing->flush()); @@ -523,15 +493,14 @@ void SparkMergeTreeWriter::checkAndMerge(bool force) { for (const auto & selected_part : prepare_merge_parts) tmp_parts.emplace(selected_part->name); - + // check thread group initailized in task thread + currentThreadGroupMemoryUsage(); thread_pool.scheduleOrThrow( [this, prepare_merge_parts, thread_group = CurrentThread::getGroup()]() -> void { Stopwatch watch; - setThreadName("InsertWithMerge"); - ThreadStatus thread_status; - thread_status.attachToGroup(thread_group); - + CurrentThread::detachFromGroupIfNotDetached(); + CurrentThread::attachToGroup(thread_group); size_t before_size = 0; size_t after_size = 0; for (const auto & prepare_merge_part : prepare_merge_parts) diff --git a/cpp-ch/local-engine/Storages/Mergetree/SparkMergeTreeWriter.h b/cpp-ch/local-engine/Storages/Mergetree/SparkMergeTreeWriter.h index 269b0352c0566..3fd1491ea3891 100644 --- a/cpp-ch/local-engine/Storages/Mergetree/SparkMergeTreeWriter.h +++ b/cpp-ch/local-engine/Storages/Mergetree/SparkMergeTreeWriter.h @@ -73,7 +73,6 @@ class SparkMergeTreeWriter void checkAndMerge(bool force = false); void safeEmplaceBackPart(DB::MergeTreeDataPartPtr); void safeAddPart(DB::MergeTreeDataPartPtr); - void manualFreeMemory(size_t before_write_memory); void saveMetadata(); void commitPartToRemoteStorageIfNeeded(); void finalizeMerge(); @@ -99,7 +98,7 @@ class SparkMergeTreeWriter DB::Block header; bool merge_after_insert; bool insert_without_local_storage; - FreeThreadPool thread_pool; + ThreadPool thread_pool; size_t merge_min_size = 1024 * 1024 * 1024; size_t merge_limit_parts = 10; std::mutex memory_mutex; diff --git a/cpp-ch/local-engine/Storages/StorageMergeTreeFactory.cpp b/cpp-ch/local-engine/Storages/StorageMergeTreeFactory.cpp index 0731ac92cd078..c59d6ddb4bd41 100644 --- a/cpp-ch/local-engine/Storages/StorageMergeTreeFactory.cpp +++ b/cpp-ch/local-engine/Storages/StorageMergeTreeFactory.cpp @@ -16,6 +16,8 @@ */ #include "StorageMergeTreeFactory.h" +#include + namespace local_engine { @@ -67,14 +69,12 @@ DataPartsVector StorageMergeTreeFactory::getDataPartsByNames(const StorageID & i { DataPartsVector res; auto table_name = getTableName(id, snapshot_id); - + auto config = MergeTreeConfig::loadFromContext(SerializedPlanParser::global_context); std::lock_guard lock(datapart_mutex); std::unordered_set missing_names; if (!datapart_map->has(table_name)) [[unlikely]] { - auto cache = std::make_shared>( - SerializedPlanParser::global_context->getConfigRef().getInt64("table_part_metadata_cache_max_count", 1000000) - ); + auto cache = std::make_shared>(config.table_part_metadata_cache_max_count); datapart_map->add(table_name, cache); } diff --git a/cpp-ch/local-engine/Storages/StorageMergeTreeFactory.h b/cpp-ch/local-engine/Storages/StorageMergeTreeFactory.h index d7bcb93c07d7d..f372175bb02ce 100644 --- a/cpp-ch/local-engine/Storages/StorageMergeTreeFactory.h +++ b/cpp-ch/local-engine/Storages/StorageMergeTreeFactory.h @@ -15,6 +15,7 @@ * limitations under the License. */ #pragma once +#include #include #include #include @@ -34,11 +35,11 @@ class StorageMergeTreeFactory static DataPartsVector getDataPartsByNames(const StorageID & id, const String & snapshot_id, std::unordered_set part_name); static void init_cache_map() { + auto config = MergeTreeConfig::loadFromContext(SerializedPlanParser::global_context); auto & storage_map_v = storage_map; if (!storage_map_v) { - storage_map_v = std::make_unique>( - SerializedPlanParser::global_context->getConfigRef().getInt64("table_metadata_cache_max_count", 100)); + storage_map_v = std::make_unique>(config.table_metadata_cache_max_count); } else { @@ -48,7 +49,7 @@ class StorageMergeTreeFactory if (!datapart_map_v) { datapart_map_v = std::make_unique>>>( - SerializedPlanParser::global_context->getConfigRef().getInt64("table_metadata_cache_max_count", 100)); + config.table_metadata_cache_max_count); } else { diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.cpp index 0221afd885141..00acaf58398cc 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.cpp @@ -36,6 +36,7 @@ #include #endif +#include #include namespace DB @@ -81,8 +82,8 @@ FormatFilePtr FormatFileUtil::createFile( #if USE_PARQUET if (file.has_parquet()) { - bool useLocalFormat = context->getConfigRef().getBool("use_local_format", false); - return std::make_shared(context, file, read_buffer_builder, useLocalFormat); + auto config = ExecutorConfig::loadFromContext(context); + return std::make_shared(context, file, read_buffer_builder, config.use_local_format); } #endif diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp index e73ca8ecee2bb..da15890070b09 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp @@ -48,6 +48,7 @@ #include #include #include +#include #include #include #include @@ -211,7 +212,7 @@ class HDFSFileReadBufferBuilder : public ReadBufferBuilder std::unique_ptr build(const substrait::ReadRel::LocalFiles::FileOrFiles & file_info, bool set_read_util_position) override { - bool enable_async_io = context->getConfigRef().getBool("hdfs.enable_async_io", true); + auto config = HdfsConfig::loadFromContext(context); Poco::URI file_uri(file_info.uri_file()); std::string uri_path = "hdfs://" + file_uri.getHost(); if (file_uri.getPort()) @@ -233,7 +234,7 @@ class HDFSFileReadBufferBuilder : public ReadBufferBuilder auto read_buffer_impl = std::make_unique( uri_path, file_uri.getPath(), context->getConfigRef(), read_settings, start_end_pos.second, true); - if (enable_async_io) + if (config.hdfs_async) { auto & pool_reader = context->getThreadPoolReader(DB::FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); read_buffer = std::make_unique(pool_reader, read_settings, std::move(read_buffer_impl)); @@ -249,7 +250,7 @@ class HDFSFileReadBufferBuilder : public ReadBufferBuilder { auto read_buffer_impl = std::make_unique(uri_path, file_uri.getPath(), context->getConfigRef(), read_settings, 0, true); - if (enable_async_io) + if (config.hdfs_async) { read_buffer = std::make_unique( context->getThreadPoolReader(DB::FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER), @@ -380,14 +381,15 @@ class S3FileReadBufferBuilder : public ReadBufferBuilder public: explicit S3FileReadBufferBuilder(DB::ContextPtr context_) : ReadBufferBuilder(context_) { + auto config = S3Config::loadFromContext(context); new_settings = context->getReadSettings(); - new_settings.enable_filesystem_cache = context->getConfigRef().getBool("s3.local_cache.enabled", false); + new_settings.enable_filesystem_cache = config.s3_local_cache_enabled; if (new_settings.enable_filesystem_cache) { DB::FileCacheSettings file_cache_settings; - file_cache_settings.max_size = static_cast(context->getConfigRef().getUInt64("s3.local_cache.max_size", 100L << 30)); - auto cache_base_path = context->getConfigRef().getString("s3.local_cache.cache_path", "/tmp/gluten/local_cache"); + file_cache_settings.max_size = config.s3_local_cache_max_size; + auto cache_base_path = config.s3_local_cache_cache_path; if (!fs::exists(cache_base_path)) fs::create_directories(cache_base_path); diff --git a/cpp-ch/local-engine/jni/ReservationListenerWrapper.cpp b/cpp-ch/local-engine/jni/ReservationListenerWrapper.cpp deleted file mode 100644 index dad0ecf66ed76..0000000000000 --- a/cpp-ch/local-engine/jni/ReservationListenerWrapper.cpp +++ /dev/null @@ -1,75 +0,0 @@ -/* - * 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. - */ -#include "ReservationListenerWrapper.h" -#include -#include - -namespace local_engine -{ -jclass ReservationListenerWrapper::reservation_listener_class = nullptr; -jmethodID ReservationListenerWrapper::reservation_listener_reserve = nullptr; -jmethodID ReservationListenerWrapper::reservation_listener_reserve_or_throw = nullptr; -jmethodID ReservationListenerWrapper::reservation_listener_unreserve = nullptr; -jmethodID ReservationListenerWrapper::reservation_listener_currentMemory = nullptr; - -ReservationListenerWrapper::ReservationListenerWrapper(jobject listener_) : listener(listener_) -{ -} - -ReservationListenerWrapper::~ReservationListenerWrapper() -{ - GET_JNIENV(env) - env->DeleteGlobalRef(listener); - CLEAN_JNIENV -} - -void ReservationListenerWrapper::reserve(int64_t size) -{ - GET_JNIENV(env) - safeCallVoidMethod(env, listener, reservation_listener_reserve, size); - CLEAN_JNIENV -} - -void ReservationListenerWrapper::reserveOrThrow(int64_t size) -{ - GET_JNIENV(env) - safeCallVoidMethod(env, listener, reservation_listener_reserve_or_throw, size); - CLEAN_JNIENV -} - -void ReservationListenerWrapper::free(int64_t size) -{ - GET_JNIENV(env) - safeCallVoidMethod(env, listener, reservation_listener_unreserve, size); - CLEAN_JNIENV -} - -void ReservationListenerWrapper::tryFree(int64_t size) -{ - GET_JNIENV(env) - tryCallVoidMethod(env, listener, reservation_listener_unreserve, size); - CLEAN_JNIENV -} - -size_t ReservationListenerWrapper::currentMemory() -{ - GET_JNIENV(env) - int64_t res = safeCallLongMethod(env, listener, reservation_listener_currentMemory); - return res; - CLEAN_JNIENV -} -} diff --git a/cpp-ch/local-engine/jni/ReservationListenerWrapper.h b/cpp-ch/local-engine/jni/ReservationListenerWrapper.h deleted file mode 100644 index a4d26cb5417ea..0000000000000 --- a/cpp-ch/local-engine/jni/ReservationListenerWrapper.h +++ /dev/null @@ -1,48 +0,0 @@ -/* - * 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. - */ -#pragma once -#include -#include -#include - -namespace local_engine -{ -class ReservationListenerWrapper -{ -public: - static jclass reservation_listener_class; - static jmethodID reservation_listener_reserve; - static jmethodID reservation_listener_reserve_or_throw; - static jmethodID reservation_listener_unreserve; - static jmethodID reservation_listener_currentMemory; - - explicit ReservationListenerWrapper(jobject listener); - ~ReservationListenerWrapper(); - void reserve(int64_t size); - void reserveOrThrow(int64_t size); - void free(int64_t size); - /// Make sure destructors in CH Backend do not throw exceptions - void tryFree(int64_t size); - size_t currentMemory(); - - - -private: - jobject listener; -}; -using ReservationListenerWrapperPtr = std::shared_ptr; -} diff --git a/cpp-ch/local-engine/local_engine_jni.cpp b/cpp-ch/local-engine/local_engine_jni.cpp index a6ca55052ef9d..3a18771cdd520 100644 --- a/cpp-ch/local-engine/local_engine_jni.cpp +++ b/cpp-ch/local-engine/local_engine_jni.cpp @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +#include #include #include #include @@ -33,11 +34,10 @@ #include #include #include -#include +#include #include #include #include -#include #include #include #include @@ -45,7 +45,6 @@ #include #include #include -#include #include #include #include @@ -162,17 +161,6 @@ JNIEXPORT jint JNI_OnLoad(JavaVM * vm, void * /*reserved*/) local_engine::SparkRowToCHColumn::spark_row_iterator_nextBatch = local_engine::GetMethodID( env, local_engine::SparkRowToCHColumn::spark_row_interator_class, "nextBatch", "()Ljava/nio/ByteBuffer;"); - local_engine::ReservationListenerWrapper::reservation_listener_class - = local_engine::CreateGlobalClassReference(env, "Lorg/apache/gluten/memory/alloc/CHReservationListener;"); - local_engine::ReservationListenerWrapper::reservation_listener_reserve - = local_engine::GetMethodID(env, local_engine::ReservationListenerWrapper::reservation_listener_class, "reserve", "(J)J"); - local_engine::ReservationListenerWrapper::reservation_listener_reserve_or_throw - = local_engine::GetMethodID(env, local_engine::ReservationListenerWrapper::reservation_listener_class, "reserveOrThrow", "(J)V"); - local_engine::ReservationListenerWrapper::reservation_listener_unreserve - = local_engine::GetMethodID(env, local_engine::ReservationListenerWrapper::reservation_listener_class, "unreserve", "(J)J"); - local_engine::ReservationListenerWrapper::reservation_listener_currentMemory - = local_engine::GetMethodID(env, local_engine::ReservationListenerWrapper::reservation_listener_class, "currentMemory", "()J"); - local_engine::BroadCastJoinBuilder::init(env); local_engine::JNIUtils::vm = vm; @@ -198,7 +186,6 @@ JNIEXPORT void JNI_OnUnload(JavaVM * vm, void * /*reserved*/) env->DeleteGlobalRef(local_engine::WriteBufferFromJavaOutputStream::output_stream_class); env->DeleteGlobalRef(local_engine::SourceFromJavaIter::serialized_record_batch_iterator_class); env->DeleteGlobalRef(local_engine::SparkRowToCHColumn::spark_row_interator_class); - env->DeleteGlobalRef(local_engine::ReservationListenerWrapper::reservation_listener_class); } JNIEXPORT void Java_org_apache_gluten_vectorized_ExpressionEvaluatorJniWrapper_nativeInitNative(JNIEnv * env, jclass, jbyteArray conf_plan) @@ -218,10 +205,10 @@ JNIEXPORT void Java_org_apache_gluten_vectorized_ExpressionEvaluatorJniWrapper_n } JNIEXPORT void Java_org_apache_gluten_vectorized_ExpressionEvaluatorJniWrapper_injectWriteFilesTempPath( - JNIEnv * env, jclass, jlong allocator_id, jbyteArray temp_path, jbyteArray filename) + JNIEnv * env, jclass, jbyteArray temp_path, jbyteArray filename) { LOCAL_ENGINE_JNI_METHOD_START - const auto query_context = local_engine::getAllocator(allocator_id)->query_context; + const auto query_context = local_engine::QueryContextManager::instance().currentQueryContext(); const auto path_array = local_engine::getByteArrayElementsSafe(env, temp_path); const std::string c_path{reinterpret_cast(path_array.elems()), static_cast(path_array.length())}; @@ -237,7 +224,6 @@ JNIEXPORT void Java_org_apache_gluten_vectorized_ExpressionEvaluatorJniWrapper_i JNIEXPORT jlong Java_org_apache_gluten_vectorized_ExpressionEvaluatorJniWrapper_nativeCreateKernelWithIterator( JNIEnv * env, jclass , - jlong allocator_id, jbyteArray plan, jobjectArray split_infos, jobjectArray iter_arr, @@ -245,7 +231,7 @@ JNIEXPORT jlong Java_org_apache_gluten_vectorized_ExpressionEvaluatorJniWrapper_ jboolean materialize_input) { LOCAL_ENGINE_JNI_METHOD_START - auto query_context = local_engine::getAllocator(allocator_id)->query_context; + auto query_context = local_engine::QueryContextManager::instance().currentQueryContext(); // by task update new configs ( in case of dynamic config update ) const auto conf_plan_a = local_engine::getByteArrayElementsSafe(env, conf_plan); @@ -560,14 +546,9 @@ JNIEXPORT jlong Java_org_apache_gluten_vectorized_CHShuffleSplitterJniWrapper_na jstring data_file, jstring local_dirs, jint num_sub_dirs, - jboolean prefer_spill, jlong spill_threshold, jstring hash_algorithm, - jboolean throw_if_memory_exceed, - jboolean flush_block_buffer_before_evict, jlong max_sort_buffer_size, - jboolean spill_firstly_before_stop, - jboolean force_external_sort, jboolean force_memory_sort) { LOCAL_ENGINE_JNI_METHOD_START @@ -605,18 +586,10 @@ JNIEXPORT jlong Java_org_apache_gluten_vectorized_CHShuffleSplitterJniWrapper_na .compress_method = jstring2string(env, codec), .spill_threshold = static_cast(spill_threshold), .hash_algorithm = jstring2string(env, hash_algorithm), - .throw_if_memory_exceed = static_cast(throw_if_memory_exceed), - .flush_block_buffer_before_evict = static_cast(flush_block_buffer_before_evict), .max_sort_buffer_size = static_cast(max_sort_buffer_size), - .spill_firstly_before_stop = static_cast(spill_firstly_before_stop), - .force_external_sort = static_cast(force_external_sort), - .force_mermory_sort = static_cast(force_memory_sort)}; + .force_memory_sort = static_cast(force_memory_sort)}; auto name = jstring2string(env, short_name); - local_engine::SplitterHolder * splitter; - if (prefer_spill) - splitter = new local_engine::SplitterHolder{.splitter = local_engine::ShuffleSplitter::create(name, options)}; - else - splitter = new local_engine::SplitterHolder{.splitter = std::make_unique(name, options)}; + local_engine::SplitterHolder * splitter = new local_engine::SplitterHolder{.splitter = std::make_unique(name, options)}; return reinterpret_cast(splitter); LOCAL_ENGINE_JNI_METHOD_END(env, -1) } @@ -635,9 +608,6 @@ JNIEXPORT jlong Java_org_apache_gluten_vectorized_CHShuffleSplitterJniWrapper_na jlong spill_threshold, jstring hash_algorithm, jobject pusher, - jboolean throw_if_memory_exceed, - jboolean flush_block_buffer_before_evict, - jboolean force_external_sort, jboolean force_memory_sort) { LOCAL_ENGINE_JNI_METHOD_START @@ -668,10 +638,7 @@ JNIEXPORT jlong Java_org_apache_gluten_vectorized_CHShuffleSplitterJniWrapper_na .compress_method = jstring2string(env, codec), .spill_threshold = static_cast(spill_threshold), .hash_algorithm = jstring2string(env, hash_algorithm), - .throw_if_memory_exceed = static_cast(throw_if_memory_exceed), - .flush_block_buffer_before_evict = static_cast(flush_block_buffer_before_evict), - .force_external_sort = static_cast(force_external_sort), - .force_mermory_sort = static_cast(force_memory_sort)}; + .force_memory_sort = static_cast(force_memory_sort)}; auto name = jstring2string(env, short_name); local_engine::SplitterHolder * splitter; splitter = new local_engine::SplitterHolder{.splitter = std::make_unique(name, options, pusher)}; @@ -688,15 +655,6 @@ JNIEXPORT void Java_org_apache_gluten_vectorized_CHShuffleSplitterJniWrapper_spl LOCAL_ENGINE_JNI_METHOD_END(env, ) } -JNIEXPORT jlong Java_org_apache_gluten_vectorized_CHShuffleSplitterJniWrapper_evict(JNIEnv * env, jobject, jlong splitterId) -{ - LOCAL_ENGINE_JNI_METHOD_START - local_engine::SplitterHolder * splitter = reinterpret_cast(splitterId); - auto size = splitter->splitter->evictPartitions(); - return size; - LOCAL_ENGINE_JNI_METHOD_END(env, 0) -} - JNIEXPORT jobject Java_org_apache_gluten_vectorized_CHShuffleSplitterJniWrapper_stop(JNIEnv * env, jobject, jlong splitterId) { LOCAL_ENGINE_JNI_METHOD_START @@ -810,8 +768,7 @@ JNIEXPORT jlong Java_org_apache_gluten_vectorized_CHBlockConverterJniWrapper_con env->DeleteLocalRef(name); env->DeleteLocalRef(type); } - local_engine::SparkRowToCHColumn converter; - auto * block = converter.convertSparkRowItrToCHColumn(java_iter, c_names, c_types); + auto * block = local_engine::SparkRowToCHColumn::convertSparkRowItrToCHColumn(java_iter, c_names, c_types); return reinterpret_cast(block); LOCAL_ENGINE_JNI_METHOD_END(env, -1) } @@ -881,9 +838,9 @@ JNIEXPORT jlong Java_org_apache_spark_sql_execution_datasources_CHDatasourceJniW env->DeleteLocalRef(name); } const auto file_uri = jstring2string(env, file_uri_); - const auto format_hint = jstring2string(env, format_hint_); // for HiveFileFormat, the file url may not end with .parquet, so we pass in the format as a hint - const auto context = DB::Context::createCopy(local_engine::SerializedPlanParser::global_context); + const auto format_hint = jstring2string(env, format_hint_); + const auto context = local_engine::QueryContextManager::instance().currentQueryContext(); auto * writer = local_engine::createFileWriterWrapper(context, file_uri, names, format_hint).release(); return reinterpret_cast(writer); LOCAL_ENGINE_JNI_METHOD_END(env, 0) @@ -898,11 +855,10 @@ JNIEXPORT jlong Java_org_apache_spark_sql_execution_datasources_CHDatasourceJniW jstring task_id_, jstring partition_dir_, jstring bucket_dir_, - jbyteArray conf_plan, - jlong allocator_id) + jbyteArray conf_plan) { LOCAL_ENGINE_JNI_METHOD_START - auto query_context = local_engine::getAllocator(allocator_id)->query_context; + auto query_context = local_engine::QueryContextManager::instance().currentQueryContext(); // by task update new configs ( in case of dynamic config update ) const auto conf_plan_a = local_engine::getByteArrayElementsSafe(env, conf_plan); const std::string::size_type conf_plan_size = conf_plan_a.length(); @@ -1007,10 +963,6 @@ JNIEXPORT jstring Java_org_apache_spark_sql_execution_datasources_CHDatasourceJn { LOCAL_ENGINE_JNI_METHOD_START - // without this line, subsequent executeHere will throw an exception - auto status = std::make_shared(true); - - const auto uuid_str = jstring2string(env, uuid_); const auto task_id = jstring2string(env, task_id_); const auto partition_dir = jstring2string(env, partition_dir_); @@ -1028,16 +980,17 @@ JNIEXPORT jstring Java_org_apache_spark_sql_execution_datasources_CHDatasourceJn table.ParseFromString(extension_table.detail().value()); auto merge_tree_table = local_engine::parseMergeTreeTableString(table.value()); + auto context = local_engine::QueryContextManager::instance().currentQueryContext(); // each task using its own CustomStorageMergeTree, don't reuse auto temp_storage - = local_engine::MergeTreeRelParser::copyToVirtualStorage(merge_tree_table, local_engine::SerializedPlanParser::global_context); + = local_engine::MergeTreeRelParser::copyToVirtualStorage(merge_tree_table, context); local_engine::TempStorageFreer freer{temp_storage->getStorageID()}; // to release temp CustomStorageMergeTree with RAII std::vector selected_parts = local_engine::StorageMergeTreeFactory::instance().getDataPartsByNames( temp_storage->getStorageID(), "", merge_tree_table.getPartNames()); std::unordered_map partition_values; - std::vector loaded + std::vector loaded = local_engine::mergeParts(selected_parts, partition_values, uuid_str, temp_storage, partition_dir, bucket_dir); std::vector res; @@ -1045,9 +998,9 @@ JNIEXPORT jstring Java_org_apache_spark_sql_execution_datasources_CHDatasourceJn { saveFileStatus( *temp_storage, - local_engine::SerializedPlanParser::global_context, + context, partPtr->name, - const_cast(partPtr->getDataPartStorage())); + const_cast(partPtr->getDataPartStorage())); res.emplace_back(local_engine::PartInfo{ partPtr->name, partPtr->getMarksCount(), partPtr->getBytesOnDisk(), partPtr->rows_count, partition_values, bucket_dir}); } @@ -1268,34 +1221,37 @@ JNIEXPORT jlong Java_org_apache_gluten_vectorized_SimpleExpressionEval_nativeNex LOCAL_ENGINE_JNI_METHOD_END(env, -1) } -JNIEXPORT jlong Java_org_apache_gluten_memory_alloc_CHNativeMemoryAllocator_getDefaultAllocator(JNIEnv *, jclass) +JNIEXPORT jlong Java_org_apache_gluten_memory_CHThreadGroup_createThreadGroup(JNIEnv * env, jclass) { - return -1; + LOCAL_ENGINE_JNI_METHOD_START + return local_engine::QueryContextManager::instance().initializeQuery(); + LOCAL_ENGINE_JNI_METHOD_END(env, 0l) } -JNIEXPORT jlong -Java_org_apache_gluten_memory_alloc_CHNativeMemoryAllocator_createListenableAllocator(JNIEnv * env, jclass, jobject listener) +JNIEXPORT jlong Java_org_apache_gluten_memory_CHThreadGroup_threadGroupPeakMemory(JNIEnv * env, jclass, jlong id) { LOCAL_ENGINE_JNI_METHOD_START - auto listener_wrapper = std::make_shared(env->NewGlobalRef(listener)); - return local_engine::initializeQuery(listener_wrapper); - LOCAL_ENGINE_JNI_METHOD_END(env, -1) + return local_engine::QueryContextManager::instance().currentPeakMemory(id); + LOCAL_ENGINE_JNI_METHOD_END(env, 0l) } -JNIEXPORT void Java_org_apache_gluten_memory_alloc_CHNativeMemoryAllocator_releaseAllocator(JNIEnv * env, jclass, jlong allocator_id) +JNIEXPORT void Java_org_apache_gluten_memory_CHThreadGroup_releaseThreadGroup(JNIEnv * env, jclass, jlong id) { LOCAL_ENGINE_JNI_METHOD_START - local_engine::releaseAllocator(allocator_id); + local_engine::QueryContextManager::instance().finalizeQuery(id); LOCAL_ENGINE_JNI_METHOD_END(env, ) } -JNIEXPORT jlong Java_org_apache_gluten_memory_alloc_CHNativeMemoryAllocator_bytesAllocated(JNIEnv * env, jclass, jlong allocator_id) +// only for UT GlutenClickHouseNativeExceptionSuite +JNIEXPORT void Java_org_apache_gluten_utils_TestExceptionUtils_generateNativeException(JNIEnv * env) { LOCAL_ENGINE_JNI_METHOD_START - return local_engine::allocatorMemoryUsage(allocator_id); - LOCAL_ENGINE_JNI_METHOD_END(env, -1) + throw DB::Exception(DB::ErrorCodes::UNKNOWN_EXCEPTION, "test native exception"); + LOCAL_ENGINE_JNI_METHOD_END(env, ) } + + #ifdef __cplusplus } diff --git a/docs/get-started/ClickHouse.md b/docs/get-started/ClickHouse.md index f0b7fc13b2975..b9826548bdc30 100644 --- a/docs/get-started/ClickHouse.md +++ b/docs/get-started/ClickHouse.md @@ -677,14 +677,4 @@ spark.celeborn.storage.hdfs.dir hdfs:///celeborn spark.dynamicAllocation.enabled false ``` -### Columnar shuffle mode -We have two modes of columnar shuffle -1. prefer cache -2. prefer spill - -Switch through the configuration `spark.gluten.sql.columnar.backend.ch.shuffle.preferSpill`, the default is `false`, enable prefer cache shuffle. - -In the prefer cache mode, as much memory as possible will be used to cache the shuffle data. When the memory is insufficient, -spark will actively trigger the memory spill. You can also specify the threshold size through `spark.gluten.sql.columnar.backend.ch.spillThreshold` to Limit memory usage. The default value is `0MB`, which means no limit on memory usage. - diff --git a/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornColumnarShuffleWriter.scala b/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornColumnarShuffleWriter.scala index 40390a7151608..5279426c14bc7 100644 --- a/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornColumnarShuffleWriter.scala +++ b/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornColumnarShuffleWriter.scala @@ -18,8 +18,6 @@ package org.apache.spark.shuffle import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.clickhouse.CHBackendSettings -import org.apache.gluten.memory.alloc.CHNativeMemoryAllocators -import org.apache.gluten.memory.memtarget.{MemoryTarget, Spiller, Spillers} import org.apache.gluten.vectorized._ import org.apache.spark._ @@ -102,32 +100,9 @@ class CHCelebornColumnarShuffleWriter[K, V]( GlutenConfig.getConf.chColumnarShuffleSpillThreshold, CHBackendSettings.shuffleHashAlgorithm, celebornPartitionPusher, - GlutenConfig.getConf.chColumnarThrowIfMemoryExceed, - GlutenConfig.getConf.chColumnarFlushBlockBufferBeforeEvict, - GlutenConfig.getConf.chColumnarForceExternalSortShuffle, GlutenConfig.getConf.chColumnarForceMemorySortShuffle || ShuffleMode.SORT.name.equalsIgnoreCase(shuffleWriterType) ) - CHNativeMemoryAllocators.createSpillable( - "CelebornShuffleWriter", - new Spiller() { - override def spill(self: MemoryTarget, phase: Spiller.Phase, size: Long): Long = { - if (!Spillers.PHASE_SET_SPILL_ONLY.contains(phase)) { - return 0L - } - if (nativeShuffleWriter == -1L) { - throw new IllegalStateException( - "Fatal: spill() called before a celeborn shuffle writer is created. " + - "This behavior should be optimized by moving memory allocations " + - "from make() to split()") - } - logInfo(s"Gluten shuffle writer: Trying to push $size bytes of data") - val spilled = jniWrapper.evict(nativeShuffleWriter) - logInfo(s"Gluten shuffle writer: Spilled $spilled / $size bytes of data") - spilled - } - } - ) } override def closeShuffleWriter(): Unit = { diff --git a/gluten-celeborn/clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseRSSColumnarExternalSortShuffleSuite.scala b/gluten-celeborn/clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseRSSColumnarExternalSortShuffleSuite.scala deleted file mode 100644 index 3ecf1fc1ac7b3..0000000000000 --- a/gluten-celeborn/clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseRSSColumnarExternalSortShuffleSuite.scala +++ /dev/null @@ -1,208 +0,0 @@ -/* - * 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.gluten.execution - -import org.apache.spark.SparkConf -import org.apache.spark.sql.execution.CoalescedPartitionSpec -import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AdaptiveSparkPlanHelper, AQEShuffleReadExec} -import org.apache.spark.sql.internal.SQLConf - -class GlutenClickHouseRSSColumnarExternalSortShuffleSuite - extends GlutenClickHouseTPCHAbstractSuite - with AdaptiveSparkPlanHelper { - - override protected val tablesPath: String = basePath + "/tpch-data-ch" - override protected val tpchQueries: String = rootPath + "queries/tpch-queries-ch" - override protected val queriesResults: String = - rootPath + "../../../../../backends-clickhouse/src/test/resources/mergetree-queries-output" - - override protected val parquetTableDataPath: String = - "../../../../../gluten-core/src/test/resources/tpch-data" - - /** Run Gluten + ClickHouse Backend with ColumnarShuffleManager */ - override protected def sparkConf: SparkConf = { - super.sparkConf - .set( - "spark.shuffle.manager", - "org.apache.spark.shuffle.gluten.celeborn.CelebornShuffleManager") - .set("spark.io.compression.codec", "LZ4") - .set("spark.sql.shuffle.partitions", "5") - .set("spark.sql.autoBroadcastJoinThreshold", "10MB") - .set("spark.sql.adaptive.enabled", "true") - .set("spark.shuffle.service.enabled", "false") - .set("spark.celeborn.client.spark.shuffle.writer", "hash") - .set("spark.gluten.sql.columnar.backend.ch.forceExternalSortShuffle", "true") - } - - test("TPCH Q1") { - runTPCHQuery(1) { - df => - assert(df.queryExecution.executedPlan.isInstanceOf[AdaptiveSparkPlanExec]) - - val colCustomShuffleReaderExecs = collect(df.queryExecution.executedPlan) { - case csr: AQEShuffleReadExec => csr - } - assert(colCustomShuffleReaderExecs.size == 2) - val coalescedPartitionSpec0 = colCustomShuffleReaderExecs(0) - .partitionSpecs(0) - .asInstanceOf[CoalescedPartitionSpec] - assert(coalescedPartitionSpec0.startReducerIndex == 0) - assert(coalescedPartitionSpec0.endReducerIndex == 5) - val coalescedPartitionSpec1 = colCustomShuffleReaderExecs(1) - .partitionSpecs(0) - .asInstanceOf[CoalescedPartitionSpec] - assert(coalescedPartitionSpec1.startReducerIndex == 0) - assert(coalescedPartitionSpec1.endReducerIndex == 5) - } - } - - test("TPCH Q2") { - runTPCHQuery(2) { df => } - } - - test("TPCH Q3") { - runTPCHQuery(3) { df => } - } - - test("TPCH Q4") { - runTPCHQuery(4) { df => } - } - - test("TPCH Q5") { - runTPCHQuery(5) { df => } - } - - test("TPCH Q6") { - runTPCHQuery(6) { df => } - } - - test("TPCH Q7") { - runTPCHQuery(7) { df => } - } - - test("TPCH Q8") { - runTPCHQuery(8) { df => } - } - - test("TPCH Q9") { - runTPCHQuery(9) { df => } - } - - test("TPCH Q10") { - runTPCHQuery(10) { df => } - } - - test("TPCH Q11") { - runTPCHQuery(11) { - df => - assert(df.queryExecution.executedPlan.isInstanceOf[AdaptiveSparkPlanExec]) - val adaptiveSparkPlanExec = collectWithSubqueries(df.queryExecution.executedPlan) { - case adaptive: AdaptiveSparkPlanExec => adaptive - } - assert(adaptiveSparkPlanExec.size == 2) - } - } - - test("TPCH Q12") { - runTPCHQuery(12) { df => } - } - - test("TPCH Q13") { - runTPCHQuery(13) { df => } - } - - test("TPCH Q14") { - runTPCHQuery(14) { df => } - } - - test("TPCH Q15") { - runTPCHQuery(15) { - df => - assert(df.queryExecution.executedPlan.isInstanceOf[AdaptiveSparkPlanExec]) - val adaptiveSparkPlanExec = collectWithSubqueries(df.queryExecution.executedPlan) { - case adaptive: AdaptiveSparkPlanExec => adaptive - } - assert(adaptiveSparkPlanExec.size == 2) - } - } - - test("TPCH Q16") { - runTPCHQuery(16, noFallBack = false) { df => } - } - - test("TPCH Q17") { - runTPCHQuery(17) { df => } - } - - test("TPCH Q18") { - runTPCHQuery(18) { df => } - } - - test("TPCH Q19") { - runTPCHQuery(19) { df => } - } - - test("TPCH Q20") { - runTPCHQuery(20) { df => } - } - - test("TPCH Q21") { - runTPCHQuery(21, noFallBack = false) { df => } - } - - test("TPCH Q22") { - runTPCHQuery(22) { - df => - assert(df.queryExecution.executedPlan.isInstanceOf[AdaptiveSparkPlanExec]) - val adaptiveSparkPlanExec = collectWithSubqueries(df.queryExecution.executedPlan) { - case adaptive: AdaptiveSparkPlanExec => adaptive - } - assert(adaptiveSparkPlanExec.size == 3) - assert(adaptiveSparkPlanExec(1) == adaptiveSparkPlanExec(2)) - } - } - - test("fix partiton_id when spill_to_celeborn") { - import testImplicits._ - withSQLConf( - SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "5", - SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "10B", - SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "2", - SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_SIZE.key -> "1B" - ) { - val df = spark.sparkContext - .parallelize( - TestData(0) :: - TestData(0) :: - TestData(1) :: - TestData(1) :: - TestData(2) :: - TestData(2) :: Nil, - 3) - .toDF() - df.createOrReplaceTempView("t") - val res = spark.sql("select spark_partition_id(), id from t group by id").collect() - assert(res.length == 3) - assert(res(0).getInt(0) == 0) - assert(res(0).getInt(1) == 0) - assert(res(1).getInt(0) == 1) - assert(res(1).getInt(1) == 1) - assert(res(2).getInt(0) == 2) - assert(res(2).getInt(1) == 2) - } - } -} diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index 586a277f78554..69d2e21e67e54 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -314,8 +314,6 @@ class GlutenConfig(conf: SQLConf) extends Logging { .getOrElse(defaultSize) } - def chColumnarShufflePreferSpill: Boolean = conf.getConf(COLUMNAR_CH_SHUFFLE_PREFER_SPILL_ENABLED) - def chColumnarShuffleSpillThreshold: Long = { val threshold = conf.getConf(COLUMNAR_CH_SHUFFLE_SPILL_THRESHOLD) if (threshold == 0) { @@ -325,19 +323,8 @@ class GlutenConfig(conf: SQLConf) extends Logging { } } - def chColumnarThrowIfMemoryExceed: Boolean = conf.getConf(COLUMNAR_CH_THROW_IF_MEMORY_EXCEED) - - def chColumnarFlushBlockBufferBeforeEvict: Boolean = - conf.getConf(COLUMNAR_CH_FLUSH_BLOCK_BUFFER_BEFORE_EVICT) - def chColumnarMaxSortBufferSize: Long = conf.getConf(COLUMNAR_CH_MAX_SORT_BUFFER_SIZE) - def chColumnarSpillFirstlyBeforeStop: Boolean = - conf.getConf(COLUMNAR_CH_SPILL_FIRSTLY_BEFORE_STOP) - - def chColumnarForceExternalSortShuffle: Boolean = - conf.getConf(COLUMNAR_CH_FORCE_EXTERNAL_SORT_SHUFFLE) - def chColumnarForceMemorySortShuffle: Boolean = conf.getConf(COLUMNAR_CH_FORCE_MEMORY_SORT_SHUFFLE) @@ -1468,16 +1455,6 @@ object GlutenConfig { .intConf .createOptional - val COLUMNAR_CH_SHUFFLE_PREFER_SPILL_ENABLED = - buildConf("spark.gluten.sql.columnar.backend.ch.shuffle.preferSpill") - .internal() - .doc( - "Whether to spill the partition buffers when buffers are full. " + - "If false, the partition buffers will be cached in memory first, " + - "and the cached buffers will be spilled when reach maximum memory.") - .booleanConf - .createWithDefault(false) - val COLUMNAR_CH_SHUFFLE_SPILL_THRESHOLD = buildConf("spark.gluten.sql.columnar.backend.ch.spillThreshold") .internal() @@ -1485,20 +1462,6 @@ object GlutenConfig { .bytesConf(ByteUnit.BYTE) .createWithDefaultString("0MB") - val COLUMNAR_CH_THROW_IF_MEMORY_EXCEED = - buildConf("spark.gluten.sql.columnar.backend.ch.throwIfMemoryExceed") - .internal() - .doc("Throw exception if memory exceeds threshold on ch backend.") - .booleanConf - .createWithDefault(true) - - val COLUMNAR_CH_FLUSH_BLOCK_BUFFER_BEFORE_EVICT = - buildConf("spark.gluten.sql.columnar.backend.ch.flushBlockBufferBeforeEvict") - .internal() - .doc("Whether to flush partition_block_buffer before execute evict in CH PartitionWriter.") - .booleanConf - .createWithDefault(false) - val COLUMNAR_CH_MAX_SORT_BUFFER_SIZE = buildConf("spark.gluten.sql.columnar.backend.ch.maxSortBufferSize") .internal() @@ -1506,20 +1469,6 @@ object GlutenConfig { .bytesConf(ByteUnit.BYTE) .createWithDefaultString("0") - val COLUMNAR_CH_SPILL_FIRSTLY_BEFORE_STOP = - buildConf("spark.gluten.sql.columnar.backend.ch.spillFirstlyBeforeStop") - .internal() - .doc("Whether to spill the sort buffers before stopping the shuffle writer.") - .booleanConf - .createWithDefault(true) - - val COLUMNAR_CH_FORCE_EXTERNAL_SORT_SHUFFLE = - buildConf("spark.gluten.sql.columnar.backend.ch.forceExternalSortShuffle") - .internal() - .doc("Whether to force to use external sort shuffle in CH backend. ") - .booleanConf - .createWithDefault(false) - val COLUMNAR_CH_FORCE_MEMORY_SORT_SHUFFLE = buildConf("spark.gluten.sql.columnar.backend.ch.forceMemorySortShuffle") .internal() From dba4439a20ef9ffac78b14ff0f229460a0e84089 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Tue, 30 Jul 2024 17:16:00 +0800 Subject: [PATCH 25/61] [GLUTEN-6561][CH] Fix incompatiable type exception throw in capture function while processing array literal with `transform` (#6601) * fix style * fix issue https://github.com/apache/incubator-gluten/issues/6561 * add uts * add uts * fix uts * fix style * ignore some checks when spark 3.3 --- ...lutenClickHouseNativeWriteTableSuite.scala | 28 ++++--------------- cpp-ch/local-engine/Common/CHUtil.cpp | 15 +++++++++- cpp-ch/local-engine/Common/CHUtil.h | 7 +++++ .../CommonScalarFunctionParser.cpp | 2 +- .../arrayHighOrderFunctions.cpp | 24 +++++++++++----- 5 files changed, 45 insertions(+), 31 deletions(-) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeWriteTableSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeWriteTableSuite.scala index 99f946cd7dc86..578c432927477 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeWriteTableSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeWriteTableSuite.scala @@ -903,33 +903,17 @@ class GlutenClickHouseNativeWriteTableSuite | ) partitioned by (day string) | stored as $format""".stripMargin - // FIXME: - // Spark analyzer(>=3.4) will resolve map type to - // map_from_arrays(transform(map_keys(map('t1','a','t2','b')), v->v), - // transform(map_values(map('t1','a','t2','b')), v->v)) - // which cause core dump. see https://github.com/apache/incubator-gluten/issues/6561 - // for details. val insert_sql = - if (isSparkVersionLE("3.3")) { - s"""insert overwrite $table_name partition (day) - |select id as a, - | str_to_map(concat('t1:','a','&t2:','b'),'&',':'), - | struct('1', null) as c, - | '2024-01-08' as day - |from range(10)""".stripMargin - } else { - s"""insert overwrite $table_name partition (day) - |select id as a, - | map('t1', 'a', 't2', 'b'), - | struct('1', null) as c, - | '2024-01-08' as day - |from range(10)""".stripMargin - } + s"""insert overwrite $table_name partition (day) + |select id as a, + | str_to_map(concat('t1:','a','&t2:','b'),'&',':'), + | struct('1', null) as c, + | '2024-01-08' as day + |from range(10)""".stripMargin (table_name, create_sql, insert_sql) }, (table_name, _) => if (isSparkVersionGE("3.4")) { - // FIXME: Don't Know Why Failed compareResultsAgainstVanillaSpark( s"select * from $table_name", compareResult = true, diff --git a/cpp-ch/local-engine/Common/CHUtil.cpp b/cpp-ch/local-engine/Common/CHUtil.cpp index a606a06b5c151..003accf00f8c2 100644 --- a/cpp-ch/local-engine/Common/CHUtil.cpp +++ b/cpp-ch/local-engine/Common/CHUtil.cpp @@ -477,6 +477,19 @@ const DB::ActionsDAG::Node * ActionsDAGUtil::convertNodeType( DB::createInternalCastOverloadResolver(cast_type, std::move(diagnostic)), std::move(children), result_name); } +const DB::ActionsDAG::Node * ActionsDAGUtil::convertNodeTypeIfNeeded( + DB::ActionsDAGPtr & actions_dag, + const DB::ActionsDAG::Node * node, + const DB::DataTypePtr & dst_type, + const std::string & result_name, + CastType cast_type) +{ + if (node->result_type->equals(*dst_type)) + return node; + + return convertNodeType(actions_dag, node, dst_type->getName(), result_name, cast_type); +} + String QueryPipelineUtil::explainPipeline(DB::QueryPipeline & pipeline) { DB::WriteBufferFromOwnString buf; @@ -844,7 +857,7 @@ void BackendInitializerUtil::initContexts(DB::Context::ConfigurationPtr config) size_t index_uncompressed_cache_size = config->getUInt64("index_uncompressed_cache_size", DEFAULT_INDEX_UNCOMPRESSED_CACHE_MAX_SIZE); double index_uncompressed_cache_size_ratio = config->getDouble("index_uncompressed_cache_size_ratio", DEFAULT_INDEX_UNCOMPRESSED_CACHE_SIZE_RATIO); global_context->setIndexUncompressedCache(index_uncompressed_cache_policy, index_uncompressed_cache_size, index_uncompressed_cache_size_ratio); - + String index_mark_cache_policy = config->getString("index_mark_cache_policy", DEFAULT_INDEX_MARK_CACHE_POLICY); size_t index_mark_cache_size = config->getUInt64("index_mark_cache_size", DEFAULT_INDEX_MARK_CACHE_MAX_SIZE); double index_mark_cache_size_ratio = config->getDouble("index_mark_cache_size_ratio", DEFAULT_INDEX_MARK_CACHE_SIZE_RATIO); diff --git a/cpp-ch/local-engine/Common/CHUtil.h b/cpp-ch/local-engine/Common/CHUtil.h index 05b730552fa53..8a2a32df3071d 100644 --- a/cpp-ch/local-engine/Common/CHUtil.h +++ b/cpp-ch/local-engine/Common/CHUtil.h @@ -132,6 +132,13 @@ class ActionsDAGUtil const std::string & type_name, const std::string & result_name = "", DB::CastType cast_type = DB::CastType::nonAccurate); + + static const DB::ActionsDAG::Node * convertNodeTypeIfNeeded( + DB::ActionsDAGPtr & actions_dag, + const DB::ActionsDAG::Node * node, + const DB::DataTypePtr & dst_type, + const std::string & result_name = "", + DB::CastType cast_type = DB::CastType::nonAccurate); }; class QueryPipelineUtil diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/CommonScalarFunctionParser.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/CommonScalarFunctionParser.cpp index e4855b507f90e..726d1683dbffd 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/CommonScalarFunctionParser.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/CommonScalarFunctionParser.cpp @@ -64,7 +64,7 @@ REGISTER_COMMON_SCALAR_FUNCTION_PARSER(ToUnixTimestamp, to_unix_timestamp, parse REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Position, positive, identity); REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Negative, negative, negate); REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Pmod, pmod, pmod); -REGISTER_COMMON_SCALAR_FUNCTION_PARSER(abs, abs, abs); +REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Abs, abs, abs); REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Ceil, ceil, ceil); REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Round, round, roundHalfUp); REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Bround, bround, roundBankers); diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayHighOrderFunctions.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayHighOrderFunctions.cpp index eacd72ed044fa..f9f093cbad505 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayHighOrderFunctions.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayHighOrderFunctions.cpp @@ -15,16 +15,17 @@ * limitations under the License. */ -#include -#include -#include -#include -#include +#include +#include #include #include -#include +#include #include #include +#include +#include +#include +#include namespace DB::ErrorCodes { @@ -90,7 +91,16 @@ class ArrayTransform : public FunctionParser assert(parsed_args.size() == 2); if (lambda_args.size() == 1) { - return toFunctionNode(actions_dag, ch_func_name, {parsed_args[1], parsed_args[0]}); + /// Convert Array(T) to Array(U) if needed, Array(T) is the type of the first argument of transform. + /// U is the argument type of lambda function. In some cases Array(T) is not equal to Array(U). + /// e.g. in the second query of https://github.com/apache/incubator-gluten/issues/6561, T is String, and U is Nullable(String) + /// The difference of both types will result in runtime exceptions in function capture. + const auto & src_array_type = parsed_args[0]->result_type; + DataTypePtr dst_array_type = std::make_shared(lambda_args.front().type); + if (isNullableOrLowCardinalityNullable(src_array_type)) + dst_array_type = std::make_shared(dst_array_type); + const auto * dst_array_arg = ActionsDAGUtil::convertNodeTypeIfNeeded(actions_dag, parsed_args[0], dst_array_type); + return toFunctionNode(actions_dag, ch_func_name, {parsed_args[1], dst_array_arg}); } /// transform with index argument. From 97f4eb2e0c4af73978a5999eaf6f3115e72c6a97 Mon Sep 17 00:00:00 2001 From: Nicholas Jiang Date: Tue, 30 Jul 2024 17:58:21 +0800 Subject: [PATCH 26/61] [GLUTEN-6632] Bump Celeborn 0.4.2 and 0.5.1 (#6633) --- .github/workflows/velox_docker.yml | 6 +++--- docs/get-started/ClickHouse.md | 2 +- docs/get-started/Velox.md | 2 +- pom.xml | 2 +- tools/gluten-it/pom.xml | 4 ++-- 5 files changed, 8 insertions(+), 8 deletions(-) diff --git a/.github/workflows/velox_docker.yml b/.github/workflows/velox_docker.yml index 47dd7a9190b84..ca3317d12d40e 100644 --- a/.github/workflows/velox_docker.yml +++ b/.github/workflows/velox_docker.yml @@ -533,7 +533,7 @@ jobs: fail-fast: false matrix: spark: [ "spark-3.2" ] - celeborn: [ "celeborn-0.5.0", "celeborn-0.4.1", "celeborn-0.3.2-incubating" ] + celeborn: [ "celeborn-0.5.1", "celeborn-0.4.2", "celeborn-0.3.2-incubating" ] runs-on: ubuntu-20.04 container: ubuntu:22.04 steps: @@ -564,9 +564,9 @@ jobs: - name: TPC-H SF1.0 && TPC-DS SF1.0 Parquet local spark3.2 with ${{ matrix.celeborn }} run: | EXTRA_PROFILE="" - if [ "${{ matrix.celeborn }}" = "celeborn-0.4.1" ]; then + if [ "${{ matrix.celeborn }}" = "celeborn-0.4.2" ]; then EXTRA_PROFILE="-Pceleborn-0.4" - elif [ "${{ matrix.celeborn }}" = "celeborn-0.5.0" ]; then + elif [ "${{ matrix.celeborn }}" = "celeborn-0.5.1" ]; then EXTRA_PROFILE="-Pceleborn-0.5" fi echo "EXTRA_PROFILE: ${EXTRA_PROFILE}" diff --git a/docs/get-started/ClickHouse.md b/docs/get-started/ClickHouse.md index b9826548bdc30..ab90f2312c543 100644 --- a/docs/get-started/ClickHouse.md +++ b/docs/get-started/ClickHouse.md @@ -629,7 +629,7 @@ public read-only account:gluten/hN2xX3uQ4m ### Celeborn support -Gluten with clickhouse backend supports [Celeborn](https://github.com/apache/celeborn) as remote shuffle service. Currently, the supported Celeborn versions are `0.3.x`, `0.4.x` and `0.5.0`. +Gluten with clickhouse backend supports [Celeborn](https://github.com/apache/celeborn) as remote shuffle service. Currently, the supported Celeborn versions are `0.3.x`, `0.4.x` and `0.5.x`. Below introduction is used to enable this feature. diff --git a/docs/get-started/Velox.md b/docs/get-started/Velox.md index a349427afe0d3..6778bb9b72070 100644 --- a/docs/get-started/Velox.md +++ b/docs/get-started/Velox.md @@ -215,7 +215,7 @@ Currently there are several ways to asscess S3 in Spark. Please refer [Velox S3] ## Celeborn support -Gluten with velox backend supports [Celeborn](https://github.com/apache/celeborn) as remote shuffle service. Currently, the supported Celeborn versions are `0.3.x`, `0.4.x` and `0.5.0`. +Gluten with velox backend supports [Celeborn](https://github.com/apache/celeborn) as remote shuffle service. Currently, the supported Celeborn versions are `0.3.x`, `0.4.x` and `0.5.x`. Below introduction is used to enable this feature. diff --git a/pom.xml b/pom.xml index 81c355868159e..bb59ad2a7faa0 100644 --- a/pom.xml +++ b/pom.xml @@ -65,7 +65,7 @@ delta-core 2.4.0 24 - 0.4.1 + 0.5.1 0.8.0 15.0.0 15.0.0-gluten diff --git a/tools/gluten-it/pom.xml b/tools/gluten-it/pom.xml index a65324f2728de..43bc3ae092b5d 100644 --- a/tools/gluten-it/pom.xml +++ b/tools/gluten-it/pom.xml @@ -171,13 +171,13 @@ celeborn-0.4 - 0.4.1 + 0.4.2 celeborn-0.5 - 0.5.0 + 0.5.1 From 3a5e5b1d25cbec09b0fd54da1f9cc3ac68f95539 Mon Sep 17 00:00:00 2001 From: Kyligence Git Date: Tue, 30 Jul 2024 12:05:31 -0500 Subject: [PATCH 27/61] [GLUTEN-1632][CH]Daily Update Clickhouse Version (20240730) (#6640) * [GLUTEN-1632][CH]Daily Update Clickhouse Version (20240730) * Fix build due to https://github.com/ClickHouse/ClickHouse/pull/65414 * FIXME: very slow after https://github.com/apache/incubator-gluten/pull/6558 --------- Co-authored-by: kyligence-git Co-authored-by: Chang Chen --- ...ickHouseMergeTreePathBasedWriteSuite.scala | 3 +- .../GlutenClickHouseMergeTreeWriteSuite.scala | 3 +- cpp-ch/clickhouse.version | 4 +- cpp-ch/local-engine/Common/CHUtil.cpp | 14 +- cpp-ch/local-engine/Common/CHUtil.h | 4 +- cpp-ch/local-engine/Common/MergeTreeTool.cpp | 2 +- .../Parser/AggregateFunctionParser.cpp | 10 +- .../Parser/AggregateFunctionParser.h | 16 +- .../Parser/AggregateRelParser.cpp | 24 +-- cpp-ch/local-engine/Parser/CrossRelParser.cpp | 49 +++-- .../local-engine/Parser/FilterRelParser.cpp | 8 +- .../local-engine/Parser/FunctionExecutor.cpp | 6 +- cpp-ch/local-engine/Parser/FunctionParser.cpp | 6 +- cpp-ch/local-engine/Parser/FunctionParser.h | 22 +-- cpp-ch/local-engine/Parser/JoinRelParser.cpp | 76 ++++---- .../Parser/MergeTreeRelParser.cpp | 22 +-- .../local-engine/Parser/MergeTreeRelParser.h | 4 +- .../local-engine/Parser/ProjectRelParser.cpp | 50 +++--- cpp-ch/local-engine/Parser/ProjectRelParser.h | 10 +- cpp-ch/local-engine/Parser/RelParser.h | 8 +- .../Parser/SerializedPlanParser.cpp | 170 +++++++----------- .../Parser/SerializedPlanParser.h | 45 ++--- .../local-engine/Parser/WindowRelParser.cpp | 24 +-- cpp-ch/local-engine/Parser/WriteRelParser.cpp | 10 +- .../CollectListParser.h | 6 +- .../aggregate_function_parser/CountParser.cpp | 6 +- .../aggregate_function_parser/CountParser.h | 2 +- .../LeadLagParser.cpp | 26 +-- .../aggregate_function_parser/LeadLagParser.h | 4 +- .../aggregate_function_parser/NtileParser.cpp | 2 +- .../aggregate_function_parser/NtileParser.h | 2 +- .../SimpleStatisticsFunctions.cpp | 6 +- .../local-engine/Parser/example_udf/myMd5.cpp | 2 +- .../Parser/scalar_function_parser/alias.cpp | 6 +- .../scalar_function_parser/arithmetic.cpp | 14 +- .../scalar_function_parser/arrayContains.cpp | 2 +- .../scalar_function_parser/arrayDistinct.cpp | 2 +- .../scalar_function_parser/arrayElement.h | 2 +- .../arrayHighOrderFunctions.cpp | 16 +- .../scalar_function_parser/arrayIntersect.cpp | 2 +- .../scalar_function_parser/arrayMaxAndMin.cpp | 2 +- .../scalar_function_parser/arrayPosition.cpp | 2 +- .../scalar_function_parser/arrayUnion.cpp | 2 +- .../scalar_function_parser/bitLength.cpp | 2 +- .../scalar_function_parser/checkOverflow.cpp | 2 +- .../Parser/scalar_function_parser/chr.cpp | 2 +- .../Parser/scalar_function_parser/concat.cpp | 2 +- .../scalar_function_parser/concatWs.cpp | 2 +- .../Parser/scalar_function_parser/cot.cpp | 2 +- .../Parser/scalar_function_parser/csc.cpp | 2 +- .../scalar_function_parser/dateFormat.cpp | 2 +- .../Parser/scalar_function_parser/decode.cpp | 2 +- .../scalar_function_parser/elementAt.cpp | 2 +- .../Parser/scalar_function_parser/elt.cpp | 2 +- .../scalar_function_parser/empty2null.cpp | 2 +- .../Parser/scalar_function_parser/encode.cpp | 2 +- .../scalar_function_parser/equalNullSafe.cpp | 2 +- .../Parser/scalar_function_parser/expm1.cpp | 2 +- .../Parser/scalar_function_parser/extract.cpp | 2 +- .../scalar_function_parser/factorial.cpp | 2 +- .../scalar_function_parser/findInset.cpp | 2 +- .../scalar_function_parser/fromJson.cpp | 2 +- .../fromUtcTimestamp.cpp | 2 +- .../scalar_function_parser/getJSONObject.cpp | 6 +- .../Parser/scalar_function_parser/isNaN.cpp | 2 +- .../scalar_function_parser/lambdaFunction.cpp | 34 ++-- .../Parser/scalar_function_parser/length.cpp | 2 +- .../Parser/scalar_function_parser/ln.cpp | 2 +- .../Parser/scalar_function_parser/locate.cpp | 2 +- .../Parser/scalar_function_parser/log.cpp | 2 +- .../Parser/scalar_function_parser/log10.cpp | 2 +- .../Parser/scalar_function_parser/log1p.cpp | 2 +- .../Parser/scalar_function_parser/log2.cpp | 2 +- .../Parser/scalar_function_parser/logarithm.h | 4 +- .../scalar_function_parser/makeDecimal.cpp | 2 +- .../Parser/scalar_function_parser/md5.cpp | 2 +- .../Parser/scalar_function_parser/nanvl.cpp | 2 +- .../scalar_function_parser/octetLength.cpp | 2 +- .../scalar_function_parser/parseUrl.cpp | 6 +- .../Parser/scalar_function_parser/parseUrl.h | 4 +- .../scalar_function_parser/regexp_extract.cpp | 2 +- .../Parser/scalar_function_parser/repeat.cpp | 2 +- .../Parser/scalar_function_parser/sec.cpp | 2 +- .../scalar_function_parser/sequence.cpp | 2 +- .../Parser/scalar_function_parser/sha1.cpp | 2 +- .../Parser/scalar_function_parser/sha2.cpp | 2 +- .../shiftRightUnsigned.cpp | 2 +- .../Parser/scalar_function_parser/size.cpp | 2 +- .../Parser/scalar_function_parser/slice.cpp | 6 +- .../scalar_function_parser/sortArray.cpp | 2 +- .../Parser/scalar_function_parser/space.cpp | 2 +- .../Parser/scalar_function_parser/split.cpp | 2 +- .../scalar_function_parser/substring.cpp | 2 +- .../scalar_function_parser/timestampAdd.cpp | 2 +- .../scalar_function_parser/toUtcTimestamp.cpp | 2 +- .../scalar_function_parser/trimFunctions.cpp | 6 +- .../Parser/scalar_function_parser/trunc.cpp | 2 +- .../Parser/scalar_function_parser/tuple.cpp | 2 +- .../scalar_function_parser/tupleElement.cpp | 2 +- .../scalar_function_parser/unixTimestamp.cpp | 2 +- .../utcTimestampTransform.h | 2 +- .../local-engine/Shuffle/SelectorBuilder.cpp | 2 +- .../Mergetree/SparkMergeTreeWriter.cpp | 7 +- .../Storages/Parquet/ColumnIndexFilter.cpp | 8 +- .../Storages/Parquet/ColumnIndexFilter.h | 2 +- .../SubstraitSource/SubstraitFileSource.cpp | 4 +- .../SubstraitSource/SubstraitFileSource.h | 2 +- .../tests/benchmark_local_engine.cpp | 8 +- .../tests/benchmark_parquet_read.cpp | 6 +- .../tests/benchmark_spark_divide_function.cpp | 32 ++-- .../local-engine/tests/gluten_test_util.cpp | 2 +- cpp-ch/local-engine/tests/gluten_test_util.h | 2 +- cpp-ch/local-engine/tests/gtest_ch_join.cpp | 14 +- .../tests/gtest_parquet_columnindex.cpp | 8 +- 114 files changed, 451 insertions(+), 510 deletions(-) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreePathBasedWriteSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreePathBasedWriteSuite.scala index 129f5405c28f4..ed6953b81a32d 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreePathBasedWriteSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreePathBasedWriteSuite.scala @@ -749,7 +749,8 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite } } - test("test mergetree path based write with bucket table") { + // FIXME: very slow after https://github.com/apache/incubator-gluten/pull/6558 + ignore("test mergetree path based write with bucket table") { val dataPath = s"$basePath/lineitem_mergetree_bucket" clearDataPath(dataPath) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteSuite.scala index 77d7f37c0369a..84218f26a07f5 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteSuite.scala @@ -801,7 +801,8 @@ class GlutenClickHouseMergeTreeWriteSuite } } - test("test mergetree write with bucket table") { + // FIXME: very slow after https://github.com/apache/incubator-gluten/pull/6558 + ignore("test mergetree write with bucket table") { spark.sql(s""" |DROP TABLE IF EXISTS lineitem_mergetree_bucket; |""".stripMargin) diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version index 9284537ad1e75..a69c80926c466 100644 --- a/cpp-ch/clickhouse.version +++ b/cpp-ch/clickhouse.version @@ -1,4 +1,4 @@ CH_ORG=Kyligence -CH_BRANCH=rebase_ch/20240727 -CH_COMMIT=d09605082e3 +CH_BRANCH=rebase_ch/20240730 +CH_COMMIT=f69def8b6a8 diff --git a/cpp-ch/local-engine/Common/CHUtil.cpp b/cpp-ch/local-engine/Common/CHUtil.cpp index 003accf00f8c2..b6867f656a07e 100644 --- a/cpp-ch/local-engine/Common/CHUtil.cpp +++ b/cpp-ch/local-engine/Common/CHUtil.cpp @@ -459,7 +459,7 @@ const DB::ColumnWithTypeAndName * NestedColumnExtractHelper::findColumn(const DB } const DB::ActionsDAG::Node * ActionsDAGUtil::convertNodeType( - DB::ActionsDAGPtr & actions_dag, + DB::ActionsDAG & actions_dag, const DB::ActionsDAG::Node * node, const std::string & type_name, const std::string & result_name, @@ -469,16 +469,16 @@ const DB::ActionsDAG::Node * ActionsDAGUtil::convertNodeType( type_name_col.name = type_name; type_name_col.column = DB::DataTypeString().createColumnConst(0, type_name_col.name); type_name_col.type = std::make_shared(); - const auto * right_arg = &actions_dag->addColumn(std::move(type_name_col)); + const auto * right_arg = &actions_dag.addColumn(std::move(type_name_col)); const auto * left_arg = node; DB::CastDiagnostic diagnostic = {node->result_name, node->result_name}; DB::ActionsDAG::NodeRawConstPtrs children = {left_arg, right_arg}; - return &actions_dag->addFunction( + return &actions_dag.addFunction( DB::createInternalCastOverloadResolver(cast_type, std::move(diagnostic)), std::move(children), result_name); } const DB::ActionsDAG::Node * ActionsDAGUtil::convertNodeTypeIfNeeded( - DB::ActionsDAGPtr & actions_dag, + DB::ActionsDAG & actions_dag, const DB::ActionsDAG::Node * node, const DB::DataTypePtr & dst_type, const std::string & result_name, @@ -1079,14 +1079,14 @@ UInt64 MemoryUtil::getMemoryRSS() void JoinUtil::reorderJoinOutput(DB::QueryPlan & plan, DB::Names cols) { - ActionsDAGPtr project = std::make_shared(plan.getCurrentDataStream().header.getNamesAndTypesList()); + ActionsDAG project{plan.getCurrentDataStream().header.getNamesAndTypesList()}; NamesWithAliases project_cols; for (const auto & col : cols) { project_cols.emplace_back(NameWithAlias(col, col)); } - project->project(project_cols); - QueryPlanStepPtr project_step = std::make_unique(plan.getCurrentDataStream(), project); + project.project(project_cols); + QueryPlanStepPtr project_step = std::make_unique(plan.getCurrentDataStream(), std::move(project)); project_step->setStepDescription("Reorder Join Output"); plan.addStep(std::move(project_step)); } diff --git a/cpp-ch/local-engine/Common/CHUtil.h b/cpp-ch/local-engine/Common/CHUtil.h index 8a2a32df3071d..f528128033357 100644 --- a/cpp-ch/local-engine/Common/CHUtil.h +++ b/cpp-ch/local-engine/Common/CHUtil.h @@ -127,14 +127,14 @@ class ActionsDAGUtil { public: static const DB::ActionsDAG::Node * convertNodeType( - DB::ActionsDAGPtr & actions_dag, + DB::ActionsDAG & actions_dag, const DB::ActionsDAG::Node * node, const std::string & type_name, const std::string & result_name = "", DB::CastType cast_type = DB::CastType::nonAccurate); static const DB::ActionsDAG::Node * convertNodeTypeIfNeeded( - DB::ActionsDAGPtr & actions_dag, + DB::ActionsDAG & actions_dag, const DB::ActionsDAG::Node * node, const DB::DataTypePtr & dst_type, const std::string & result_name = "", diff --git a/cpp-ch/local-engine/Common/MergeTreeTool.cpp b/cpp-ch/local-engine/Common/MergeTreeTool.cpp index 63bf64726bf24..d3b7d7b229a12 100644 --- a/cpp-ch/local-engine/Common/MergeTreeTool.cpp +++ b/cpp-ch/local-engine/Common/MergeTreeTool.cpp @@ -113,7 +113,7 @@ std::shared_ptr buildMetaData( if (table.order_by_key != MergeTreeTable::TUPLE) metadata->primary_key = KeyDescription::parse(table.order_by_key, metadata->getColumns(), context); else - metadata->primary_key.expression = std::make_shared(std::make_shared()); + metadata->primary_key.expression = std::make_shared(ActionsDAG{}); } else { diff --git a/cpp-ch/local-engine/Parser/AggregateFunctionParser.cpp b/cpp-ch/local-engine/Parser/AggregateFunctionParser.cpp index f183cc0a46906..f976d50ad3b2f 100644 --- a/cpp-ch/local-engine/Parser/AggregateFunctionParser.cpp +++ b/cpp-ch/local-engine/Parser/AggregateFunctionParser.cpp @@ -40,7 +40,7 @@ namespace local_engine DB::ActionsDAG::NodeRawConstPtrs AggregateFunctionParser::parseFunctionArguments( const CommonFunctionInfo & func_info, - DB::ActionsDAGPtr & actions_dag) const + DB::ActionsDAG & actions_dag) const { DB::ActionsDAG::NodeRawConstPtrs collected_args; for (const auto & arg : func_info.arguments) @@ -56,7 +56,7 @@ DB::ActionsDAG::NodeRawConstPtrs AggregateFunctionParser::parseFunctionArguments DB::ActionsDAG::NodeRawConstPtrs args; args.emplace_back(arg_node); const auto * node = toFunctionNode(actions_dag, "toNullable", args); - actions_dag->addOrReplaceInOutputs(*node); + actions_dag.addOrReplaceInOutputs(*node); arg_node = node; } @@ -147,7 +147,7 @@ std::pair AggregateFunctionParser::tryApplyCHCombinator( const DB::ActionsDAG::Node * AggregateFunctionParser::convertNodeTypeIfNeeded( const CommonFunctionInfo & func_info, const DB::ActionsDAG::Node * func_node, - DB::ActionsDAGPtr & actions_dag, + DB::ActionsDAG & actions_dag, bool with_nullability) const { const auto & output_type = func_info.output_type; @@ -156,7 +156,7 @@ const DB::ActionsDAG::Node * AggregateFunctionParser::convertNodeTypeIfNeeded( { func_node = ActionsDAGUtil::convertNodeType( actions_dag, func_node, TypeParser::parseType(output_type)->getName(), func_node->result_name); - actions_dag->addOrReplaceInOutputs(*func_node); + actions_dag.addOrReplaceInOutputs(*func_node); } if (output_type.has_decimal()) @@ -167,7 +167,7 @@ const DB::ActionsDAG::Node * AggregateFunctionParser::convertNodeTypeIfNeeded( plan_parser->addColumn(actions_dag, std::make_shared(), output_type.decimal().precision()), plan_parser->addColumn(actions_dag, std::make_shared(), output_type.decimal().scale())}; func_node = toFunctionNode(actions_dag, checkDecimalOverflowSparkOrNull, func_node->result_name, overflow_args); - actions_dag->addOrReplaceInOutputs(*func_node); + actions_dag.addOrReplaceInOutputs(*func_node); } return func_node; diff --git a/cpp-ch/local-engine/Parser/AggregateFunctionParser.h b/cpp-ch/local-engine/Parser/AggregateFunctionParser.h index 215c09626b7e7..ea63e9993e63d 100644 --- a/cpp-ch/local-engine/Parser/AggregateFunctionParser.h +++ b/cpp-ch/local-engine/Parser/AggregateFunctionParser.h @@ -97,7 +97,7 @@ class AggregateFunctionParser /// Do some preprojections for the function arguments, and return the necessary arguments for the CH function. virtual DB::ActionsDAG::NodeRawConstPtrs - parseFunctionArguments(const CommonFunctionInfo & func_info, DB::ActionsDAGPtr & actions_dag) const; + parseFunctionArguments(const CommonFunctionInfo & func_info, DB::ActionsDAG & actions_dag) const; // `PartialMerge` is applied on the merging stages. // `If` is applied when the aggreate function has a filter. This should only happen on the 1st stage. @@ -109,7 +109,7 @@ class AggregateFunctionParser virtual const DB::ActionsDAG::Node * convertNodeTypeIfNeeded( const CommonFunctionInfo & func_info, const DB::ActionsDAG::Node * func_node, - DB::ActionsDAGPtr & actions_dag, + DB::ActionsDAG & actions_dag, bool with_nullability) const; /// Parameters are only used in aggregate functions at present. e.g. percentiles(0.5)(x). @@ -129,28 +129,28 @@ class AggregateFunctionParser String getUniqueName(const String & name) const { return plan_parser->getUniqueName(name); } const DB::ActionsDAG::Node * - addColumnToActionsDAG(DB::ActionsDAGPtr & actions_dag, const DB::DataTypePtr & type, const DB::Field & field) const + addColumnToActionsDAG(DB::ActionsDAG & actions_dag, const DB::DataTypePtr & type, const DB::Field & field) const { - return &actions_dag->addColumn(ColumnWithTypeAndName(type->createColumnConst(1, field), type, getUniqueName(toString(field)))); + return &actions_dag.addColumn(ColumnWithTypeAndName(type->createColumnConst(1, field), type, getUniqueName(toString(field)))); } const DB::ActionsDAG::Node * - toFunctionNode(DB::ActionsDAGPtr & action_dag, const String & func_name, const DB::ActionsDAG::NodeRawConstPtrs & args) const + toFunctionNode(DB::ActionsDAG & action_dag, const String & func_name, const DB::ActionsDAG::NodeRawConstPtrs & args) const { return plan_parser->toFunctionNode(action_dag, func_name, args); } const DB::ActionsDAG::Node * toFunctionNode( - DB::ActionsDAGPtr & action_dag, + DB::ActionsDAG & action_dag, const String & func_name, const String & result_name, const DB::ActionsDAG::NodeRawConstPtrs & args) const { auto function_builder = DB::FunctionFactory::instance().get(func_name, getContext()); - return &action_dag->addFunction(function_builder, args, result_name); + return &action_dag.addFunction(function_builder, args, result_name); } - const DB::ActionsDAG::Node * parseExpression(DB::ActionsDAGPtr actions_dag, const substrait::Expression & rel) const + const DB::ActionsDAG::Node * parseExpression(DB::ActionsDAG& actions_dag, const substrait::Expression & rel) const { return plan_parser->parseExpression(actions_dag, rel); } diff --git a/cpp-ch/local-engine/Parser/AggregateRelParser.cpp b/cpp-ch/local-engine/Parser/AggregateRelParser.cpp index 532b4114b8f07..bf5129f13277f 100644 --- a/cpp-ch/local-engine/Parser/AggregateRelParser.cpp +++ b/cpp-ch/local-engine/Parser/AggregateRelParser.cpp @@ -168,8 +168,8 @@ void AggregateRelParser::setup(DB::QueryPlanPtr query_plan, const substrait::Rel void AggregateRelParser::addPreProjection() { auto input_header = plan->getCurrentDataStream().header; - ActionsDAGPtr projection_action = std::make_shared(input_header.getColumnsWithTypeAndName()); - std::string dag_footprint = projection_action->dumpDAG(); + ActionsDAG projection_action{input_header.getColumnsWithTypeAndName()}; + std::string dag_footprint = projection_action.dumpDAG(); for (auto & agg_info : aggregates) { auto arg_nodes = agg_info.function_parser->parseFunctionArguments(agg_info.parser_func_info, projection_action); @@ -179,14 +179,14 @@ void AggregateRelParser::addPreProjection() { agg_info.arg_column_names.emplace_back(arg_node->result_name); agg_info.arg_column_types.emplace_back(arg_node->result_type); - projection_action->addOrReplaceInOutputs(*arg_node); + projection_action.addOrReplaceInOutputs(*arg_node); } } - if (projection_action->dumpDAG() != dag_footprint) + if (projection_action.dumpDAG() != dag_footprint) { /// Avoid unnecessary evaluation - projection_action->removeUnusedActions(); - auto projection_step = std::make_unique(plan->getCurrentDataStream(), projection_action); + projection_action.removeUnusedActions(); + auto projection_step = std::make_unique(plan->getCurrentDataStream(), std::move(projection_action)); projection_step->setStepDescription("Projection before aggregate"); steps.emplace_back(projection_step.get()); plan->addStep(std::move(projection_step)); @@ -482,14 +482,14 @@ void AggregateRelParser::addAggregatingStep() void AggregateRelParser::addPostProjection() { auto input_header = plan->getCurrentDataStream().header; - ActionsDAGPtr project_actions_dag = std::make_shared(input_header.getColumnsWithTypeAndName()); - auto dag_footprint = project_actions_dag->dumpDAG(); + ActionsDAG project_actions_dag{input_header.getColumnsWithTypeAndName()}; + auto dag_footprint = project_actions_dag.dumpDAG(); if (has_final_stage) { for (const auto & agg_info : aggregates) { - for (const auto * input_node : project_actions_dag->getInputs()) + for (const auto * input_node : project_actions_dag.getInputs()) { if (input_node->result_name == agg_info.measure_column_name) { @@ -503,7 +503,7 @@ void AggregateRelParser::addPostProjection() // on the complete mode, it must consider the nullability when converting node type for (const auto & agg_info : aggregates) { - for (const auto * output_node : project_actions_dag->getOutputs()) + for (const auto * output_node : project_actions_dag.getOutputs()) { if (output_node->result_name == agg_info.measure_column_name) { @@ -512,9 +512,9 @@ void AggregateRelParser::addPostProjection() } } } - if (project_actions_dag->dumpDAG() != dag_footprint) + if (project_actions_dag.dumpDAG() != dag_footprint) { - QueryPlanStepPtr convert_step = std::make_unique(plan->getCurrentDataStream(), project_actions_dag); + QueryPlanStepPtr convert_step = std::make_unique(plan->getCurrentDataStream(), std::move(project_actions_dag)); convert_step->setStepDescription("Post-projection for aggregate"); steps.emplace_back(convert_step.get()); plan->addStep(std::move(convert_step)); diff --git a/cpp-ch/local-engine/Parser/CrossRelParser.cpp b/cpp-ch/local-engine/Parser/CrossRelParser.cpp index 9d6252f66c219..debfc2a1eac86 100644 --- a/cpp-ch/local-engine/Parser/CrossRelParser.cpp +++ b/cpp-ch/local-engine/Parser/CrossRelParser.cpp @@ -101,21 +101,17 @@ DB::QueryPlanPtr CrossRelParser::parseOp(const substrait::Rel & rel, std::list 0 && right_ori_header[0].name != BlockUtil::VIRTUAL_ROW_COUNT_COLUMN) { - project = ActionsDAG::makeConvertingActions( + ActionsDAG right_project = ActionsDAG::makeConvertingActions( right_ori_header, storage_join.getRightSampleBlock().getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Position); - if (project) - { - QueryPlanStepPtr project_step = std::make_unique(right.getCurrentDataStream(), project); - project_step->setStepDescription("Rename Broadcast Table Name"); - steps.emplace_back(project_step.get()); - right.addStep(std::move(project_step)); - } + QueryPlanStepPtr project_step = std::make_unique(right.getCurrentDataStream(), std::move(right_project)); + project_step->setStepDescription("Rename Broadcast Table Name"); + steps.emplace_back(project_step.get()); + right.addStep(std::move(project_step)); } /// If the columns name in right table is duplicated with left table, we need to rename the left table's columns, @@ -130,15 +126,12 @@ void CrossRelParser::renamePlanColumns(DB::QueryPlan & left, DB::QueryPlan & rig else new_left_cols.emplace_back(col.column, col.type, col.name); auto left_header = left.getCurrentDataStream().header.getColumnsWithTypeAndName(); - project = ActionsDAG::makeConvertingActions(left_header, new_left_cols, ActionsDAG::MatchColumnsMode::Position); + ActionsDAG left_project = ActionsDAG::makeConvertingActions(left_header, new_left_cols, ActionsDAG::MatchColumnsMode::Position); - if (project) - { - QueryPlanStepPtr project_step = std::make_unique(left.getCurrentDataStream(), project); - project_step->setStepDescription("Rename Left Table Name for broadcast join"); - steps.emplace_back(project_step.get()); - left.addStep(std::move(project_step)); - } + QueryPlanStepPtr project_step = std::make_unique(left.getCurrentDataStream(), std::move(left_project)); + project_step->setStepDescription("Rename Left Table Name for broadcast join"); + steps.emplace_back(project_step.get()); + left.addStep(std::move(project_step)); } DB::QueryPlanPtr CrossRelParser::parseJoin(const substrait::CrossRel & join, DB::QueryPlanPtr left, DB::QueryPlanPtr right) @@ -229,7 +222,7 @@ void CrossRelParser::addPostFilter(DB::QueryPlan & query_plan, const substrait:: auto expression = join_rel.expression(); std::string filter_name; - auto actions_dag = std::make_shared(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); + ActionsDAG actions_dag(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); if (!expression.has_scalar_function()) { // It may be singular_or_list @@ -238,9 +231,9 @@ void CrossRelParser::addPostFilter(DB::QueryPlan & query_plan, const substrait:: } else { - getPlanParser()->parseFunction(query_plan.getCurrentDataStream().header, expression, filter_name, actions_dag, true); + getPlanParser()->parseFunctionWithDAG(expression, filter_name, actions_dag, true); } - auto filter_step = std::make_unique(query_plan.getCurrentDataStream(), actions_dag, filter_name, true); + auto filter_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(actions_dag), filter_name, true); filter_step->setStepDescription("Post Join Filter"); steps.emplace_back(filter_step.get()); query_plan.addStep(std::move(filter_step)); @@ -268,19 +261,19 @@ void CrossRelParser::addConvertStep(TableJoin & table_join, DB::QueryPlan & left } if (!right_table_alias.empty()) { - ActionsDAGPtr rename_dag = std::make_shared(right.getCurrentDataStream().header.getNamesAndTypesList()); + ActionsDAG rename_dag(right.getCurrentDataStream().header.getNamesAndTypesList()); auto original_right_columns = right.getCurrentDataStream().header; for (const auto & column_alias : right_table_alias) { if (original_right_columns.has(column_alias.first)) { auto pos = original_right_columns.getPositionByName(column_alias.first); - const auto & alias = rename_dag->addAlias(*rename_dag->getInputs()[pos], column_alias.second); - rename_dag->getOutputs()[pos] = &alias; + const auto & alias = rename_dag.addAlias(*rename_dag.getInputs()[pos], column_alias.second); + rename_dag.getOutputs()[pos] = &alias; } } - QueryPlanStepPtr project_step = std::make_unique(right.getCurrentDataStream(), rename_dag); + QueryPlanStepPtr project_step = std::make_unique(right.getCurrentDataStream(), std::move(rename_dag)); project_step->setStepDescription("Right Table Rename"); steps.emplace_back(project_step.get()); right.addStep(std::move(project_step)); @@ -290,14 +283,14 @@ void CrossRelParser::addConvertStep(TableJoin & table_join, DB::QueryPlan & left { table_join.addJoinedColumn(column); } - ActionsDAGPtr left_convert_actions = nullptr; - ActionsDAGPtr right_convert_actions = nullptr; + std::optional left_convert_actions; + std::optional right_convert_actions; std::tie(left_convert_actions, right_convert_actions) = table_join.createConvertingActions( left.getCurrentDataStream().header.getColumnsWithTypeAndName(), right.getCurrentDataStream().header.getColumnsWithTypeAndName()); if (right_convert_actions) { - auto converting_step = std::make_unique(right.getCurrentDataStream(), right_convert_actions); + auto converting_step = std::make_unique(right.getCurrentDataStream(), std::move(*right_convert_actions)); converting_step->setStepDescription("Convert joined columns"); steps.emplace_back(converting_step.get()); right.addStep(std::move(converting_step)); @@ -305,7 +298,7 @@ void CrossRelParser::addConvertStep(TableJoin & table_join, DB::QueryPlan & left if (left_convert_actions) { - auto converting_step = std::make_unique(left.getCurrentDataStream(), left_convert_actions); + auto converting_step = std::make_unique(left.getCurrentDataStream(), std::move(*left_convert_actions)); converting_step->setStepDescription("Convert joined columns"); steps.emplace_back(converting_step.get()); left.addStep(std::move(converting_step)); diff --git a/cpp-ch/local-engine/Parser/FilterRelParser.cpp b/cpp-ch/local-engine/Parser/FilterRelParser.cpp index e0098f747c2ab..2c99c4788f76c 100644 --- a/cpp-ch/local-engine/Parser/FilterRelParser.cpp +++ b/cpp-ch/local-engine/Parser/FilterRelParser.cpp @@ -35,11 +35,11 @@ DB::QueryPlanPtr FilterRelParser::parse(DB::QueryPlanPtr query_plan, const subst std::string filter_name; auto input_header = query_plan->getCurrentDataStream().header; - DB::ActionsDAGPtr actions_dag = std::make_shared(input_header.getColumnsWithTypeAndName()); + DB::ActionsDAG actions_dag{input_header.getColumnsWithTypeAndName()}; const auto condition_node = parseExpression(actions_dag, filter_rel.condition()); if (filter_rel.condition().has_scalar_function()) { - actions_dag->addOrReplaceInOutputs(*condition_node); + actions_dag.addOrReplaceInOutputs(*condition_node); } filter_name = condition_node->result_name; @@ -51,11 +51,11 @@ DB::QueryPlanPtr FilterRelParser::parse(DB::QueryPlanPtr query_plan, const subst else input_with_condition.insert(condition_node->result_name); - actions_dag->removeUnusedActions(input_with_condition); + actions_dag.removeUnusedActions(input_with_condition); NonNullableColumnsResolver non_nullable_columns_resolver(input_header, *getPlanParser(), filter_rel.condition()); auto non_nullable_columns = non_nullable_columns_resolver.resolve(); - auto filter_step = std::make_unique(query_plan->getCurrentDataStream(), actions_dag, filter_name, remove_filter_column); + auto filter_step = std::make_unique(query_plan->getCurrentDataStream(), std::move(actions_dag), filter_name, remove_filter_column); filter_step->setStepDescription("WHERE"); steps.emplace_back(filter_step.get()); query_plan->addStep(std::move(filter_step)); diff --git a/cpp-ch/local-engine/Parser/FunctionExecutor.cpp b/cpp-ch/local-engine/Parser/FunctionExecutor.cpp index c96d96be4ba13..1b621082cd92e 100644 --- a/cpp-ch/local-engine/Parser/FunctionExecutor.cpp +++ b/cpp-ch/local-engine/Parser/FunctionExecutor.cpp @@ -18,6 +18,7 @@ #include #include +#include namespace DB { @@ -79,12 +80,13 @@ void FunctionExecutor::parseExtensions() void FunctionExecutor::parseExpression() { + DB::ActionsDAG actions_dag{blockToNameAndTypeList(header)}; /// Notice keep_result must be true, because result_node of current function must be output node in actions_dag - auto actions_dag = plan_parser.parseFunction(header, expression, result_name, nullptr, true); + plan_parser.parseFunctionWithDAG(expression, result_name, actions_dag, true); // std::cout << "actions_dag:" << std::endl; // std::cout << actions_dag->dumpDAG() << std::endl; - expression_actions = std::make_unique(actions_dag); + expression_actions = std::make_unique(std::move(actions_dag)); } Block FunctionExecutor::getHeader() const diff --git a/cpp-ch/local-engine/Parser/FunctionParser.cpp b/cpp-ch/local-engine/Parser/FunctionParser.cpp index 513470d0c250b..d46110431ab40 100644 --- a/cpp-ch/local-engine/Parser/FunctionParser.cpp +++ b/cpp-ch/local-engine/Parser/FunctionParser.cpp @@ -47,7 +47,7 @@ String FunctionParser::getCHFunctionName(const substrait::Expression_ScalarFunct } ActionsDAG::NodeRawConstPtrs FunctionParser::parseFunctionArguments( - const substrait::Expression_ScalarFunction & substrait_func, ActionsDAGPtr & actions_dag) const + const substrait::Expression_ScalarFunction & substrait_func, ActionsDAG & actions_dag) const { ActionsDAG::NodeRawConstPtrs parsed_args; const auto & args = substrait_func.arguments(); @@ -59,7 +59,7 @@ ActionsDAG::NodeRawConstPtrs FunctionParser::parseFunctionArguments( const ActionsDAG::Node * -FunctionParser::parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAGPtr & actions_dag) const +FunctionParser::parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAG & actions_dag) const { auto ch_func_name = getCHFunctionName(substrait_func); auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); @@ -68,7 +68,7 @@ FunctionParser::parse(const substrait::Expression_ScalarFunction & substrait_fun } const ActionsDAG::Node * FunctionParser::convertNodeTypeIfNeeded( - const substrait::Expression_ScalarFunction & substrait_func, const ActionsDAG::Node * func_node, ActionsDAGPtr & actions_dag) const + const substrait::Expression_ScalarFunction & substrait_func, const ActionsDAG::Node * func_node, ActionsDAG & actions_dag) const { const auto & output_type = substrait_func.output_type(); if (!TypeParser::isTypeMatched(output_type, func_node->result_type)) diff --git a/cpp-ch/local-engine/Parser/FunctionParser.h b/cpp-ch/local-engine/Parser/FunctionParser.h index 6ac162a953c63..6b8176d931911 100644 --- a/cpp-ch/local-engine/Parser/FunctionParser.h +++ b/cpp-ch/local-engine/Parser/FunctionParser.h @@ -47,7 +47,7 @@ class FunctionParser /// - make a post-projection for the function result. e.g. type conversion. virtual const DB::ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - DB::ActionsDAGPtr & actions_dag) const; + DB::ActionsDAG & actions_dag) const; virtual String getCHFunctionName(const substrait::Expression_ScalarFunction & substrait_func) const; protected: @@ -55,47 +55,47 @@ class FunctionParser virtual DB::ActionsDAG::NodeRawConstPtrs parseFunctionArguments( const substrait::Expression_ScalarFunction & substrait_func, const String & /*function_name*/, - DB::ActionsDAGPtr & actions_dag) const + DB::ActionsDAG & actions_dag) const { return parseFunctionArguments(substrait_func, actions_dag); } virtual DB::ActionsDAG::NodeRawConstPtrs parseFunctionArguments( const substrait::Expression_ScalarFunction & substrait_func, - DB::ActionsDAGPtr & actions_dag) const; + DB::ActionsDAG & actions_dag) const; virtual const DB::ActionsDAG::Node * convertNodeTypeIfNeeded( const substrait::Expression_ScalarFunction & substrait_func, const DB::ActionsDAG::Node * func_node, - DB::ActionsDAGPtr & actions_dag) const; + DB::ActionsDAG & actions_dag) const; DB::ContextPtr getContext() const { return plan_parser->context; } String getUniqueName(const String & name) const { return plan_parser->getUniqueName(name); } - const DB::ActionsDAG::Node * addColumnToActionsDAG(DB::ActionsDAGPtr & actions_dag, const DB::DataTypePtr & type, const DB::Field & field) const + const DB::ActionsDAG::Node * addColumnToActionsDAG(DB::ActionsDAG & actions_dag, const DB::DataTypePtr & type, const DB::Field & field) const { - return &actions_dag->addColumn(ColumnWithTypeAndName(type->createColumnConst(1, field), type, getUniqueName(toString(field)))); + return &actions_dag.addColumn(ColumnWithTypeAndName(type->createColumnConst(1, field), type, getUniqueName(toString(field)))); } const DB::ActionsDAG::Node * - toFunctionNode(DB::ActionsDAGPtr & action_dag, const String & func_name, const DB::ActionsDAG::NodeRawConstPtrs & args) const + toFunctionNode(DB::ActionsDAG & action_dag, const String & func_name, const DB::ActionsDAG::NodeRawConstPtrs & args) const { return plan_parser->toFunctionNode(action_dag, func_name, args); } const DB::ActionsDAG::Node * - toFunctionNode(DB::ActionsDAGPtr & action_dag, const String & func_name, const String & result_name, const DB::ActionsDAG::NodeRawConstPtrs & args) const + toFunctionNode(DB::ActionsDAG & action_dag, const String & func_name, const String & result_name, const DB::ActionsDAG::NodeRawConstPtrs & args) const { auto function_builder = DB::FunctionFactory::instance().get(func_name, getContext()); - return &action_dag->addFunction(function_builder, args, result_name); + return &action_dag.addFunction(function_builder, args, result_name); } const ActionsDAG::Node * - parseFunctionWithDAG(const substrait::Expression & rel, std::string & result_name, DB::ActionsDAGPtr actions_dag, bool keep_result = false) const + parseFunctionWithDAG(const substrait::Expression & rel, std::string & result_name, DB::ActionsDAG& actions_dag, bool keep_result = false) const { return plan_parser->parseFunctionWithDAG(rel, result_name, actions_dag, keep_result); } - const DB::ActionsDAG::Node * parseExpression(DB::ActionsDAGPtr actions_dag, const substrait::Expression & rel) const + const DB::ActionsDAG::Node * parseExpression(DB::ActionsDAG& actions_dag, const substrait::Expression & rel) const { return plan_parser->parseExpression(actions_dag, rel); } diff --git a/cpp-ch/local-engine/Parser/JoinRelParser.cpp b/cpp-ch/local-engine/Parser/JoinRelParser.cpp index 24ba7acdb6545..8d0891bd03850 100644 --- a/cpp-ch/local-engine/Parser/JoinRelParser.cpp +++ b/cpp-ch/local-engine/Parser/JoinRelParser.cpp @@ -164,18 +164,16 @@ void JoinRelParser::renamePlanColumns(DB::QueryPlan & left, DB::QueryPlan & righ { /// To support mixed join conditions, we must make sure that the column names in the right be the same as /// storage_join's right sample block. - ActionsDAGPtr project = ActionsDAG::makeConvertingActions( + ActionsDAG right_project = ActionsDAG::makeConvertingActions( right.getCurrentDataStream().header.getColumnsWithTypeAndName(), storage_join.getRightSampleBlock().getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Position); - if (project) - { - QueryPlanStepPtr project_step = std::make_unique(right.getCurrentDataStream(), project); - project_step->setStepDescription("Rename Broadcast Table Name"); - steps.emplace_back(project_step.get()); - right.addStep(std::move(project_step)); - } + QueryPlanStepPtr right_project_step = + std::make_unique(right.getCurrentDataStream(), std::move(right_project)); + right_project_step->setStepDescription("Rename Broadcast Table Name"); + steps.emplace_back(right_project_step.get()); + right.addStep(std::move(right_project_step)); /// If the columns name in right table is duplicated with left table, we need to rename the left table's columns, /// avoid the columns name in the right table be changed in `addConvertStep`. @@ -194,18 +192,16 @@ void JoinRelParser::renamePlanColumns(DB::QueryPlan & left, DB::QueryPlan & righ new_left_cols.emplace_back(col.column, col.type, col.name); } } - project = ActionsDAG::makeConvertingActions( + ActionsDAG left_project = ActionsDAG::makeConvertingActions( left.getCurrentDataStream().header.getColumnsWithTypeAndName(), new_left_cols, ActionsDAG::MatchColumnsMode::Position); - if (project) - { - QueryPlanStepPtr project_step = std::make_unique(left.getCurrentDataStream(), project); - project_step->setStepDescription("Rename Left Table Name for broadcast join"); - steps.emplace_back(project_step.get()); - left.addStep(std::move(project_step)); - } + QueryPlanStepPtr left_project_step = + std::make_unique(left.getCurrentDataStream(), std::move(left_project)); + left_project_step->setStepDescription("Rename Left Table Name for broadcast join"); + steps.emplace_back(left_project_step.get()); + left.addStep(std::move(left_project_step)); } DB::QueryPlanPtr JoinRelParser::parseJoin(const substrait::JoinRel & join, DB::QueryPlanPtr left, DB::QueryPlanPtr right) @@ -370,15 +366,15 @@ DB::QueryPlanPtr JoinRelParser::parseJoin(const substrait::JoinRel & join, DB::Q /// we mark the flag 0, otherwise mark it 1. void JoinRelParser::existenceJoinPostProject(DB::QueryPlan & plan, const DB::Names & left_input_cols) { - auto actions_dag = std::make_shared(plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); - const auto * right_col_node = actions_dag->getInputs().back(); + DB::ActionsDAG actions_dag{plan.getCurrentDataStream().header.getColumnsWithTypeAndName()}; + const auto * right_col_node = actions_dag.getInputs().back(); auto function_builder = DB::FunctionFactory::instance().get("isNotNull", getContext()); - const auto * not_null_node = &actions_dag->addFunction(function_builder, {right_col_node}, right_col_node->result_name); - actions_dag->addOrReplaceInOutputs(*not_null_node); + const auto * not_null_node = &actions_dag.addFunction(function_builder, {right_col_node}, right_col_node->result_name); + actions_dag.addOrReplaceInOutputs(*not_null_node); DB::Names required_cols = left_input_cols; required_cols.emplace_back(not_null_node->result_name); - actions_dag->removeUnusedActions(required_cols); - auto project_step = std::make_unique(plan.getCurrentDataStream(), actions_dag); + actions_dag.removeUnusedActions(required_cols); + auto project_step = std::make_unique(plan.getCurrentDataStream(), std::move(actions_dag)); project_step->setStepDescription("ExistenceJoin Post Project"); steps.emplace_back(project_step.get()); plan.addStep(std::move(project_step)); @@ -406,19 +402,19 @@ void JoinRelParser::addConvertStep(TableJoin & table_join, DB::QueryPlan & left, } if (!right_table_alias.empty()) { - ActionsDAGPtr rename_dag = std::make_shared(right.getCurrentDataStream().header.getNamesAndTypesList()); + ActionsDAG rename_dag{right.getCurrentDataStream().header.getNamesAndTypesList()}; auto original_right_columns = right.getCurrentDataStream().header; for (const auto & column_alias : right_table_alias) { if (original_right_columns.has(column_alias.first)) { auto pos = original_right_columns.getPositionByName(column_alias.first); - const auto & alias = rename_dag->addAlias(*rename_dag->getInputs()[pos], column_alias.second); - rename_dag->getOutputs()[pos] = &alias; + const auto & alias = rename_dag.addAlias(*rename_dag.getInputs()[pos], column_alias.second); + rename_dag.getOutputs()[pos] = &alias; } } - QueryPlanStepPtr project_step = std::make_unique(right.getCurrentDataStream(), rename_dag); + QueryPlanStepPtr project_step = std::make_unique(right.getCurrentDataStream(), std::move(rename_dag)); project_step->setStepDescription("Right Table Rename"); steps.emplace_back(project_step.get()); right.addStep(std::move(project_step)); @@ -428,14 +424,14 @@ void JoinRelParser::addConvertStep(TableJoin & table_join, DB::QueryPlan & left, { table_join.addJoinedColumn(column); } - ActionsDAGPtr left_convert_actions = nullptr; - ActionsDAGPtr right_convert_actions = nullptr; + std::optional left_convert_actions; + std::optional right_convert_actions; std::tie(left_convert_actions, right_convert_actions) = table_join.createConvertingActions( left.getCurrentDataStream().header.getColumnsWithTypeAndName(), right.getCurrentDataStream().header.getColumnsWithTypeAndName()); if (right_convert_actions) { - auto converting_step = std::make_unique(right.getCurrentDataStream(), right_convert_actions); + auto converting_step = std::make_unique(right.getCurrentDataStream(), std::move(*right_convert_actions)); converting_step->setStepDescription("Convert joined columns"); steps.emplace_back(converting_step.get()); right.addStep(std::move(converting_step)); @@ -443,7 +439,7 @@ void JoinRelParser::addConvertStep(TableJoin & table_join, DB::QueryPlan & left, if (left_convert_actions) { - auto converting_step = std::make_unique(left.getCurrentDataStream(), left_convert_actions); + auto converting_step = std::make_unique(left.getCurrentDataStream(), std::move(*left_convert_actions)); converting_step->setStepDescription("Convert joined columns"); steps.emplace_back(converting_step.get()); left.addStep(std::move(converting_step)); @@ -564,8 +560,8 @@ bool JoinRelParser::applyJoinFilter( auto input_exprs = get_input_expressions(left_header); input_exprs.push_back(expr); auto actions_dag = expressionsToActionsDAG(input_exprs, left_header); - table_join.getClauses().back().analyzer_left_filter_condition_column_name = actions_dag->getOutputs().back()->result_name; - QueryPlanStepPtr before_join_step = std::make_unique(left.getCurrentDataStream(), actions_dag); + table_join.getClauses().back().analyzer_left_filter_condition_column_name = actions_dag.getOutputs().back()->result_name; + QueryPlanStepPtr before_join_step = std::make_unique(left.getCurrentDataStream(), std::move(actions_dag)); before_join_step->setStepDescription("Before JOIN LEFT"); steps.emplace_back(before_join_step.get()); left.addStep(std::move(before_join_step)); @@ -581,12 +577,12 @@ bool JoinRelParser::applyJoinFilter( /// clear unused columns in actions_dag for (const auto & col : left_header.getColumnsWithTypeAndName()) { - actions_dag->removeUnusedResult(col.name); + actions_dag.removeUnusedResult(col.name); } - actions_dag->removeUnusedActions(); + actions_dag.removeUnusedActions(); - table_join.getClauses().back().analyzer_right_filter_condition_column_name = actions_dag->getOutputs().back()->result_name; - QueryPlanStepPtr before_join_step = std::make_unique(right.getCurrentDataStream(), actions_dag); + table_join.getClauses().back().analyzer_right_filter_condition_column_name = actions_dag.getOutputs().back()->result_name; + QueryPlanStepPtr before_join_step = std::make_unique(right.getCurrentDataStream(), std::move(actions_dag)); before_join_step->setStepDescription("Before JOIN RIGHT"); steps.emplace_back(before_join_step.get()); right.addStep(std::move(before_join_step)); @@ -598,7 +594,7 @@ bool JoinRelParser::applyJoinFilter( return false; auto mixed_join_expressions_actions = expressionsToActionsDAG({expr}, mixed_header); table_join.getMixedJoinExpression() - = std::make_shared(mixed_join_expressions_actions, ExpressionActionsSettings::fromContext(context)); + = std::make_shared(std::move(mixed_join_expressions_actions), ExpressionActionsSettings::fromContext(context)); } else { @@ -610,7 +606,7 @@ bool JoinRelParser::applyJoinFilter( void JoinRelParser::addPostFilter(DB::QueryPlan & query_plan, const substrait::JoinRel & join) { std::string filter_name; - auto actions_dag = std::make_shared(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); + ActionsDAG actions_dag{query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()}; if (!join.post_join_filter().has_scalar_function()) { // It may be singular_or_list @@ -619,9 +615,9 @@ void JoinRelParser::addPostFilter(DB::QueryPlan & query_plan, const substrait::J } else { - getPlanParser()->parseFunction(query_plan.getCurrentDataStream().header, join.post_join_filter(), filter_name, actions_dag, true); + getPlanParser()->parseFunctionWithDAG(join.post_join_filter(), filter_name, actions_dag, true); } - auto filter_step = std::make_unique(query_plan.getCurrentDataStream(), actions_dag, filter_name, true); + auto filter_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(actions_dag), filter_name, true); filter_step->setStepDescription("Post Join Filter"); steps.emplace_back(filter_step.get()); query_plan.addStep(std::move(filter_step)); diff --git a/cpp-ch/local-engine/Parser/MergeTreeRelParser.cpp b/cpp-ch/local-engine/Parser/MergeTreeRelParser.cpp index 7d906a837441e..15e44b7ee591d 100644 --- a/cpp-ch/local-engine/Parser/MergeTreeRelParser.cpp +++ b/cpp-ch/local-engine/Parser/MergeTreeRelParser.cpp @@ -179,7 +179,7 @@ DB::QueryPlanPtr MergeTreeRelParser::parseReadRel( auto * source_step_with_filter = static_cast(read_step.get()); if (const auto & storage_prewhere_info = query_info->prewhere_info) { - source_step_with_filter->addFilter(storage_prewhere_info->prewhere_actions, storage_prewhere_info->prewhere_column_name); + source_step_with_filter->addFilter(storage_prewhere_info->prewhere_actions.clone(), storage_prewhere_info->prewhere_column_name); source_step_with_filter->applyFilters(); } @@ -213,11 +213,11 @@ PrewhereInfoPtr MergeTreeRelParser::parsePreWhereInfo(const substrait::Expressio prewhere_info->remove_prewhere_column = true; for (const auto & name : input.getNames()) - prewhere_info->prewhere_actions->tryRestoreColumn(name); + prewhere_info->prewhere_actions.tryRestoreColumn(name); return prewhere_info; } -DB::ActionsDAGPtr MergeTreeRelParser::optimizePrewhereAction(const substrait::Expression & rel, std::string & filter_name, Block & block) +DB::ActionsDAG MergeTreeRelParser::optimizePrewhereAction(const substrait::Expression & rel, std::string & filter_name, Block & block) { Conditions res; std::set pk_positions; @@ -238,7 +238,7 @@ DB::ActionsDAGPtr MergeTreeRelParser::optimizePrewhereAction(const substrait::Ex // filter less size column first res.sort(); - auto filter_action = std::make_shared(block.getNamesAndTypesList()); + ActionsDAG filter_action{block.getNamesAndTypesList()}; if (res.size() == 1) { @@ -252,28 +252,28 @@ DB::ActionsDAGPtr MergeTreeRelParser::optimizePrewhereAction(const substrait::Ex { String ignore; parseToAction(filter_action, cond.node, ignore); - args.emplace_back(&filter_action->getNodes().back()); + args.emplace_back(&filter_action.getNodes().back()); } auto function_builder = FunctionFactory::instance().get("and", context); std::string args_name = join(args, ','); filter_name = +"and(" + args_name + ")"; - const auto * and_function = &filter_action->addFunction(function_builder, args, filter_name); - filter_action->addOrReplaceInOutputs(*and_function); + const auto * and_function = &filter_action.addFunction(function_builder, args, filter_name); + filter_action.addOrReplaceInOutputs(*and_function); } - filter_action->removeUnusedActions(Names{filter_name}, false, true); + filter_action.removeUnusedActions(Names{filter_name}, false, true); return filter_action; } -void MergeTreeRelParser::parseToAction(ActionsDAGPtr & filter_action, const substrait::Expression & rel, std::string & filter_name) +void MergeTreeRelParser::parseToAction(ActionsDAG & filter_action, const substrait::Expression & rel, std::string & filter_name) { if (rel.has_scalar_function()) getPlanParser()->parseFunctionWithDAG(rel, filter_name, filter_action, true); else { const auto * in_node = parseExpression(filter_action, rel); - filter_action->addOrReplaceInOutputs(*in_node); + filter_action.addOrReplaceInOutputs(*in_node); filter_name = in_node->result_name; } } @@ -423,7 +423,7 @@ String MergeTreeRelParser::filterRangesOnDriver(const substrait::ReadRel & read_ { ActionDAGNodes filter_nodes; filter_nodes.nodes.emplace_back( - &storage_prewhere_info->prewhere_actions->findInOutputs(storage_prewhere_info->prewhere_column_name)); + &storage_prewhere_info->prewhere_actions.findInOutputs(storage_prewhere_info->prewhere_column_name)); read_from_mergetree->applyFilters(std::move(filter_nodes)); } diff --git a/cpp-ch/local-engine/Parser/MergeTreeRelParser.h b/cpp-ch/local-engine/Parser/MergeTreeRelParser.h index bf27b184f9876..1c9ea736cd43a 100644 --- a/cpp-ch/local-engine/Parser/MergeTreeRelParser.h +++ b/cpp-ch/local-engine/Parser/MergeTreeRelParser.h @@ -100,9 +100,9 @@ class MergeTreeRelParser : public RelParser std::unordered_map column_sizes; private: - void parseToAction(ActionsDAGPtr & filter_action, const substrait::Expression & rel, std::string & filter_name); + void parseToAction(ActionsDAG & filter_action, const substrait::Expression & rel, std::string & filter_name); PrewhereInfoPtr parsePreWhereInfo(const substrait::Expression & rel, Block & input); - ActionsDAGPtr optimizePrewhereAction(const substrait::Expression & rel, std::string & filter_name, Block & block); + ActionsDAG optimizePrewhereAction(const substrait::Expression & rel, std::string & filter_name, Block & block); String getCHFunctionName(const substrait::Expression_ScalarFunction & substrait_func); void collectColumns(const substrait::Expression & rel, NameSet & columns, Block & block); UInt64 getColumnsSize(const NameSet & columns); diff --git a/cpp-ch/local-engine/Parser/ProjectRelParser.cpp b/cpp-ch/local-engine/Parser/ProjectRelParser.cpp index 2f75ac396dfe3..6fb1f3d961cc9 100644 --- a/cpp-ch/local-engine/Parser/ProjectRelParser.cpp +++ b/cpp-ch/local-engine/Parser/ProjectRelParser.cpp @@ -62,7 +62,7 @@ ProjectRelParser::parseProject(DB::QueryPlanPtr query_plan, const substrait::Rel expressions.emplace_back(project_rel.expressions(i)); } auto actions_dag = expressionsToActionsDAG(expressions, header); - auto expression_step = std::make_unique(query_plan->getCurrentDataStream(), actions_dag); + auto expression_step = std::make_unique(query_plan->getCurrentDataStream(), std::move(actions_dag)); expression_step->setStepDescription("Project"); steps.emplace_back(expression_step.get()); query_plan->addStep(std::move(expression_step)); @@ -78,10 +78,10 @@ ProjectRelParser::parseProject(DB::QueryPlanPtr query_plan, const substrait::Rel } } -const DB::ActionsDAG::Node * ProjectRelParser::findArrayJoinNode(ActionsDAGPtr actions_dag) +const DB::ActionsDAG::Node * ProjectRelParser::findArrayJoinNode(const ActionsDAG& actions_dag) { const ActionsDAG::Node * array_join_node = nullptr; - const auto & nodes = actions_dag->getNodes(); + const auto & nodes = actions_dag.getNodes(); for (const auto & node : nodes) { if (node.type == ActionsDAG::ActionType::ARRAY_JOIN) @@ -94,21 +94,21 @@ const DB::ActionsDAG::Node * ProjectRelParser::findArrayJoinNode(ActionsDAGPtr a return array_join_node; } -ProjectRelParser::SplittedActionsDAGs ProjectRelParser::splitActionsDAGInGenerate(ActionsDAGPtr actions_dag) +ProjectRelParser::SplittedActionsDAGs ProjectRelParser::splitActionsDAGInGenerate(const ActionsDAG& actions_dag) { SplittedActionsDAGs res; auto array_join_node = findArrayJoinNode(actions_dag); std::unordered_set first_split_nodes(array_join_node->children.begin(), array_join_node->children.end()); - auto first_split_result = actions_dag->split(first_split_nodes); - res.before_array_join = first_split_result.first; + auto first_split_result = actions_dag.split(first_split_nodes); + res.before_array_join = std::move(first_split_result.first); array_join_node = findArrayJoinNode(first_split_result.second); std::unordered_set second_split_nodes = {array_join_node}; - auto second_split_result = first_split_result.second->split(second_split_nodes); - res.array_join = second_split_result.first; - second_split_result.second->removeUnusedActions(); - res.after_array_join = second_split_result.second; + auto second_split_result = first_split_result.second.split(second_split_nodes); + res.array_join = std::move(second_split_result.first); + second_split_result.second.removeUnusedActions(); + res.after_array_join = std::move(second_split_result.second); return res; } @@ -126,7 +126,7 @@ DB::QueryPlanPtr ProjectRelParser::parseReplicateRows(DB::QueryPlanPtr query_pla } auto header = query_plan->getCurrentDataStream().header; auto actions_dag = expressionsToActionsDAG(expressions, header); - auto before_replicate_rows = std::make_unique(query_plan->getCurrentDataStream(), actions_dag); + auto before_replicate_rows = std::make_unique(query_plan->getCurrentDataStream(), std::move(actions_dag)); before_replicate_rows->setStepDescription("Before ReplicateRows"); steps.emplace_back(before_replicate_rows.get()); query_plan->addStep(std::move(before_replicate_rows)); @@ -159,7 +159,7 @@ ProjectRelParser::parseGenerate(DB::QueryPlanPtr query_plan, const substrait::Re if (!findArrayJoinNode(actions_dag)) { /// If generator in generate rel is not explode/posexplode, e.g. json_tuple - auto expression_step = std::make_unique(query_plan->getCurrentDataStream(), actions_dag); + auto expression_step = std::make_unique(query_plan->getCurrentDataStream(), std::move(actions_dag)); expression_step->setStepDescription("Generate"); steps.emplace_back(expression_step.get()); query_plan->addStep(std::move(expression_step)); @@ -168,13 +168,13 @@ ProjectRelParser::parseGenerate(DB::QueryPlanPtr query_plan, const substrait::Re { /// If generator in generate rel is explode/posexplode, transform arrayJoin function to ARRAY JOIN STEP to apply max_block_size /// which avoids OOM when several lateral view explode/posexplode is used in spark sqls - LOG_DEBUG(logger, "original actions_dag:{}", actions_dag->dumpDAG()); + LOG_DEBUG(logger, "original actions_dag:{}", actions_dag.dumpDAG()); auto splitted_actions_dags = splitActionsDAGInGenerate(actions_dag); - LOG_DEBUG(logger, "actions_dag before arrayJoin:{}", splitted_actions_dags.before_array_join->dumpDAG()); - LOG_DEBUG(logger, "actions_dag during arrayJoin:{}", splitted_actions_dags.array_join->dumpDAG()); - LOG_DEBUG(logger, "actions_dag after arrayJoin:{}", splitted_actions_dags.after_array_join->dumpDAG()); + LOG_DEBUG(logger, "actions_dag before arrayJoin:{}", splitted_actions_dags.before_array_join.dumpDAG()); + LOG_DEBUG(logger, "actions_dag during arrayJoin:{}", splitted_actions_dags.array_join.dumpDAG()); + LOG_DEBUG(logger, "actions_dag after arrayJoin:{}", splitted_actions_dags.after_array_join.dumpDAG()); - auto ignore_actions_dag = [](ActionsDAGPtr actions_dag_) -> bool + auto ignore_actions_dag = [](const ActionsDAG& actions_dag_) -> bool { /* We should ignore actions_dag like: @@ -182,16 +182,15 @@ ProjectRelParser::parseGenerate(DB::QueryPlanPtr query_plan, const substrait::Re 1 : INPUT () (no column) String b Output nodes: 0, 1 */ - return actions_dag_->getOutputs().size() == actions_dag_->getNodes().size() - && actions_dag_->getInputs().size() == actions_dag_->getNodes().size(); + return actions_dag_.getOutputs().size() == actions_dag_.getNodes().size() + && actions_dag_.getInputs().size() == actions_dag_.getNodes().size(); }; /// Pre-projection before array join - const auto & before_array_join = splitted_actions_dags.before_array_join; - if (!ignore_actions_dag(before_array_join)) + if (!ignore_actions_dag(splitted_actions_dags.before_array_join)) { auto step_before_array_join - = std::make_unique(query_plan->getCurrentDataStream(), splitted_actions_dags.before_array_join); + = std::make_unique(query_plan->getCurrentDataStream(), std::move(splitted_actions_dags.before_array_join)); step_before_array_join->setStepDescription("Pre-projection In Generate"); steps.emplace_back(step_before_array_join.get()); query_plan->addStep(std::move(step_before_array_join)); @@ -199,7 +198,7 @@ ProjectRelParser::parseGenerate(DB::QueryPlanPtr query_plan, const substrait::Re } /// ARRAY JOIN - NameSet array_joined_columns = {findArrayJoinNode(splitted_actions_dags.array_join)->result_name}; + NameSet array_joined_columns{findArrayJoinNode(splitted_actions_dags.array_join)->result_name}; auto array_join_action = std::make_shared(array_joined_columns, false, getContext()); auto array_join_step = std::make_unique(query_plan->getCurrentDataStream(), array_join_action); array_join_step->setStepDescription("ARRAY JOIN In Generate"); @@ -208,10 +207,9 @@ ProjectRelParser::parseGenerate(DB::QueryPlanPtr query_plan, const substrait::Re // LOG_DEBUG(logger, "plan2:{}", PlanUtil::explainPlan(*query_plan)); /// Post-projection after array join(Optional) - const auto & after_array_join = splitted_actions_dags.after_array_join; - if (!ignore_actions_dag(after_array_join)) + if (!ignore_actions_dag(splitted_actions_dags.after_array_join)) { - auto step_after_array_join = std::make_unique(query_plan->getCurrentDataStream(), after_array_join); + auto step_after_array_join = std::make_unique(query_plan->getCurrentDataStream(), std::move(splitted_actions_dags.after_array_join)); step_after_array_join->setStepDescription("Post-projection In Generate"); steps.emplace_back(step_after_array_join.get()); query_plan->addStep(std::move(step_after_array_join)); diff --git a/cpp-ch/local-engine/Parser/ProjectRelParser.h b/cpp-ch/local-engine/Parser/ProjectRelParser.h index 328acfc72fd5f..94accff2dc513 100644 --- a/cpp-ch/local-engine/Parser/ProjectRelParser.h +++ b/cpp-ch/local-engine/Parser/ProjectRelParser.h @@ -27,9 +27,9 @@ class ProjectRelParser : public RelParser public: struct SplittedActionsDAGs { - ActionsDAGPtr before_array_join; /// Optional - ActionsDAGPtr array_join; - ActionsDAGPtr after_array_join; /// Optional + ActionsDAG before_array_join; /// Optional + ActionsDAG array_join; + ActionsDAG after_array_join; /// Optional }; explicit ProjectRelParser(SerializedPlanParser * plan_paser_); @@ -44,10 +44,10 @@ class ProjectRelParser : public RelParser DB::QueryPlanPtr parseProject(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, std::list & rel_stack_); DB::QueryPlanPtr parseGenerate(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, std::list & rel_stack_); - static const DB::ActionsDAG::Node * findArrayJoinNode(ActionsDAGPtr actions_dag); + static const DB::ActionsDAG::Node * findArrayJoinNode(const ActionsDAG& actions_dag); /// Split actions_dag of generate rel into 3 parts: before array join + during array join + after array join - static SplittedActionsDAGs splitActionsDAGInGenerate(ActionsDAGPtr actions_dag); + static SplittedActionsDAGs splitActionsDAGInGenerate(const ActionsDAG& actions_dag); bool isReplicateRows(substrait::GenerateRel rel); diff --git a/cpp-ch/local-engine/Parser/RelParser.h b/cpp-ch/local-engine/Parser/RelParser.h index 0228c2867a269..885622281eaac 100644 --- a/cpp-ch/local-engine/Parser/RelParser.h +++ b/cpp-ch/local-engine/Parser/RelParser.h @@ -59,16 +59,16 @@ class RelParser // Get coresponding function name in ClickHouse. std::optional parseFunctionName(UInt32 function_ref, const substrait::Expression_ScalarFunction & function); - const DB::ActionsDAG::Node * parseArgument(ActionsDAGPtr action_dag, const substrait::Expression & rel) + const DB::ActionsDAG::Node * parseArgument(ActionsDAG& action_dag, const substrait::Expression & rel) { return plan_parser->parseExpression(action_dag, rel); } - const DB::ActionsDAG::Node * parseExpression(ActionsDAGPtr action_dag, const substrait::Expression & rel) + const DB::ActionsDAG::Node * parseExpression(ActionsDAG& action_dag, const substrait::Expression & rel) { return plan_parser->parseExpression(action_dag, rel); } - DB::ActionsDAGPtr expressionsToActionsDAG(const std::vector & expressions, const DB::Block & header) + DB::ActionsDAG expressionsToActionsDAG(const std::vector & expressions, const DB::Block & header) { return plan_parser->expressionsToActionsDAG(expressions, header, header); } @@ -77,7 +77,7 @@ class RelParser std::vector steps; const ActionsDAG::Node * - buildFunctionNode(ActionsDAGPtr action_dag, const String & function, const DB::ActionsDAG::NodeRawConstPtrs & args) + buildFunctionNode(ActionsDAG& action_dag, const String & function, const DB::ActionsDAG::NodeRawConstPtrs & args) { return plan_parser->toFunctionNode(action_dag, function, args); } diff --git a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp index 5aaf006a362e8..bff296de57172 100644 --- a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp +++ b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp @@ -134,9 +134,9 @@ void logDebugMessage(const google::protobuf::Message & message, const char * typ } } -const ActionsDAG::Node * SerializedPlanParser::addColumn(ActionsDAGPtr actions_dag, const DataTypePtr & type, const Field & field) +const ActionsDAG::Node * SerializedPlanParser::addColumn(ActionsDAG& actions_dag, const DataTypePtr & type, const Field & field) { - return &actions_dag->addColumn( + return &actions_dag.addColumn( ColumnWithTypeAndName(type->createColumnConst(1, field), type, getUniqueName(toString(field).substr(0, 10)))); } @@ -154,10 +154,10 @@ void SerializedPlanParser::parseExtensions( } } -std::shared_ptr SerializedPlanParser::expressionsToActionsDAG( +ActionsDAG SerializedPlanParser::expressionsToActionsDAG( const std::vector & expressions, const Block & header, const Block & read_schema) { - auto actions_dag = std::make_shared(blockToNameAndTypeList(header)); + ActionsDAG actions_dag{blockToNameAndTypeList(header)}; NamesWithAliases required_columns; std::set distinct_columns; @@ -167,7 +167,7 @@ std::shared_ptr SerializedPlanParser::expressionsToActionsDAG( { auto position = expr.selection().direct_reference().struct_field().field(); auto col_name = read_schema.getByPosition(position).name; - const ActionsDAG::Node * field = actions_dag->tryFindInOutputs(col_name); + const ActionsDAG::Node * field = actions_dag.tryFindInOutputs(col_name); if (distinct_columns.contains(field->result_name)) { auto unique_name = getUniqueName(field->result_name); @@ -187,15 +187,15 @@ std::shared_ptr SerializedPlanParser::expressionsToActionsDAG( std::vector result_names; if (startsWith(function_signature, "explode:")) - actions_dag = parseArrayJoin(header, expr, result_names, actions_dag, true, false); + parseArrayJoinWithDAG(expr, result_names, actions_dag, true, false); else if (startsWith(function_signature, "posexplode:")) - actions_dag = parseArrayJoin(header, expr, result_names, actions_dag, true, true); + parseArrayJoinWithDAG(expr, result_names, actions_dag, true, true); else if (startsWith(function_signature, "json_tuple:")) - actions_dag = parseJsonTuple(header, expr, result_names, actions_dag, true, false); + parseJsonTuple(expr, result_names, actions_dag, true, false); else { result_names.resize(1); - actions_dag = parseFunction(header, expr, result_names[0], actions_dag, true); + parseFunctionWithDAG(expr, result_names[0], actions_dag, true); } for (const auto & result_name : result_names) @@ -219,7 +219,7 @@ std::shared_ptr SerializedPlanParser::expressionsToActionsDAG( else if (expr.has_cast() || expr.has_if_then() || expr.has_literal() || expr.has_singular_or_list()) { const auto * node = parseExpression(actions_dag, expr); - actions_dag->addOrReplaceInOutputs(*node); + actions_dag.addOrReplaceInOutputs(*node); if (distinct_columns.contains(node->result_name)) { auto unique_name = getUniqueName(node->result_name); @@ -235,8 +235,8 @@ std::shared_ptr SerializedPlanParser::expressionsToActionsDAG( else throw Exception(ErrorCodes::BAD_ARGUMENTS, "unsupported projection type {}.", magic_enum::enum_name(expr.rex_type_case())); } - actions_dag->project(required_columns); - actions_dag->appendInputsForUnusedColumns(header); + actions_dag.project(required_columns); + actions_dag.appendInputsForUnusedColumns(header); return actions_dag; } @@ -292,11 +292,11 @@ QueryPlanStepPtr SerializedPlanParser::parseReadRealWithLocalFile(const substrai source_step->setStepDescription("read local files"); if (rel.has_filter()) { - const ActionsDAGPtr actions_dag = std::make_shared(blockToNameAndTypeList(header)); + ActionsDAG actions_dag{blockToNameAndTypeList(header)}; const ActionsDAG::Node * filter_node = parseExpression(actions_dag, rel.filter()); - actions_dag->addOrReplaceInOutputs(*filter_node); - assert(filter_node == &(actions_dag->findInOutputs(filter_node->result_name))); - source_step->addFilter(actions_dag, filter_node->result_name); + actions_dag.addOrReplaceInOutputs(*filter_node); + assert(filter_node == &(actions_dag.findInOutputs(filter_node->result_name))); + source_step->addFilter(std::move(actions_dag), filter_node->result_name); } return source_step; } @@ -329,9 +329,9 @@ IQueryPlanStep * SerializedPlanParser::addRemoveNullableStep(QueryPlan & plan, c if (columns.empty()) return nullptr; - auto remove_nullable_actions_dag = std::make_shared(blockToNameAndTypeList(plan.getCurrentDataStream().header)); + ActionsDAG remove_nullable_actions_dag{blockToNameAndTypeList(plan.getCurrentDataStream().header)}; removeNullableForRequiredColumns(columns, remove_nullable_actions_dag); - auto expression_step = std::make_unique(plan.getCurrentDataStream(), remove_nullable_actions_dag); + auto expression_step = std::make_unique(plan.getCurrentDataStream(), std::move(remove_nullable_actions_dag)); expression_step->setStepDescription("Remove nullable properties"); auto * step_ptr = expression_step.get(); plan.addStep(std::move(expression_step)); @@ -344,7 +344,7 @@ IQueryPlanStep * SerializedPlanParser::addRollbackFilterHeaderStep(QueryPlanPtr query_plan->getCurrentDataStream().header.getColumnsWithTypeAndName(), input_header.getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); - auto expression_step = std::make_unique(query_plan->getCurrentDataStream(), convert_actions_dag); + auto expression_step = std::make_unique(query_plan->getCurrentDataStream(), std::move(convert_actions_dag)); expression_step->setStepDescription("Generator for rollback filter"); auto * step_ptr = expression_step.get(); query_plan->addStep(std::move(expression_step)); @@ -355,15 +355,15 @@ void adjustOutput(const DB::QueryPlanPtr & query_plan, const substrait::PlanRel { if (root_rel.root().names_size()) { - ActionsDAGPtr actions_dag = std::make_shared(blockToNameAndTypeList(query_plan->getCurrentDataStream().header)); + ActionsDAG actions_dag{blockToNameAndTypeList(query_plan->getCurrentDataStream().header)}; NamesWithAliases aliases; auto cols = query_plan->getCurrentDataStream().header.getNamesAndTypesList(); if (cols.getNames().size() != static_cast(root_rel.root().names_size())) throw Exception(ErrorCodes::LOGICAL_ERROR, "Missmatch result columns size."); for (int i = 0; i < static_cast(cols.getNames().size()); i++) aliases.emplace_back(NameWithAlias(cols.getNames()[i], root_rel.root().names(i))); - actions_dag->project(aliases); - auto expression_step = std::make_unique(query_plan->getCurrentDataStream(), actions_dag); + actions_dag.project(aliases); + auto expression_step = std::make_unique(query_plan->getCurrentDataStream(), std::move(actions_dag)); expression_step->setStepDescription("Rename Output"); query_plan->addStep(std::move(expression_step)); } @@ -405,9 +405,9 @@ void adjustOutput(const DB::QueryPlanPtr & query_plan, const substrait::PlanRel } if (need_final_project) { - ActionsDAGPtr final_project + ActionsDAG final_project = ActionsDAG::makeConvertingActions(original_cols, final_cols, ActionsDAG::MatchColumnsMode::Position); - QueryPlanStepPtr final_project_step = std::make_unique(query_plan->getCurrentDataStream(), final_project); + QueryPlanStepPtr final_project_step = std::make_unique(query_plan->getCurrentDataStream(), std::move(final_project)); final_project_step->setStepDescription("Project for output schema"); query_plan->addStep(std::move(final_project_step)); } @@ -560,7 +560,7 @@ SerializedPlanParser::getFunctionName(const std::string & function_signature, co } void SerializedPlanParser::parseArrayJoinArguments( - ActionsDAGPtr & actions_dag, + ActionsDAG & actions_dag, const std::string & function_name, const substrait::Expression_ScalarFunction & scalar_function, bool position, @@ -597,7 +597,7 @@ void SerializedPlanParser::parseArrayJoinArguments( /// assumeNotNull(ifNull(arg, array())) or assumeNotNull(ifNull(arg, map())) const auto * not_null_node = toFunctionNode(actions_dag, "assumeNotNull", {if_null_node}); /// Wrap with materalize function to make sure column input to ARRAY JOIN STEP is materaized - arg = &actions_dag->materializeNode(*not_null_node); + arg = &actions_dag.materializeNode(*not_null_node); /// If spark function is posexplode, we need to add position column together with input argument if (position) @@ -614,7 +614,7 @@ void SerializedPlanParser::parseArrayJoinArguments( } ActionsDAG::NodeRawConstPtrs SerializedPlanParser::parseArrayJoinWithDAG( - const substrait::Expression & rel, std::vector & result_names, ActionsDAGPtr actions_dag, bool keep_result, bool position) + const substrait::Expression & rel, std::vector & result_names, ActionsDAG& actions_dag, bool keep_result, bool position) { if (!rel.has_scalar_function()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The root of expression should be a scalar function:\n {}", rel.DebugString()); @@ -634,16 +634,16 @@ ActionsDAG::NodeRawConstPtrs SerializedPlanParser::parseArrayJoinWithDAG( const auto & arg_not_null = args[0]; auto array_join_name = arg_not_null->result_name; /// arrayJoin(arg_not_null) - const auto * array_join_node = &actions_dag->addArrayJoin(*arg_not_null, array_join_name); + const auto * array_join_node = &actions_dag.addArrayJoin(*arg_not_null, array_join_name); auto tuple_element_builder = FunctionFactory::instance().get("sparkTupleElement", context); auto tuple_index_type = std::make_shared(); auto add_tuple_element = [&](const ActionsDAG::Node * tuple_node, size_t i) -> const ActionsDAG::Node * { ColumnWithTypeAndName index_col(tuple_index_type->createColumnConst(1, i), tuple_index_type, getUniqueName(std::to_string(i))); - const auto * index_node = &actions_dag->addColumn(std::move(index_col)); + const auto * index_node = &actions_dag.addColumn(std::move(index_col)); auto result_name = "sparkTupleElement(" + tuple_node->result_name + ", " + index_node->result_name + ")"; - return &actions_dag->addFunction(tuple_element_builder, {tuple_node, index_node}, result_name); + return &actions_dag.addFunction(tuple_element_builder, {tuple_node, index_node}, result_name); }; /// Special process to keep compatiable with Spark @@ -666,8 +666,8 @@ ActionsDAG::NodeRawConstPtrs SerializedPlanParser::parseArrayJoinWithDAG( result_names.push_back(val_node->result_name); if (keep_result) { - actions_dag->addOrReplaceInOutputs(*key_node); - actions_dag->addOrReplaceInOutputs(*val_node); + actions_dag.addOrReplaceInOutputs(*key_node); + actions_dag.addOrReplaceInOutputs(*val_node); } return {key_node, val_node}; } @@ -675,7 +675,7 @@ ActionsDAG::NodeRawConstPtrs SerializedPlanParser::parseArrayJoinWithDAG( { result_names.push_back(array_join_name); if (keep_result) - actions_dag->addOrReplaceInOutputs(*array_join_node); + actions_dag.addOrReplaceInOutputs(*array_join_node); return {array_join_node}; } } @@ -708,9 +708,9 @@ ActionsDAG::NodeRawConstPtrs SerializedPlanParser::parseArrayJoinWithDAG( result_names.push_back(value_node->result_name); if (keep_result) { - actions_dag->addOrReplaceInOutputs(*pos_node); - actions_dag->addOrReplaceInOutputs(*key_node); - actions_dag->addOrReplaceInOutputs(*value_node); + actions_dag.addOrReplaceInOutputs(*pos_node); + actions_dag.addOrReplaceInOutputs(*key_node); + actions_dag.addOrReplaceInOutputs(*value_node); } return {pos_node, key_node, value_node}; @@ -722,8 +722,8 @@ ActionsDAG::NodeRawConstPtrs SerializedPlanParser::parseArrayJoinWithDAG( result_names.push_back(item_node->result_name); if (keep_result) { - actions_dag->addOrReplaceInOutputs(*pos_node); - actions_dag->addOrReplaceInOutputs(*item_node); + actions_dag.addOrReplaceInOutputs(*pos_node); + actions_dag.addOrReplaceInOutputs(*item_node); } return {pos_node, item_node}; } @@ -731,7 +731,7 @@ ActionsDAG::NodeRawConstPtrs SerializedPlanParser::parseArrayJoinWithDAG( } const ActionsDAG::Node * SerializedPlanParser::parseFunctionWithDAG( - const substrait::Expression & rel, std::string & result_name, ActionsDAGPtr actions_dag, bool keep_result) + const substrait::Expression & rel, std::string & result_name, ActionsDAG& actions_dag, bool keep_result) { if (!rel.has_scalar_function()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "the root of expression should be a scalar function:\n {}", rel.DebugString()); @@ -749,14 +749,14 @@ const ActionsDAG::Node * SerializedPlanParser::parseFunctionWithDAG( LOG_DEBUG(&Poco::Logger::get("SerializedPlanParser"), "parse function {} by function parser: {}", func_name, func_parser->getName()); const auto * result_node = func_parser->parse(scalar_function, actions_dag); if (keep_result) - actions_dag->addOrReplaceInOutputs(*result_node); + actions_dag.addOrReplaceInOutputs(*result_node); result_name = result_node->result_name; return result_node; } void SerializedPlanParser::parseFunctionArguments( - ActionsDAGPtr & actions_dag, ActionsDAG::NodeRawConstPtrs & parsed_args, const substrait::Expression_ScalarFunction & scalar_function) + ActionsDAG & actions_dag, ActionsDAG::NodeRawConstPtrs & parsed_args, const substrait::Expression_ScalarFunction & scalar_function) { auto function_signature = function_mapping.at(std::to_string(scalar_function.function_reference())); const auto & args = scalar_function.arguments(); @@ -792,21 +792,9 @@ bool SerializedPlanParser::isFunction(substrait::Expression_ScalarFunction rel, return func_signature.starts_with(function_name + ":"); } -ActionsDAGPtr SerializedPlanParser::parseFunction( - const Block & header, const substrait::Expression & rel, std::string & result_name, ActionsDAGPtr actions_dag, bool keep_result) +void SerializedPlanParser::parseFunctionOrExpression( + const substrait::Expression & rel, std::string & result_name, ActionsDAG& actions_dag, bool keep_result) { - if (!actions_dag) - actions_dag = std::make_shared(blockToNameAndTypeList(header)); - - parseFunctionWithDAG(rel, result_name, actions_dag, keep_result); - return actions_dag; -} - -ActionsDAGPtr SerializedPlanParser::parseFunctionOrExpression( - const Block & header, const substrait::Expression & rel, std::string & result_name, ActionsDAGPtr actions_dag, bool keep_result) -{ - if (!actions_dag) - actions_dag = std::make_shared(blockToNameAndTypeList(header)); if (rel.has_scalar_function()) parseFunctionWithDAG(rel, result_name, actions_dag, keep_result); @@ -815,38 +803,15 @@ ActionsDAGPtr SerializedPlanParser::parseFunctionOrExpression( const auto * result_node = parseExpression(actions_dag, rel); result_name = result_node->result_name; } - - return actions_dag; } -ActionsDAGPtr SerializedPlanParser::parseArrayJoin( - const Block & input, +void SerializedPlanParser::parseJsonTuple( const substrait::Expression & rel, std::vector & result_names, - ActionsDAGPtr actions_dag, - bool keep_result, - bool position) -{ - if (!actions_dag) - actions_dag = std::make_shared(blockToNameAndTypeList(input)); - - parseArrayJoinWithDAG(rel, result_names, actions_dag, keep_result, position); - return actions_dag; -} - -ActionsDAGPtr SerializedPlanParser::parseJsonTuple( - const Block & input, - const substrait::Expression & rel, - std::vector & result_names, - ActionsDAGPtr actions_dag, + ActionsDAG& actions_dag, bool keep_result, bool) { - if (!actions_dag) - { - actions_dag = std::make_shared(blockToNameAndTypeList(input)); - } - const auto & scalar_function = rel.scalar_function(); auto function_signature = function_mapping.at(std::to_string(rel.scalar_function().function_reference())); String function_name = "json_tuple"; @@ -882,35 +847,34 @@ ActionsDAGPtr SerializedPlanParser::parseJsonTuple( auto json_extract_builder = FunctionFactory::instance().get("JSONExtract", context); auto json_extract_result_name = "JSONExtract(" + json_expr_node->result_name + "," + extract_expr_node->result_name + ")"; const ActionsDAG::Node * json_extract_node - = &actions_dag->addFunction(json_extract_builder, {json_expr_node, extract_expr_node}, json_extract_result_name); + = &actions_dag.addFunction(json_extract_builder, {json_expr_node, extract_expr_node}, json_extract_result_name); auto tuple_element_builder = FunctionFactory::instance().get("sparkTupleElement", context); auto tuple_index_type = std::make_shared(); auto add_tuple_element = [&](const ActionsDAG::Node * tuple_node, size_t i) -> const ActionsDAG::Node * { ColumnWithTypeAndName index_col(tuple_index_type->createColumnConst(1, i), tuple_index_type, getUniqueName(std::to_string(i))); - const auto * index_node = &actions_dag->addColumn(std::move(index_col)); + const auto * index_node = &actions_dag.addColumn(std::move(index_col)); auto result_name = "sparkTupleElement(" + tuple_node->result_name + ", " + index_node->result_name + ")"; - return &actions_dag->addFunction(tuple_element_builder, {tuple_node, index_node}, result_name); + return &actions_dag.addFunction(tuple_element_builder, {tuple_node, index_node}, result_name); }; for (int i = 1; i < args.size(); i++) { const ActionsDAG::Node * tuple_node = add_tuple_element(json_extract_node, i); if (keep_result) { - actions_dag->addOrReplaceInOutputs(*tuple_node); + actions_dag.addOrReplaceInOutputs(*tuple_node); result_names.push_back(tuple_node->result_name); } } - return actions_dag; } const ActionsDAG::Node * -SerializedPlanParser::toFunctionNode(ActionsDAGPtr actions_dag, const String & function, const ActionsDAG::NodeRawConstPtrs & args) +SerializedPlanParser::toFunctionNode(ActionsDAG& actions_dag, const String & function, const ActionsDAG::NodeRawConstPtrs & args) { auto function_builder = FunctionFactory::instance().get(function, context); std::string args_name = join(args, ','); auto result_name = function + "(" + args_name + ")"; - const auto * function_node = &actions_dag->addFunction(function_builder, args, result_name); + const auto * function_node = &actions_dag.addFunction(function_builder, args, result_name); return function_node; } @@ -1117,7 +1081,7 @@ std::pair SerializedPlanParser::parseLiteral(const substrait return std::make_pair(std::move(type), std::move(field)); } -const ActionsDAG::Node * SerializedPlanParser::parseExpression(ActionsDAGPtr actions_dag, const substrait::Expression & rel) +const ActionsDAG::Node * SerializedPlanParser::parseExpression(ActionsDAG& actions_dag, const substrait::Expression & rel) { switch (rel.rex_type_case()) { @@ -1132,8 +1096,8 @@ const ActionsDAG::Node * SerializedPlanParser::parseExpression(ActionsDAGPtr act if (!rel.selection().has_direct_reference() || !rel.selection().direct_reference().has_struct_field()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Can only have direct struct references in selections"); - const auto * field = actions_dag->getInputs()[rel.selection().direct_reference().struct_field().field()]; - return actions_dag->tryFindInOutputs(field->result_name); + const auto * field = actions_dag.getInputs()[rel.selection().direct_reference().struct_field().field()]; + return actions_dag.tryFindInOutputs(field->result_name); } case substrait::Expression::RexTypeCase::kCast: { @@ -1186,7 +1150,7 @@ const ActionsDAG::Node * SerializedPlanParser::parseExpression(ActionsDAGPtr act function_node = toFunctionNode(actions_dag, "CAST", args); } - actions_dag->addOrReplaceInOutputs(*function_node); + actions_dag.addOrReplaceInOutputs(*function_node); return function_node; } @@ -1218,8 +1182,8 @@ const ActionsDAG::Node * SerializedPlanParser::parseExpression(ActionsDAGPtr act result_name = "if(" + args_name + ")"; else result_name = "multiIf(" + args_name + ")"; - const auto * function_node = &actions_dag->addFunction(function_ptr, args, result_name); - actions_dag->addOrReplaceInOutputs(*function_node); + const auto * function_node = &actions_dag.addFunction(function_ptr, args, result_name); + actions_dag.addOrReplaceInOutputs(*function_node); return function_node; } @@ -1280,10 +1244,10 @@ const ActionsDAG::Node * SerializedPlanParser::parseExpression(ActionsDAGPtr act auto future_set = std::make_shared(elem_block, context->getSettingsRef()); auto arg = ColumnSet::create(1, std::move(future_set)); - args.emplace_back(&actions_dag->addColumn(ColumnWithTypeAndName(std::move(arg), std::make_shared(), name))); + args.emplace_back(&actions_dag.addColumn(ColumnWithTypeAndName(std::move(arg), std::make_shared(), name))); const auto * function_node = toFunctionNode(actions_dag, "in", args); - actions_dag->addOrReplaceInOutputs(*function_node); + actions_dag.addOrReplaceInOutputs(*function_node); if (nullable) { /// if sets has `null` and value not in sets @@ -1295,7 +1259,7 @@ const ActionsDAG::Node * SerializedPlanParser::parseExpression(ActionsDAGPtr act {function_node, addColumn(actions_dag, type, true), addColumn(actions_dag, type, Field())}); auto cast = FunctionFactory::instance().get("if", context); function_node = toFunctionNode(actions_dag, "if", cast_args); - actions_dag->addOrReplaceInOutputs(*function_node); + actions_dag.addOrReplaceInOutputs(*function_node); } return function_node; } @@ -1581,29 +1545,29 @@ ASTPtr ASTParser::parseArgumentToAST(const Names & names, const substrait::Expre } void SerializedPlanParser::removeNullableForRequiredColumns( - const std::set & require_columns, const ActionsDAGPtr & actions_dag) const + const std::set & require_columns, ActionsDAG & actions_dag) const { for (const auto & item : require_columns) { - if (const auto * require_node = actions_dag->tryFindInOutputs(item)) + if (const auto * require_node = actions_dag.tryFindInOutputs(item)) { auto function_builder = FunctionFactory::instance().get("assumeNotNull", context); ActionsDAG::NodeRawConstPtrs args = {require_node}; - const auto & node = actions_dag->addFunction(function_builder, args, item); - actions_dag->addOrReplaceInOutputs(node); + const auto & node = actions_dag.addFunction(function_builder, args, item); + actions_dag.addOrReplaceInOutputs(node); } } } void SerializedPlanParser::wrapNullable( - const std::vector & columns, ActionsDAGPtr actions_dag, std::map & nullable_measure_names) + const std::vector & columns, ActionsDAG& actions_dag, std::map & nullable_measure_names) { for (const auto & item : columns) { ActionsDAG::NodeRawConstPtrs args; - args.emplace_back(&actions_dag->findInOutputs(item)); + args.emplace_back(&actions_dag.findInOutputs(item)); const auto * node = toFunctionNode(actions_dag, "toNullable", args); - actions_dag->addOrReplaceInOutputs(*node); + actions_dag.addOrReplaceInOutputs(*node); nullable_measure_names[item] = node->result_name; } } diff --git a/cpp-ch/local-engine/Parser/SerializedPlanParser.h b/cpp-ch/local-engine/Parser/SerializedPlanParser.h index a7d77fde84c1b..e44a7f657a205 100644 --- a/cpp-ch/local-engine/Parser/SerializedPlanParser.h +++ b/cpp-ch/local-engine/Parser/SerializedPlanParser.h @@ -118,7 +118,7 @@ class SerializedPlanParser } void parseExtensions(const ::google::protobuf::RepeatedPtrField & extensions); - std::shared_ptr expressionsToActionsDAG( + DB::ActionsDAG expressionsToActionsDAG( const std::vector & expressions, const DB::Block & header, const DB::Block & read_schema); RelMetricPtr getMetric() { return metrics.empty() ? nullptr : metrics.at(0); } const std::unordered_map & getFunctionMapping() { return function_mapping; } @@ -141,47 +141,32 @@ class SerializedPlanParser void collectJoinKeys(const substrait::Expression & condition, std::vector> & join_keys, int32_t right_key_start); - DB::ActionsDAGPtr parseFunction( - const Block & header, + void parseFunctionOrExpression( const substrait::Expression & rel, std::string & result_name, - DB::ActionsDAGPtr actions_dag = nullptr, + DB::ActionsDAG& actions_dag, bool keep_result = false); - DB::ActionsDAGPtr parseFunctionOrExpression( - const Block & header, - const substrait::Expression & rel, - std::string & result_name, - DB::ActionsDAGPtr actions_dag = nullptr, - bool keep_result = false); - DB::ActionsDAGPtr parseArrayJoin( - const Block & input, - const substrait::Expression & rel, - std::vector & result_names, - DB::ActionsDAGPtr actions_dag = nullptr, - bool keep_result = false, - bool position = false); - DB::ActionsDAGPtr parseJsonTuple( - const Block & input, + void parseJsonTuple( const substrait::Expression & rel, std::vector & result_names, - DB::ActionsDAGPtr actions_dag = nullptr, + DB::ActionsDAG& actions_dag, bool keep_result = false, bool position = false); const ActionsDAG::Node * parseFunctionWithDAG( - const substrait::Expression & rel, std::string & result_name, DB::ActionsDAGPtr actions_dag = nullptr, bool keep_result = false); + const substrait::Expression & rel, std::string & result_name, DB::ActionsDAG& actions_dag, bool keep_result = false); ActionsDAG::NodeRawConstPtrs parseArrayJoinWithDAG( const substrait::Expression & rel, std::vector & result_name, - DB::ActionsDAGPtr actions_dag = nullptr, + DB::ActionsDAG& actions_dag, bool keep_result = false, bool position = false); void parseFunctionArguments( - DB::ActionsDAGPtr & actions_dag, + DB::ActionsDAG & actions_dag, ActionsDAG::NodeRawConstPtrs & parsed_args, const substrait::Expression_ScalarFunction & scalar_function); void parseArrayJoinArguments( - DB::ActionsDAGPtr & actions_dag, + DB::ActionsDAG & actions_dag, const std::string & function_name, const substrait::Expression_ScalarFunction & scalar_function, bool position, @@ -189,14 +174,14 @@ class SerializedPlanParser bool & is_map); - const DB::ActionsDAG::Node * parseExpression(DB::ActionsDAGPtr actions_dag, const substrait::Expression & rel); + const DB::ActionsDAG::Node * parseExpression(DB::ActionsDAG& actions_dag, const substrait::Expression & rel); const ActionsDAG::Node * - toFunctionNode(ActionsDAGPtr actions_dag, const String & function, const DB::ActionsDAG::NodeRawConstPtrs & args); + toFunctionNode(ActionsDAG& actions_dag, const String & function, const DB::ActionsDAG::NodeRawConstPtrs & args); // remove nullable after isNotNull - void removeNullableForRequiredColumns(const std::set & require_columns, const ActionsDAGPtr & actions_dag) const; + void removeNullableForRequiredColumns(const std::set & require_columns, ActionsDAG & actions_dag) const; std::string getUniqueName(const std::string & name) { return name + "_" + std::to_string(name_no++); } void wrapNullable( - const std::vector & columns, ActionsDAGPtr actions_dag, std::map & nullable_measure_names); + const std::vector & columns, ActionsDAG& actions_dag, std::map & nullable_measure_names); static std::pair convertStructFieldType(const DB::DataTypePtr & type, const DB::Field & field); bool isFunction(substrait::Expression_ScalarFunction rel, String function_name); @@ -213,7 +198,7 @@ class SerializedPlanParser std::vector metrics; public: - const ActionsDAG::Node * addColumn(DB::ActionsDAGPtr actions_dag, const DataTypePtr & type, const Field & field); + const ActionsDAG::Node * addColumn(DB::ActionsDAG& actions_dag, const DataTypePtr & type, const Field & field); }; struct SparkBuffer @@ -237,7 +222,7 @@ class LocalExecutor : public BlockIterator Block & getHeader(); RelMetricPtr getMetric() const { return metric; } - void setMetric(RelMetricPtr metric_) { metric = metric_; } + void setMetric(const RelMetricPtr & metric_) { metric = metric_; } void setExtraPlanHolder(std::vector & extra_plan_holder_) { extra_plan_holder = std::move(extra_plan_holder_); } private: diff --git a/cpp-ch/local-engine/Parser/WindowRelParser.cpp b/cpp-ch/local-engine/Parser/WindowRelParser.cpp index 2317c8098b858..0676924c1f571 100644 --- a/cpp-ch/local-engine/Parser/WindowRelParser.cpp +++ b/cpp-ch/local-engine/Parser/WindowRelParser.cpp @@ -323,8 +323,8 @@ void WindowRelParser::initWindowsInfos(const substrait::WindowRel & win_rel) void WindowRelParser::tryAddProjectionBeforeWindow() { auto header = current_plan->getCurrentDataStream().header; - ActionsDAGPtr actions_dag = std::make_shared(header.getColumnsWithTypeAndName()); - auto dag_footprint = actions_dag->dumpDAG(); + ActionsDAG actions_dag{header.getColumnsWithTypeAndName()}; + auto dag_footprint = actions_dag.dumpDAG(); for (auto & win_info : win_infos ) { @@ -335,13 +335,13 @@ void WindowRelParser::tryAddProjectionBeforeWindow() { win_info.arg_column_names.emplace_back(arg_node->result_name); win_info.arg_column_types.emplace_back(arg_node->result_type); - actions_dag->addOrReplaceInOutputs(*arg_node); + actions_dag.addOrReplaceInOutputs(*arg_node); } } - if (actions_dag->dumpDAG() != dag_footprint) + if (actions_dag.dumpDAG() != dag_footprint) { - auto project_step = std::make_unique(current_plan->getCurrentDataStream(), actions_dag); + auto project_step = std::make_unique(current_plan->getCurrentDataStream(), std::move(actions_dag)); project_step->setStepDescription("Add projections before window"); steps.emplace_back(project_step.get()); current_plan->addStep(std::move(project_step)); @@ -352,19 +352,19 @@ void WindowRelParser::tryAddProjectionAfterWindow() { // The final result header is : original header ++ [window aggregate columns] auto header = current_plan->getCurrentDataStream().header; - ActionsDAGPtr actions_dag = std::make_shared(header.getColumnsWithTypeAndName()); - auto dag_footprint = actions_dag->dumpDAG(); + ActionsDAG actions_dag{header.getColumnsWithTypeAndName()}; + auto dag_footprint = actions_dag.dumpDAG(); for (size_t i = 0; i < win_infos.size(); ++i) { auto & win_info = win_infos[i]; - const auto * win_result_node = &actions_dag->findInOutputs(win_info.result_column_name); + const auto * win_result_node = &actions_dag.findInOutputs(win_info.result_column_name); win_info.function_parser->convertNodeTypeIfNeeded(win_info.parser_func_info, win_result_node, actions_dag, false); } - if (actions_dag->dumpDAG() != dag_footprint) + if (actions_dag.dumpDAG() != dag_footprint) { - auto project_step = std::make_unique(current_plan->getCurrentDataStream(), actions_dag); + auto project_step = std::make_unique(current_plan->getCurrentDataStream(), std::move(actions_dag)); project_step->setStepDescription("Add projections for window result"); steps.emplace_back(project_step.get()); current_plan->addStep(std::move(project_step)); @@ -374,11 +374,11 @@ void WindowRelParser::tryAddProjectionAfterWindow() auto current_header = current_plan->getCurrentDataStream().header; if (!DB::blocksHaveEqualStructure(output_header, current_header)) { - ActionsDAGPtr convert_action = ActionsDAG::makeConvertingActions( + ActionsDAG convert_action = ActionsDAG::makeConvertingActions( current_header.getColumnsWithTypeAndName(), output_header.getColumnsWithTypeAndName(), DB::ActionsDAG::MatchColumnsMode::Name); - QueryPlanStepPtr convert_step = std::make_unique(current_plan->getCurrentDataStream(), convert_action); + QueryPlanStepPtr convert_step = std::make_unique(current_plan->getCurrentDataStream(), std::move(convert_action)); convert_step->setStepDescription("Convert window Output"); steps.emplace_back(convert_step.get()); current_plan->addStep(std::move(convert_step)); diff --git a/cpp-ch/local-engine/Parser/WriteRelParser.cpp b/cpp-ch/local-engine/Parser/WriteRelParser.cpp index b32b7bc6337b4..9b6226adbed81 100644 --- a/cpp-ch/local-engine/Parser/WriteRelParser.cpp +++ b/cpp-ch/local-engine/Parser/WriteRelParser.cpp @@ -66,9 +66,9 @@ DB::ExpressionActionsPtr create_rename_action(const DB::Block & input, const DB: for (auto ouput_name = output.begin(), input_iter = input.begin(); ouput_name != output.end(); ++ouput_name, ++input_iter) aliases.emplace_back(DB::NameWithAlias(input_iter->name, ouput_name->name)); - const auto actions_dag = std::make_shared(blockToNameAndTypeList(input)); - actions_dag->project(aliases); - return std::make_shared(actions_dag); + ActionsDAG actions_dag{blockToNameAndTypeList(input)}; + actions_dag.project(aliases); + return std::make_shared(std::move(actions_dag)); } DB::ExpressionActionsPtr create_project_action(const DB::Block & input, const DB::Block & output) @@ -82,8 +82,8 @@ DB::ExpressionActionsPtr create_project_action(const DB::Block & input, const DB assert(final_cols.size() == output.columns()); const auto & original_cols = input.getColumnsWithTypeAndName(); - ActionsDAGPtr final_project = ActionsDAG::makeConvertingActions(original_cols, final_cols, ActionsDAG::MatchColumnsMode::Position); - return std::make_shared(final_project); + ActionsDAG final_project = ActionsDAG::makeConvertingActions(original_cols, final_cols, ActionsDAG::MatchColumnsMode::Position); + return std::make_shared(std::move(final_project)); } void adjust_output(const DB::QueryPipelineBuilderPtr & builder, const DB::Block& output) diff --git a/cpp-ch/local-engine/Parser/aggregate_function_parser/CollectListParser.h b/cpp-ch/local-engine/Parser/aggregate_function_parser/CollectListParser.h index 60e1b4eaedd30..fe2b5fba3dc4c 100644 --- a/cpp-ch/local-engine/Parser/aggregate_function_parser/CollectListParser.h +++ b/cpp-ch/local-engine/Parser/aggregate_function_parser/CollectListParser.h @@ -52,7 +52,7 @@ class CollectFunctionParser : public AggregateFunctionParser throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Not implement"); } const DB::ActionsDAG::Node * convertNodeTypeIfNeeded( - const CommonFunctionInfo &, const DB::ActionsDAG::Node * func_node, DB::ActionsDAGPtr & actions_dag, bool /* with_nullability */) const override + const CommonFunctionInfo &, const DB::ActionsDAG::Node * func_node, DB::ActionsDAG & actions_dag, bool /* with_nullability */) const override { const DB::ActionsDAG::Node * ret_node = func_node; if (func_node->result_type->isNullable()) @@ -60,11 +60,11 @@ class CollectFunctionParser : public AggregateFunctionParser DB::ActionsDAG::NodeRawConstPtrs args = {func_node}; auto nested_type = typeid_cast(func_node->result_type.get())->getNestedType(); DB::Field empty_field = nested_type->getDefault(); - const auto * default_value_node = &actions_dag->addColumn( + const auto * default_value_node = &actions_dag.addColumn( ColumnWithTypeAndName(nested_type->createColumnConst(1, empty_field), nested_type, getUniqueName("[]"))); args.push_back(default_value_node); const auto * if_null_node = toFunctionNode(actions_dag, "ifNull", func_node->result_name, args); - actions_dag->addOrReplaceInOutputs(*if_null_node); + actions_dag.addOrReplaceInOutputs(*if_null_node); ret_node = if_null_node; } return ret_node; diff --git a/cpp-ch/local-engine/Parser/aggregate_function_parser/CountParser.cpp b/cpp-ch/local-engine/Parser/aggregate_function_parser/CountParser.cpp index 123d13c365875..fb768c09a5ee0 100644 --- a/cpp-ch/local-engine/Parser/aggregate_function_parser/CountParser.cpp +++ b/cpp-ch/local-engine/Parser/aggregate_function_parser/CountParser.cpp @@ -43,7 +43,7 @@ String CountParser::getCHFunctionName(DB::DataTypes &) const } DB::ActionsDAG::NodeRawConstPtrs CountParser::parseFunctionArguments( - const CommonFunctionInfo & func_info, DB::ActionsDAGPtr & actions_dag) const + const CommonFunctionInfo & func_info, DB::ActionsDAG & actions_dag) const { if (func_info.arguments.size() < 1) { @@ -63,9 +63,9 @@ DB::ActionsDAG::NodeRawConstPtrs CountParser::parseFunctionArguments( auto nullable_uint_col = nullable_uint8_type->createColumn(); nullable_uint_col->insertDefault(); const auto * const_1_node - = &actions_dag->addColumn(DB::ColumnWithTypeAndName(uint8_type->createColumnConst(1, 1), uint8_type, getUniqueName("1"))); + = &actions_dag.addColumn(DB::ColumnWithTypeAndName(uint8_type->createColumnConst(1, 1), uint8_type, getUniqueName("1"))); const auto * null_node - = &actions_dag->addColumn(DB::ColumnWithTypeAndName(std::move(nullable_uint_col), nullable_uint8_type, getUniqueName("null"))); + = &actions_dag.addColumn(DB::ColumnWithTypeAndName(std::move(nullable_uint_col), nullable_uint8_type, getUniqueName("null"))); DB::ActionsDAG::NodeRawConstPtrs multi_if_args; for (const auto & arg : func_info.arguments) diff --git a/cpp-ch/local-engine/Parser/aggregate_function_parser/CountParser.h b/cpp-ch/local-engine/Parser/aggregate_function_parser/CountParser.h index a83ec2d5a337c..a07fc16e2cf92 100644 --- a/cpp-ch/local-engine/Parser/aggregate_function_parser/CountParser.h +++ b/cpp-ch/local-engine/Parser/aggregate_function_parser/CountParser.h @@ -30,6 +30,6 @@ class CountParser : public AggregateFunctionParser String getCHFunctionName(const CommonFunctionInfo &) const override; String getCHFunctionName(DB::DataTypes &) const override; DB::ActionsDAG::NodeRawConstPtrs parseFunctionArguments( - const CommonFunctionInfo & func_info, DB::ActionsDAGPtr & actions_dag) const override; + const CommonFunctionInfo & func_info, DB::ActionsDAG & actions_dag) const override; }; } diff --git a/cpp-ch/local-engine/Parser/aggregate_function_parser/LeadLagParser.cpp b/cpp-ch/local-engine/Parser/aggregate_function_parser/LeadLagParser.cpp index 6a56a82d5044d..6d0075705c444 100644 --- a/cpp-ch/local-engine/Parser/aggregate_function_parser/LeadLagParser.cpp +++ b/cpp-ch/local-engine/Parser/aggregate_function_parser/LeadLagParser.cpp @@ -24,7 +24,7 @@ namespace local_engine { DB::ActionsDAG::NodeRawConstPtrs -LeadParser::parseFunctionArguments(const CommonFunctionInfo & func_info, DB::ActionsDAGPtr & actions_dag) const +LeadParser::parseFunctionArguments(const CommonFunctionInfo & func_info, DB::ActionsDAG & actions_dag) const { DB::ActionsDAG::NodeRawConstPtrs args; const auto & arg0 = func_info.arguments[0].value(); @@ -32,7 +32,7 @@ LeadParser::parseFunctionArguments(const CommonFunctionInfo & func_info, DB::Act /// The 3rd arg is default value /// when it is set to null, the 1st arg must be nullable const auto & arg2 = func_info.arguments[2].value(); - const auto * arg0_col = actions_dag->getInputs()[arg0.selection().direct_reference().struct_field().field()]; + const auto * arg0_col = actions_dag.getInputs()[arg0.selection().direct_reference().struct_field().field()]; auto arg0_col_name = arg0_col->result_name; auto arg0_col_type= arg0_col->result_type; const DB::ActionsDAG::Node * node = nullptr; @@ -40,10 +40,10 @@ LeadParser::parseFunctionArguments(const CommonFunctionInfo & func_info, DB::Act { node = ActionsDAGUtil::convertNodeType( actions_dag, - &actions_dag->findInOutputs(arg0_col_name), + &actions_dag.findInOutputs(arg0_col_name), DB::makeNullable(arg0_col_type)->getName(), arg0_col_name); - actions_dag->addOrReplaceInOutputs(*node); + actions_dag.addOrReplaceInOutputs(*node); args.push_back(node); } else @@ -53,13 +53,13 @@ LeadParser::parseFunctionArguments(const CommonFunctionInfo & func_info, DB::Act node = parseExpression(actions_dag, arg1); node = ActionsDAGUtil::convertNodeType(actions_dag, node, DB::DataTypeInt64().getName()); - actions_dag->addOrReplaceInOutputs(*node); + actions_dag.addOrReplaceInOutputs(*node); args.push_back(node); if (arg2.has_literal() && !arg2.literal().has_null()) { node = parseExpression(actions_dag, arg2); - actions_dag->addOrReplaceInOutputs(*node); + actions_dag.addOrReplaceInOutputs(*node); args.push_back(node); } return args; @@ -67,7 +67,7 @@ LeadParser::parseFunctionArguments(const CommonFunctionInfo & func_info, DB::Act AggregateFunctionParserRegister lead_register; DB::ActionsDAG::NodeRawConstPtrs -LagParser::parseFunctionArguments(const CommonFunctionInfo & func_info, DB::ActionsDAGPtr & actions_dag) const +LagParser::parseFunctionArguments(const CommonFunctionInfo & func_info, DB::ActionsDAG & actions_dag) const { DB::ActionsDAG::NodeRawConstPtrs args; const auto & arg0 = func_info.arguments[0].value(); @@ -75,7 +75,7 @@ LagParser::parseFunctionArguments(const CommonFunctionInfo & func_info, DB::Acti /// The 3rd arg is default value /// when it is set to null, the 1st arg must be nullable const auto & arg2 = func_info.arguments[2].value(); - const auto * arg0_col = actions_dag->getInputs()[arg0.selection().direct_reference().struct_field().field()]; + const auto * arg0_col = actions_dag.getInputs()[arg0.selection().direct_reference().struct_field().field()]; auto arg0_col_name = arg0_col->result_name; auto arg0_col_type = arg0_col->result_type; const DB::ActionsDAG::Node * node = nullptr; @@ -83,10 +83,10 @@ LagParser::parseFunctionArguments(const CommonFunctionInfo & func_info, DB::Acti { node = ActionsDAGUtil::convertNodeType( actions_dag, - &actions_dag->findInOutputs(arg0_col_name), + &actions_dag.findInOutputs(arg0_col_name), DB::makeNullable(arg0_col_type)->getName(), arg0_col_name); - actions_dag->addOrReplaceInOutputs(*node); + actions_dag.addOrReplaceInOutputs(*node); args.push_back(node); } else @@ -98,16 +98,16 @@ LagParser::parseFunctionArguments(const CommonFunctionInfo & func_info, DB::Acti auto literal_result = parseLiteral(arg1.literal()); assert(literal_result.second.safeGet() < 0); auto real_field = 0 - literal_result.second.safeGet(); - node = &actions_dag->addColumn(ColumnWithTypeAndName( + node = &actions_dag.addColumn(ColumnWithTypeAndName( literal_result.first->createColumnConst(1, real_field), literal_result.first, getUniqueName(toString(real_field)))); node = ActionsDAGUtil::convertNodeType(actions_dag, node, DB::DataTypeInt64().getName()); - actions_dag->addOrReplaceInOutputs(*node); + actions_dag.addOrReplaceInOutputs(*node); args.push_back(node); if (arg2.has_literal() && !arg2.literal().has_null()) { node = parseExpression(actions_dag, arg2); - actions_dag->addOrReplaceInOutputs(*node); + actions_dag.addOrReplaceInOutputs(*node); args.push_back(node); } return args; diff --git a/cpp-ch/local-engine/Parser/aggregate_function_parser/LeadLagParser.h b/cpp-ch/local-engine/Parser/aggregate_function_parser/LeadLagParser.h index 25f679c77b401..14c50ef40d9d3 100644 --- a/cpp-ch/local-engine/Parser/aggregate_function_parser/LeadLagParser.h +++ b/cpp-ch/local-engine/Parser/aggregate_function_parser/LeadLagParser.h @@ -29,7 +29,7 @@ class LeadParser : public AggregateFunctionParser String getCHFunctionName(const CommonFunctionInfo &) const override { return "leadInFrame"; } String getCHFunctionName(DB::DataTypes &) const override { return "leadInFrame"; } DB::ActionsDAG::NodeRawConstPtrs parseFunctionArguments( - const CommonFunctionInfo & func_info, DB::ActionsDAGPtr & actions_dag) const override; + const CommonFunctionInfo & func_info, DB::ActionsDAG & actions_dag) const override; }; class LagParser : public AggregateFunctionParser @@ -42,6 +42,6 @@ class LagParser : public AggregateFunctionParser String getCHFunctionName(const CommonFunctionInfo &) const override { return "lagInFrame"; } String getCHFunctionName(DB::DataTypes &) const override { return "lagInFrame"; } DB::ActionsDAG::NodeRawConstPtrs parseFunctionArguments( - const CommonFunctionInfo & func_info, DB::ActionsDAGPtr & actions_dag) const override; + const CommonFunctionInfo & func_info, DB::ActionsDAG & actions_dag) const override; }; } diff --git a/cpp-ch/local-engine/Parser/aggregate_function_parser/NtileParser.cpp b/cpp-ch/local-engine/Parser/aggregate_function_parser/NtileParser.cpp index 19d7930fc1fc0..62f83223c06f7 100644 --- a/cpp-ch/local-engine/Parser/aggregate_function_parser/NtileParser.cpp +++ b/cpp-ch/local-engine/Parser/aggregate_function_parser/NtileParser.cpp @@ -22,7 +22,7 @@ namespace local_engine { DB::ActionsDAG::NodeRawConstPtrs -NtileParser::parseFunctionArguments(const CommonFunctionInfo & func_info, DB::ActionsDAGPtr & actions_dag) const +NtileParser::parseFunctionArguments(const CommonFunctionInfo & func_info, DB::ActionsDAG & actions_dag) const { if (func_info.arguments.size() != 1) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Function ntile takes exactly one argument"); diff --git a/cpp-ch/local-engine/Parser/aggregate_function_parser/NtileParser.h b/cpp-ch/local-engine/Parser/aggregate_function_parser/NtileParser.h index 28878a9f89db8..39b92ed85179a 100644 --- a/cpp-ch/local-engine/Parser/aggregate_function_parser/NtileParser.h +++ b/cpp-ch/local-engine/Parser/aggregate_function_parser/NtileParser.h @@ -29,6 +29,6 @@ class NtileParser : public AggregateFunctionParser String getCHFunctionName(const CommonFunctionInfo &) const override { return "ntile"; } String getCHFunctionName(DB::DataTypes &) const override { return "ntile"; } DB::ActionsDAG::NodeRawConstPtrs parseFunctionArguments( - const CommonFunctionInfo & func_info, DB::ActionsDAGPtr & actions_dag) const override; + const CommonFunctionInfo & func_info, DB::ActionsDAG & actions_dag) const override; }; } diff --git a/cpp-ch/local-engine/Parser/aggregate_function_parser/SimpleStatisticsFunctions.cpp b/cpp-ch/local-engine/Parser/aggregate_function_parser/SimpleStatisticsFunctions.cpp index 7e75e20bb7422..062071d22d1c5 100644 --- a/cpp-ch/local-engine/Parser/aggregate_function_parser/SimpleStatisticsFunctions.cpp +++ b/cpp-ch/local-engine/Parser/aggregate_function_parser/SimpleStatisticsFunctions.cpp @@ -46,7 +46,7 @@ class AggregateFunctionParserStddev final : public AggregateFunctionParser const DB::ActionsDAG::Node * convertNodeTypeIfNeeded( const CommonFunctionInfo & func_info, const DB::ActionsDAG::Node * func_node, - DB::ActionsDAGPtr & actions_dag, + DB::ActionsDAG & actions_dag, bool with_nullability) const override { /// result is nullable. @@ -56,11 +56,11 @@ class AggregateFunctionParserStddev final : public AggregateFunctionParser auto nullable_col = null_type->createColumn(); nullable_col->insertDefault(); const auto * null_node - = &actions_dag->addColumn(DB::ColumnWithTypeAndName(std::move(nullable_col), null_type, getUniqueName("null"))); + = &actions_dag.addColumn(DB::ColumnWithTypeAndName(std::move(nullable_col), null_type, getUniqueName("null"))); DB::ActionsDAG::NodeRawConstPtrs convert_nan_func_args = {is_nan_func_node, null_node, func_node}; func_node = toFunctionNode(actions_dag, "if", func_node->result_name, convert_nan_func_args); - actions_dag->addOrReplaceInOutputs(*func_node); + actions_dag.addOrReplaceInOutputs(*func_node); return func_node; } }; diff --git a/cpp-ch/local-engine/Parser/example_udf/myMd5.cpp b/cpp-ch/local-engine/Parser/example_udf/myMd5.cpp index 1e70c775e130c..1fa8fa8bfb56c 100644 --- a/cpp-ch/local-engine/Parser/example_udf/myMd5.cpp +++ b/cpp-ch/local-engine/Parser/example_udf/myMd5.cpp @@ -39,7 +39,7 @@ class FunctionParserMyMd5 : public FunctionParser const ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - ActionsDAGPtr & actions_dag) const override + ActionsDAG & actions_dag) const override { // In Spark: md5(str) // In CH: lower(hex(MD5(str))) diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/alias.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/alias.cpp index e5493eb80b2a2..57c952053b2b4 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/alias.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/alias.cpp @@ -28,7 +28,7 @@ class SparkFunctionAliasParser : public FunctionParser String getName() const { return name; } String getCHFunctionName(const substrait::Expression_ScalarFunction &) const override { return name; } - const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAGPtr & actions_dag) const override + const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { DB::ActionsDAG::NodeRawConstPtrs parsed_args; const auto & args = substrait_func.arguments(); @@ -43,8 +43,8 @@ class SparkFunctionAliasParser : public FunctionParser parsed_args.emplace_back(parseExpression(actions_dag, arg.value())); } String result_name = parsed_args[0]->result_name; - actions_dag->addOrReplaceInOutputs(*parsed_args[0]); - return &actions_dag->addAlias(actions_dag->findInOutputs(result_name), result_name); + actions_dag.addOrReplaceInOutputs(*parsed_args[0]); + return &actions_dag.addAlias(actions_dag.findInOutputs(result_name), result_name); } }; diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/arithmetic.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/arithmetic.cpp index b621798c3b308..6aba310bf095d 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/arithmetic.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/arithmetic.cpp @@ -85,7 +85,7 @@ class FunctionParserBinaryArithmetic : public FunctionParser { protected: ActionsDAG::NodeRawConstPtrs convertBinaryArithmeticFunDecimalArgs( - ActionsDAGPtr & actions_dag, + ActionsDAG & actions_dag, const ActionsDAG::NodeRawConstPtrs & args, const DecimalType & eval_type, const substrait::Expression_ScalarFunction & arithmeticFun) const @@ -104,10 +104,10 @@ class FunctionParserBinaryArithmetic : public FunctionParser const String type_name = ch_type->getName(); const DataTypePtr str_type = std::make_shared(); const ActionsDAG::Node * type_node - = &actions_dag->addColumn(ColumnWithTypeAndName(str_type->createColumnConst(1, type_name), str_type, getUniqueName(type_name))); + = &actions_dag.addColumn(ColumnWithTypeAndName(str_type->createColumnConst(1, type_name), str_type, getUniqueName(type_name))); cast_args.emplace_back(type_node); const ActionsDAG::Node * cast_node = toFunctionNode(actions_dag, "CAST", cast_args); - actions_dag->addOrReplaceInOutputs(*cast_node); + actions_dag.addOrReplaceInOutputs(*cast_node); new_args.emplace_back(cast_node); new_args.emplace_back(args[1]); return new_args; @@ -126,7 +126,7 @@ class FunctionParserBinaryArithmetic : public FunctionParser virtual DecimalType internalEvalType(Int32 p1, Int32 s1, Int32 p2, Int32 s2) const = 0; const ActionsDAG::Node * - checkDecimalOverflow(ActionsDAGPtr & actions_dag, const ActionsDAG::Node * func_node, Int32 precision, Int32 scale) const + checkDecimalOverflow(ActionsDAG & actions_dag, const ActionsDAG::Node * func_node, Int32 precision, Int32 scale) const { //TODO: checkDecimalOverflowSpark throw exception per configuration const DB::ActionsDAG::NodeRawConstPtrs overflow_args @@ -137,14 +137,14 @@ class FunctionParserBinaryArithmetic : public FunctionParser } virtual const DB::ActionsDAG::Node * - createFunctionNode(DB::ActionsDAGPtr & actions_dag, const String & func_name, const DB::ActionsDAG::NodeRawConstPtrs & args) const + createFunctionNode(DB::ActionsDAG & actions_dag, const String & func_name, const DB::ActionsDAG::NodeRawConstPtrs & args) const { return toFunctionNode(actions_dag, func_name, args); } public: explicit FunctionParserBinaryArithmetic(SerializedPlanParser * plan_parser_) : FunctionParser(plan_parser_) { } - const ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAGPtr & actions_dag) const override + const ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAG & actions_dag) const override { const auto ch_func_name = getCHFunctionName(substrait_func); auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); @@ -262,7 +262,7 @@ class FunctionParserDivide final : public FunctionParserBinaryArithmetic } const DB::ActionsDAG::Node * createFunctionNode( - DB::ActionsDAGPtr & actions_dag, const String & func_name, const DB::ActionsDAG::NodeRawConstPtrs & new_args) const override + DB::ActionsDAG & actions_dag, const String & func_name, const DB::ActionsDAG::NodeRawConstPtrs & new_args) const override { assert(func_name == name); const auto * left_arg = new_args[0]; diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayContains.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayContains.cpp index d92a1eac7da2e..c4bf7789034b4 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayContains.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayContains.cpp @@ -46,7 +46,7 @@ class FunctionParserArrayContains : public FunctionParser const ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - ActionsDAGPtr & actions_dag) const override + ActionsDAG & actions_dag) const override { /** parse array_contains(arr, value) as diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayDistinct.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayDistinct.cpp index 30709a7e9ed6e..c1625ffcebb9b 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayDistinct.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayDistinct.cpp @@ -42,7 +42,7 @@ class FunctionParserArrayDistinct : public FunctionParser const ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - ActionsDAGPtr & actions_dag) const override + ActionsDAG & actions_dag) const override { auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); if (parsed_args.size() != 1) diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayElement.h b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayElement.h index 5873d39cc22bf..5e398760504d1 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayElement.h +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayElement.h @@ -43,7 +43,7 @@ class FunctionParserArrayElement : public FunctionParser const ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - ActionsDAGPtr & actions_dag) const override + ActionsDAG & actions_dag) const override { /** parse arrayElement(arr, idx) as diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayHighOrderFunctions.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayHighOrderFunctions.cpp index f9f093cbad505..a475a1efb367d 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayHighOrderFunctions.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayHighOrderFunctions.cpp @@ -48,8 +48,8 @@ class ArrayFilter : public FunctionParser return "arrayFilter"; } - const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, - DB::ActionsDAGPtr & actions_dag) const + const DB::ActionsDAG::Node * + parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { auto ch_func_name = getCHFunctionName(substrait_func); auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); @@ -82,8 +82,8 @@ class ArrayTransform : public FunctionParser return "arrayMap"; } - const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, - DB::ActionsDAGPtr & actions_dag) const + const DB::ActionsDAG::Node * + parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { auto ch_func_name = getCHFunctionName(substrait_func); auto lambda_args = collectLambdaArguments(*plan_parser, substrait_func.arguments()[1].value().scalar_function()); @@ -127,8 +127,8 @@ class ArrayAggregate : public FunctionParser { return "arrayFold"; } - const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, - DB::ActionsDAGPtr & actions_dag) const + const DB::ActionsDAG::Node * + parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { auto ch_func_name = getCHFunctionName(substrait_func); auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); @@ -172,8 +172,8 @@ class ArraySort : public FunctionParser { return "arraySortSpark"; } - const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, - DB::ActionsDAGPtr & actions_dag) const + const DB::ActionsDAG::Node * + parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { auto ch_func_name = getCHFunctionName(substrait_func); auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayIntersect.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayIntersect.cpp index 2891846ef014a..d86a66357f33d 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayIntersect.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayIntersect.cpp @@ -45,7 +45,7 @@ class FunctionParserArrayIntersect : public FunctionParser const ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - ActionsDAGPtr & actions_dag) const override + ActionsDAG & actions_dag) const override { auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); if (parsed_args.size() != 2) diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayMaxAndMin.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayMaxAndMin.cpp index a0e6786442ee0..7624de578da32 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayMaxAndMin.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayMaxAndMin.cpp @@ -40,7 +40,7 @@ class BaseFunctionParserArrayMaxAndMin : public FunctionParser const ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - ActionsDAGPtr & actions_dag) const override + ActionsDAG & actions_dag) const override { auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); if (parsed_args.size() != 1) diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayPosition.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayPosition.cpp index 528a80c075a64..1fda3d8fa7536 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayPosition.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayPosition.cpp @@ -43,7 +43,7 @@ class FunctionParserArrayPosition : public FunctionParser const ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - ActionsDAGPtr & actions_dag) const override + ActionsDAG & actions_dag) const override { /** parse array_position(arr, value) as diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayUnion.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayUnion.cpp index 7a48d7920d2c0..95ab72d26cdd4 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayUnion.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayUnion.cpp @@ -42,7 +42,7 @@ class FunctionParserArrayUnion : public FunctionParser const ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - ActionsDAGPtr & actions_dag) const override + ActionsDAG & actions_dag) const override { /// parse array_union(a, b) as arrayDistinctSpark(arrayConcat(a, b)) auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/bitLength.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/bitLength.cpp index b2389d276f105..693c66fcf3ee5 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/bitLength.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/bitLength.cpp @@ -38,7 +38,7 @@ class FunctionParserBitLength : public FunctionParser String getName() const override { return name; } - const ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAGPtr & actions_dag) const override + const ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAG & actions_dag) const override { // parse bit_length(a) as octet_length(a) * 8 auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/checkOverflow.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/checkOverflow.cpp index e5228d1608700..9f90dc661551c 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/checkOverflow.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/checkOverflow.cpp @@ -47,7 +47,7 @@ class SparkFunctionCheckOverflow : public FunctionParser return ch_function_name; } - const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAGPtr & actions_dag) const override + const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { DB::ActionsDAG::NodeRawConstPtrs parsed_args; const auto & args = substrait_func.arguments(); diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/chr.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/chr.cpp index 7b755b1856374..4b09bcdf94d55 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/chr.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/chr.cpp @@ -40,7 +40,7 @@ class FunctionParserChr : public FunctionParser const ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - ActionsDAGPtr & actions_dag) const override + ActionsDAG & actions_dag) const override { auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); if (parsed_args.size() != 1) diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/concat.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/concat.cpp index cfafdfd98c377..d0e1264c4ffa2 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/concat.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/concat.cpp @@ -46,7 +46,7 @@ class FunctionParserConcat : public FunctionParser const ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - ActionsDAGPtr & actions_dag) const override + ActionsDAG & actions_dag) const override { /* parse concat(args) as: diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/concatWs.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/concatWs.cpp index e2993f1f2d66a..b811f087b248b 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/concatWs.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/concatWs.cpp @@ -46,7 +46,7 @@ class FunctionParserConcatWS : public FunctionParser const ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - ActionsDAGPtr & actions_dag) const override + ActionsDAG & actions_dag) const override { /* parse concat_ws(sep, s1, s2, arr1, arr2, ...)) as diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/cot.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/cot.cpp index 47750403049ce..a996d90758186 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/cot.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/cot.cpp @@ -41,7 +41,7 @@ class FunctionParserCot : public FunctionParser const ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - ActionsDAGPtr & actions_dag) const override + ActionsDAG & actions_dag) const override { /// parse cot(x) as 1 / tan(x) auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/csc.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/csc.cpp index 009c1b764f985..50f796632a7af 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/csc.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/csc.cpp @@ -41,7 +41,7 @@ class FunctionParserCsc : public FunctionParser const ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - ActionsDAGPtr & actions_dag) const override + ActionsDAG & actions_dag) const override { /// parse csc(x) as 1 / sin(x) auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/dateFormat.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/dateFormat.cpp index 980fdd4cfec06..a1f7a57951b67 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/dateFormat.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/dateFormat.cpp @@ -32,7 +32,7 @@ class SparkFunctionDateFormatParser : public FunctionParser return "formatDateTimeInJodaSyntax"; } - const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAGPtr & actions_dag) const override + const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { DB::ActionsDAG::NodeRawConstPtrs parsed_args; const auto & args = substrait_func.arguments(); diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/decode.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/decode.cpp index 48b86ed6b58bf..c155e14b706e7 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/decode.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/decode.cpp @@ -42,7 +42,7 @@ class FunctionParserDecode : public FunctionParser const ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - ActionsDAGPtr & actions_dag) const override + ActionsDAG & actions_dag) const override { /// Parse decode(bin, charset) as convertCharset(bin, charset, 'UTF-8') auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/elementAt.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/elementAt.cpp index ce18859174ade..b5587e79dc52a 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/elementAt.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/elementAt.cpp @@ -29,7 +29,7 @@ class FunctionParserElementAt : public FunctionParserArrayElement static constexpr auto name = "element_at"; String getName() const override { return name; } - const ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAGPtr & actions_dag) const override + const ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAG & actions_dag) const override { auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); if (parsed_args.size() != 2) diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/elt.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/elt.cpp index 23f372e5aef86..992235cd9a0bb 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/elt.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/elt.cpp @@ -43,7 +43,7 @@ class FunctionParserElt : public FunctionParser const ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - ActionsDAGPtr & actions_dag) const override + ActionsDAG & actions_dag) const override { /* parse elt(index, e1, e2, e3, ...) as diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/empty2null.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/empty2null.cpp index 081cff67ee447..424625092fe97 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/empty2null.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/empty2null.cpp @@ -41,7 +41,7 @@ class FunctionParserEmpty2Null : public FunctionParser String getName() const override { return name; } - const ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAGPtr & actions_dag) const override + const ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAG & actions_dag) const override { auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); if (parsed_args.size() != 1) diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/encode.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/encode.cpp index 2dcbffca2098a..30104fc59e791 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/encode.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/encode.cpp @@ -42,7 +42,7 @@ class FunctionParserEncode : public FunctionParser const ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - ActionsDAGPtr & actions_dag) const override + ActionsDAG & actions_dag) const override { /// Parse encode(str, charset) as convertCharset(str, 'UTF-8', charset) auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/equalNullSafe.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/equalNullSafe.cpp index d35bf810ffc69..ac6e8a59dd9e4 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/equalNullSafe.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/equalNullSafe.cpp @@ -40,7 +40,7 @@ class FunctionParserEqualNullSafe : public FunctionParser String getName() const override { return name; } - const ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAGPtr & actions_dag) const override + const ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAG & actions_dag) const override { /// Parse equal_null_safe(left, right) as: /// if (isNull(left) && isNull(right)) diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/expm1.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/expm1.cpp index ef98de6417ff7..4145063acf5c2 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/expm1.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/expm1.cpp @@ -39,7 +39,7 @@ class FunctionParserExpm1 : public FunctionParser const ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - ActionsDAGPtr & actions_dag) const override + ActionsDAG & actions_dag) const override { /// parse expm1(x) as exp(x) - 1 auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/extract.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/extract.cpp index 43cf1f3a34ef1..90e1180061fff 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/extract.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/extract.cpp @@ -84,7 +84,7 @@ class SparkFunctionExtractParser : public FunctionParser return ch_function_name; } - const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAGPtr & actions_dag) const override + const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { DB::ActionsDAG::NodeRawConstPtrs parsed_args; auto ch_function_name = getCHFunctionName(substrait_func); diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/factorial.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/factorial.cpp index f1ef4ec8b9bae..5854498d7b3a7 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/factorial.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/factorial.cpp @@ -43,7 +43,7 @@ class FunctionParserFactorial : public FunctionParser const ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - ActionsDAGPtr & actions_dag) const override + ActionsDAG & actions_dag) const override { /// parse factorial(x) as if (x > 20 || x < 0) null else factorial(x) auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/findInset.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/findInset.cpp index 3453431199638..ca9fb372c2fde 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/findInset.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/findInset.cpp @@ -45,7 +45,7 @@ class FunctionParserFindInSet : public FunctionParser const ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - ActionsDAGPtr & actions_dag) const override + ActionsDAG & actions_dag) const override { /* parse find_in_set(str, str_array) as diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/fromJson.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/fromJson.cpp index 2dd8754189b70..facad6e3bbc52 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/fromJson.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/fromJson.cpp @@ -34,7 +34,7 @@ class SparkFunctionFromJsonParser : public FunctionParser return "JSONExtract"; } - const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAGPtr & actions_dag) const override + const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { DB::ActionsDAG::NodeRawConstPtrs parsed_args; auto ch_function_name = getCHFunctionName(substrait_func); diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/fromUtcTimestamp.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/fromUtcTimestamp.cpp index 8d23231055c3f..b5b1d0b5553fe 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/fromUtcTimestamp.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/fromUtcTimestamp.cpp @@ -24,7 +24,7 @@ class FunctionParserFromUtcTimestamp : public FunctionParserUtcTimestampTransfor { public: explicit FunctionParserFromUtcTimestamp(SerializedPlanParser * plan_parser_) : FunctionParserUtcTimestampTransform(plan_parser_) { } - ~FunctionParserFromUtcTimestamp() = default; + ~FunctionParserFromUtcTimestamp() override = default; static constexpr auto name = "from_utc_timestamp"; String getCHFunctionName(const substrait::Expression_ScalarFunction &) const override { return "from_utc_timestamp"; } diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/getJSONObject.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/getJSONObject.cpp index aad75130aa476..04d7e1bf7d77f 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/getJSONObject.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/getJSONObject.cpp @@ -56,7 +56,7 @@ class GetJSONObjectParser : public FunctionParser /// Force to reuse the same flatten json column node DB::ActionsDAG::NodeRawConstPtrs parseFunctionArguments( const substrait::Expression_ScalarFunction & substrait_func, - DB::ActionsDAGPtr & actions_dag) const override + DB::ActionsDAG & actions_dag) const override { const auto & args = substrait_func.arguments(); if (args.size() != 2) @@ -67,14 +67,14 @@ class GetJSONObjectParser : public FunctionParser && args[0].value().scalar_function().function_reference() == SelfDefinedFunctionReference::GET_JSON_OBJECT) { auto flatten_json_column_name = getFlatterJsonColumnName(args[0].value()); - const auto * flatten_json_column_node = actions_dag->tryFindInOutputs(flatten_json_column_name); + const auto * flatten_json_column_node = actions_dag.tryFindInOutputs(flatten_json_column_name); if (!flatten_json_column_node) { const auto flatten_function_pb = args[0].value().scalar_function(); const auto * flatten_arg0 = parseExpression(actions_dag, flatten_function_pb.arguments(0).value()); const auto * flatten_arg1 = parseExpression(actions_dag, flatten_function_pb.arguments(1).value()); flatten_json_column_node = toFunctionNode(actions_dag, FlattenJSONStringOnRequiredFunction::name, flatten_json_column_name, {flatten_arg0, flatten_arg1}); - actions_dag->addOrReplaceInOutputs(*flatten_json_column_node); + actions_dag.addOrReplaceInOutputs(*flatten_json_column_node); } return {flatten_json_column_node, parseExpression(actions_dag, args[1].value())}; } diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/isNaN.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/isNaN.cpp index 3409c61d4651c..8f134ed245146 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/isNaN.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/isNaN.cpp @@ -33,7 +33,7 @@ class SparkFunctionIsNaNParser : public FunctionParser String getName() const override { return name; } String getCHFunctionName(const substrait::Expression_ScalarFunction &) const override { return "isNaN"; } - const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAGPtr & actions_dag) const override + const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { // the result of isNaN(NULL) is NULL in CH, but false in Spark DB::ActionsDAG::NodeRawConstPtrs parsed_args; diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/lambdaFunction.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/lambdaFunction.cpp index 6647b82b95666..547ffd971fcd4 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/lambdaFunction.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/lambdaFunction.cpp @@ -71,16 +71,16 @@ class LambdaFunction : public FunctionParser throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "getCHFunctionName is not implemented for LambdaFunction"); } - const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAGPtr & actions_dag) const override + const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { /// Some special cases, for example, `transform(arr, x -> concat(arr, array(x)))` refers to /// a column `arr` out of it directly. We need a `arr` as an input column for `lambda_actions_dag` DB::NamesAndTypesList parent_header; - for (const auto * output_node : actions_dag->getOutputs()) + for (const auto * output_node : actions_dag.getOutputs()) { parent_header.emplace_back(output_node->result_name, output_node->result_type); } - auto lambda_actions_dag = std::make_shared(parent_header); + ActionsDAG lambda_actions_dag{parent_header}; /// The first argument is the lambda function body, followings are the lambda arguments which is /// needed by the lambda function body. @@ -93,20 +93,20 @@ class LambdaFunction : public FunctionParser } const auto & substrait_lambda_body = substrait_func.arguments()[0].value(); const auto * lambda_body_node = parseExpression(lambda_actions_dag, substrait_lambda_body); - lambda_actions_dag->getOutputs().push_back(lambda_body_node); - lambda_actions_dag->removeUnusedActions(Names(1, lambda_body_node->result_name)); + lambda_actions_dag.getOutputs().push_back(lambda_body_node); + lambda_actions_dag.removeUnusedActions(Names(1, lambda_body_node->result_name)); auto expression_actions_settings = DB::ExpressionActionsSettings::fromContext(getContext(), DB::CompileExpressions::yes); - auto lambda_actions = std::make_shared(lambda_actions_dag, expression_actions_settings); + auto lambda_actions = std::make_shared(std::move(lambda_actions_dag), expression_actions_settings); DB::Names captured_column_names; DB::Names required_column_names = lambda_actions->getRequiredColumns(); DB::ActionsDAG::NodeRawConstPtrs lambda_children; auto lambda_function_args = collectLambdaArguments(*plan_parser, substrait_func); - const auto & lambda_actions_inputs = lambda_actions_dag->getInputs(); + const auto & lambda_actions_inputs = lambda_actions->getActionsDAG().getInputs(); std::unordered_map parent_nodes; - for (const auto & node : actions_dag->getNodes()) + for (const auto & node : actions_dag.getNodes()) { parent_nodes[node.result_name] = &node; } @@ -131,7 +131,7 @@ class LambdaFunction : public FunctionParser { throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Not found column {} in actions dag:\n{}", required_column_name, - actions_dag->dumpDAG()); + actions_dag.dumpDAG()); } /// The nodes must be the ones in `actions_dag`, otherwise `ActionsDAG::evaluatePartialResult` will fail. Because nodes may have the /// same name but their addresses are different. @@ -147,13 +147,13 @@ class LambdaFunction : public FunctionParser lambda_body_node->result_type, lambda_body_node->result_name); - const auto * result = &actions_dag->addFunction(function_capture, lambda_children, lambda_body_node->result_name); + const auto * result = &actions_dag.addFunction(function_capture, lambda_children, lambda_body_node->result_name); return result; } protected: DB::ActionsDAG::NodeRawConstPtrs parseFunctionArguments( const substrait::Expression_ScalarFunction & substrait_func, - DB::ActionsDAGPtr & actions_dag) const override + DB::ActionsDAG & actions_dag) const override { throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "parseFunctionArguments is not implemented for LambdaFunction"); } @@ -161,7 +161,7 @@ class LambdaFunction : public FunctionParser const DB::ActionsDAG::Node * convertNodeTypeIfNeeded( const substrait::Expression_ScalarFunction & substrait_func, const DB::ActionsDAG::Node * func_node, - DB::ActionsDAGPtr & actions_dag) const override + DB::ActionsDAG & actions_dag) const override { throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "convertNodeTypeIfNeeded is not implemented for NamedLambdaVariable"); } @@ -184,24 +184,24 @@ class NamedLambdaVariable : public FunctionParser throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "getCHFunctionName is not implemented for NamedLambdaVariable"); } - const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAGPtr & actions_dag) const override + const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { auto [_, col_name_field] = parseLiteral(substrait_func.arguments()[0].value().literal()); String col_name = col_name_field.get(); auto type = TypeParser::parseType(substrait_func.output_type()); - const auto & inputs = actions_dag->getInputs(); + const auto & inputs = actions_dag.getInputs(); auto it = std::find_if(inputs.begin(), inputs.end(), [&col_name](const auto * node) { return node->result_name == col_name; }); if (it == inputs.end()) { - return &(actions_dag->addInput(col_name, type)); + return &(actions_dag.addInput(col_name, type)); } return *it; } protected: DB::ActionsDAG::NodeRawConstPtrs parseFunctionArguments( const substrait::Expression_ScalarFunction & substrait_func, - DB::ActionsDAGPtr & actions_dag) const override + DB::ActionsDAG & actions_dag) const override { throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "parseFunctionArguments is not implemented for NamedLambdaVariable"); } @@ -209,7 +209,7 @@ class NamedLambdaVariable : public FunctionParser const DB::ActionsDAG::Node * convertNodeTypeIfNeeded( const substrait::Expression_ScalarFunction & substrait_func, const DB::ActionsDAG::Node * func_node, - DB::ActionsDAGPtr & actions_dag) const override + DB::ActionsDAG & actions_dag) const override { throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "convertNodeTypeIfNeeded is not implemented for NamedLambdaVariable"); } diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/length.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/length.cpp index af998d4d2e69b..cbe317ca7317d 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/length.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/length.cpp @@ -38,7 +38,7 @@ class FunctionParserLength : public FunctionParser String getName() const override { return name; } - const ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAGPtr & actions_dag) const override + const ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAG & actions_dag) const override { /** parse length(a) as diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/ln.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/ln.cpp index 0bb19dd1d2063..1eae98fc43337 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/ln.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/ln.cpp @@ -29,7 +29,7 @@ class FunctionParserLn : public FunctionParserLogBase String getName() const override { return name; } String getCHFunctionName() const override { return name; } - const DB::ActionsDAG::Node * getParameterLowerBound(ActionsDAGPtr & actions_dag, const DataTypePtr & data_type) const override + const DB::ActionsDAG::Node * getParameterLowerBound(ActionsDAG & actions_dag, const DataTypePtr & data_type) const override { return addColumnToActionsDAG(actions_dag, data_type, 0.0); } diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/locate.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/locate.cpp index efc6da7c4659d..b948daeda0eaa 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/locate.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/locate.cpp @@ -41,7 +41,7 @@ class FunctionParserLocate : public FunctionParser String getName() const override { return name; } - const ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAGPtr & actions_dag) const override + const ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAG & actions_dag) const override { /// Parse locate(substr, str, start_pos) as if(isNull(start_pos), 0, positionUTF8Spark(str, substr, start_pos) auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/log.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/log.cpp index 75a6894597f52..ace39d32ae380 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/log.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/log.cpp @@ -44,7 +44,7 @@ class FunctionParserLog : public FunctionParser const ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - ActionsDAGPtr & actions_dag) const override + ActionsDAG & actions_dag) const override { /* parse log(x, y) as diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/log10.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/log10.cpp index b62ef486d2a2b..2a5ae70eec0a3 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/log10.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/log10.cpp @@ -29,7 +29,7 @@ class FunctionParserLog10 : public FunctionParserLogBase String getName() const override { return name; } String getCHFunctionName() const override { return name; } - const DB::ActionsDAG::Node * getParameterLowerBound(ActionsDAGPtr & actions_dag, const DataTypePtr & data_type) const override + const DB::ActionsDAG::Node * getParameterLowerBound(ActionsDAG & actions_dag, const DataTypePtr & data_type) const override { return addColumnToActionsDAG(actions_dag, data_type, 0.0); } diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/log1p.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/log1p.cpp index d7ad5aa8ba90d..e5b1735656550 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/log1p.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/log1p.cpp @@ -29,7 +29,7 @@ class FunctionParserLog1p : public FunctionParserLogBase String getName() const override { return name; } String getCHFunctionName() const override { return name; } - const DB::ActionsDAG::Node * getParameterLowerBound(ActionsDAGPtr & actions_dag, const DataTypePtr & data_type) const override + const DB::ActionsDAG::Node * getParameterLowerBound(ActionsDAG & actions_dag, const DataTypePtr & data_type) const override { return addColumnToActionsDAG(actions_dag, data_type, -1.0); } diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/log2.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/log2.cpp index 5520fa035340a..481c81d538323 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/log2.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/log2.cpp @@ -29,7 +29,7 @@ class FunctionParserLog2 : public FunctionParserLogBase String getName() const override { return name; } String getCHFunctionName() const override { return name; } - const DB::ActionsDAG::Node * getParameterLowerBound(ActionsDAGPtr & actions_dag, const DataTypePtr & data_type) const override + const DB::ActionsDAG::Node * getParameterLowerBound(ActionsDAG & actions_dag, const DataTypePtr & data_type) const override { return addColumnToActionsDAG(actions_dag, data_type, 0.0); } diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/logarithm.h b/cpp-ch/local-engine/Parser/scalar_function_parser/logarithm.h index 7a83d78fa8459..d2232f80d1973 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/logarithm.h +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/logarithm.h @@ -39,11 +39,11 @@ class FunctionParserLogBase : public FunctionParser ~FunctionParserLogBase() override = default; virtual String getCHFunctionName() const = 0; - virtual const DB::ActionsDAG::Node * getParameterLowerBound(ActionsDAGPtr &, const DataTypePtr &) const { return nullptr; } + virtual const DB::ActionsDAG::Node * getParameterLowerBound(ActionsDAG &, const DataTypePtr &) const { return nullptr; } const ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - ActionsDAGPtr & actions_dag) const override + ActionsDAG & actions_dag) const override { /* parse log(x) as diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/makeDecimal.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/makeDecimal.cpp index 977167ef36015..64a21fb1b9dac 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/makeDecimal.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/makeDecimal.cpp @@ -46,7 +46,7 @@ class SparkFunctionMakeDecimalParser : public FunctionParser return ch_function_name; } - const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAGPtr & actions_dag) const override + const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { DB::ActionsDAG::NodeRawConstPtrs parsed_args; const auto & args = substrait_func.arguments(); diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/md5.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/md5.cpp index c57197e70d0b6..2401d6272cbc4 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/md5.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/md5.cpp @@ -40,7 +40,7 @@ class FunctionParserMd5 : public FunctionParser String getName() const override { return name; } - const ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAGPtr & actions_dag) const override + const ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAG & actions_dag) const override { /// Parse md5(str) as lower(hex(md5(str))) auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/nanvl.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/nanvl.cpp index d8f29d7275764..ec2934188e85e 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/nanvl.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/nanvl.cpp @@ -44,7 +44,7 @@ class FunctionParserNaNvl : public FunctionParser const ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - ActionsDAGPtr & actions_dag) const override + ActionsDAG & actions_dag) const override { /* parse nanvl(e1, e2) as diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/octetLength.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/octetLength.cpp index d2c159a1b69e9..2f231f01d84f5 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/octetLength.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/octetLength.cpp @@ -38,7 +38,7 @@ class FunctionParserOctetLength : public FunctionParser String getName() const override { return name; } - const ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAGPtr & actions_dag) const override + const ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAG & actions_dag) const override { auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); if (parsed_args.size() != 1) diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/parseUrl.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/parseUrl.cpp index af573367448fa..ead2010695bd1 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/parseUrl.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/parseUrl.cpp @@ -99,7 +99,7 @@ String ParseURLParser::selectCHFunctionName(const substrait::Expression_ScalarFu } DB::ActionsDAG::NodeRawConstPtrs ParseURLParser::parseFunctionArguments( - const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAGPtr & actions_dag) const + const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const { DB::ActionsDAG::NodeRawConstPtrs arg_nodes; arg_nodes.push_back(parseExpression(actions_dag, substrait_func.arguments(0).value())); @@ -111,7 +111,7 @@ DB::ActionsDAG::NodeRawConstPtrs ParseURLParser::parseFunctionArguments( } const DB::ActionsDAG::Node * ParseURLParser::convertNodeTypeIfNeeded( - const substrait::Expression_ScalarFunction & substrait_func, const DB::ActionsDAG::Node * func_node, DB::ActionsDAGPtr & actions_dag) const + const substrait::Expression_ScalarFunction & substrait_func, const DB::ActionsDAG::Node * func_node, DB::ActionsDAG & actions_dag) const { auto ch_function_name = getCHFunctionName(substrait_func); if (ch_function_name != CH_URL_PROTOL_FUNCTION) @@ -121,7 +121,7 @@ const DB::ActionsDAG::Node * ParseURLParser::convertNodeTypeIfNeeded( // Empty string is converted to NULL. auto str_type = std::make_shared(); const auto * empty_str_node - = &actions_dag->addColumn(ColumnWithTypeAndName(str_type->createColumnConst(1, DB::Field("")), str_type, getUniqueName(""))); + = &actions_dag.addColumn(ColumnWithTypeAndName(str_type->createColumnConst(1, DB::Field("")), str_type, getUniqueName(""))); return toFunctionNode(actions_dag, "nullIf", {func_node, empty_str_node}); } diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/parseUrl.h b/cpp-ch/local-engine/Parser/scalar_function_parser/parseUrl.h index a4d6e0f057ea3..d9994a39c23e0 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/parseUrl.h +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/parseUrl.h @@ -31,12 +31,12 @@ class ParseURLParser final : public FunctionParser DB::ActionsDAG::NodeRawConstPtrs parseFunctionArguments( const substrait::Expression_ScalarFunction & substrait_func, - DB::ActionsDAGPtr & actions_dag) const override; + DB::ActionsDAG & actions_dag) const override; const DB::ActionsDAG::Node * convertNodeTypeIfNeeded( const substrait::Expression_ScalarFunction & substrait_func, const DB::ActionsDAG::Node * func_node, - DB::ActionsDAGPtr & actions_dag) const override; + DB::ActionsDAG & actions_dag) const override; private: String getQueryPartName(const substrait::Expression & expr) const; diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/regexp_extract.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/regexp_extract.cpp index ba30a3c59e4ca..cf69e3396bb7f 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/regexp_extract.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/regexp_extract.cpp @@ -43,7 +43,7 @@ class FunctionParserRegexpExtract : public FunctionParser const ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - ActionsDAGPtr & actions_dag) const override + ActionsDAG & actions_dag) const override { const auto & args = substrait_func.arguments(); if (args.size() != 3) diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/repeat.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/repeat.cpp index cc32fc0155357..ada91f8537fe4 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/repeat.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/repeat.cpp @@ -33,7 +33,7 @@ class SparkFunctionRepeatParser : public FunctionParser String getName() const override { return name; } String getCHFunctionName(const substrait::Expression_ScalarFunction &) const override { return name; } - const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAGPtr & actions_dag) const override + const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { // repeat. the field index must be unsigned integer in CH, cast the signed integer in substrait // which must be a positive value into unsigned integer here. diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/sec.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/sec.cpp index 4b95bcbe530f4..8dbc2b4a96838 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/sec.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/sec.cpp @@ -41,7 +41,7 @@ class FunctionParserSec : public FunctionParser const ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - ActionsDAGPtr & actions_dag) const override + ActionsDAG & actions_dag) const override { /// parse sec(x) as 1 / cos(x) auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/sequence.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/sequence.cpp index 0e98759f6c7fe..4455c83c89494 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/sequence.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/sequence.cpp @@ -42,7 +42,7 @@ class FunctionParserSequence : public FunctionParser const ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - ActionsDAGPtr & actions_dag) const override + ActionsDAG & actions_dag) const override { /** parse sequence(start, end, step) as diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/sha1.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/sha1.cpp index eb7578a3f4b65..0fed49b4cdd4c 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/sha1.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/sha1.cpp @@ -40,7 +40,7 @@ class FunctionParserSha1 : public FunctionParser String getName() const override { return name; } - const ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAGPtr & actions_dag) const override + const ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAG & actions_dag) const override { /// Parse sha1(str) as lower(hex(sha1(str))) auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/sha2.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/sha2.cpp index 75db4cd173fd6..e05fef0e68b0c 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/sha2.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/sha2.cpp @@ -41,7 +41,7 @@ class FunctionParserSha2 : public FunctionParser String getName() const override { return name; } - const ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAGPtr & actions_dag) const override + const ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAG & actions_dag) const override { /// Parse sha2(str, 0) or sha2(str, 0) as lower(hex(SHA256(str))) /// Parse sha2(str, 224) as lower(hex(SHA224(str))) diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/shiftRightUnsigned.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/shiftRightUnsigned.cpp index e0932e621b758..28288461a1dac 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/shiftRightUnsigned.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/shiftRightUnsigned.cpp @@ -39,7 +39,7 @@ class FunctionParserShiftRightUnsigned : public FunctionParser String getName() const override { return name; } - const ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAGPtr & actions_dag) const override + const ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAG & actions_dag) const override { /// parse shiftrightunsigned(a, b) as /// if (isInteger(a)) diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/size.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/size.cpp index 09db14ced0f09..3c53e7a3c3637 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/size.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/size.cpp @@ -40,7 +40,7 @@ class FunctionParserSize : public FunctionParser String getName() const override { return name; } - const ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAGPtr & actions_dag) const override + const ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAG & actions_dag) const override { /// Parse size(child, true) as ifNull(length(child), -1) /// Parse size(child, false) as length(child) diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/slice.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/slice.cpp index 46f00ce7cf552..2643207354ae7 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/slice.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/slice.cpp @@ -43,7 +43,7 @@ class FunctionParserArraySlice : public FunctionParser const ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - ActionsDAGPtr & actions_dag) const override + ActionsDAG & actions_dag) const override { /** parse slice(arr, start, length) as @@ -104,7 +104,7 @@ class FunctionParserArraySlice : public FunctionParser private: // if (start=0) then throwIf(start=0) else start const ActionsDAG::Node * makeStartIfNode( - ActionsDAGPtr & actions_dag, + ActionsDAG & actions_dag, const ActionsDAG::Node * start_arg, const ActionsDAG::Node * zero_const_node) const { @@ -116,7 +116,7 @@ class FunctionParserArraySlice : public FunctionParser // if (length<0) then throwIf(length<0) else length const ActionsDAG::Node * makeLengthIfNode( - ActionsDAGPtr & actions_dag, + ActionsDAG & actions_dag, const ActionsDAG::Node * length_arg, const ActionsDAG::Node * zero_const_node) const { diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/sortArray.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/sortArray.cpp index 3386b642fa21a..ecd38db19bb07 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/sortArray.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/sortArray.cpp @@ -43,7 +43,7 @@ class FunctionParserSortArray : public FunctionParser const ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - ActionsDAGPtr & actions_dag) const override + ActionsDAG & actions_dag) const override { auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); if (parsed_args.size() != 2) diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/space.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/space.cpp index 3698ddad78cfa..f60459c3857b5 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/space.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/space.cpp @@ -32,7 +32,7 @@ class SparkFunctionSpaceParser : public FunctionParser String getName() const override { return name; } String getCHFunctionName(const substrait::Expression_ScalarFunction &) const override { return "repeat"; } - const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAGPtr & actions_dag) const override + const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { // convert space function to repeat DB::ActionsDAG::NodeRawConstPtrs parsed_args; diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/split.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/split.cpp index 05749da895528..aba8f50dfa357 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/split.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/split.cpp @@ -28,7 +28,7 @@ class SparkFunctionSplitParser : public FunctionParser String getName() const override { return name; } String getCHFunctionName(const substrait::Expression_ScalarFunction &) const override { return "splitByRegexp"; } - const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAGPtr & actions_dag) const override + const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { DB::ActionsDAG::NodeRawConstPtrs parsed_args; const auto & args = substrait_func.arguments(); diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/substring.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/substring.cpp index 444213973cb23..cb0ae511f7d05 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/substring.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/substring.cpp @@ -40,7 +40,7 @@ class FunctionParserSubstring : public FunctionParser const ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - ActionsDAGPtr & actions_dag) const override + ActionsDAG & actions_dag) const override { auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); if (parsed_args.size() != 3) diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/timestampAdd.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/timestampAdd.cpp index af81e2bd7455b..caaa01cb5d48c 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/timestampAdd.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/timestampAdd.cpp @@ -44,7 +44,7 @@ class FunctionParserTimestampAdd : public FunctionParser String getName() const override { return name; } String getCHFunctionName(const substrait::Expression_ScalarFunction &) const override { return "timestamp_add"; } - const ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAGPtr & actions_dag) const override + const ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAG & actions_dag) const override { auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); if (parsed_args.size() < 3) diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/toUtcTimestamp.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/toUtcTimestamp.cpp index 4b04942bab310..72c52e40375e2 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/toUtcTimestamp.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/toUtcTimestamp.cpp @@ -24,7 +24,7 @@ class FunctionParserToUtcTimestamp : public FunctionParserUtcTimestampTransform { public: explicit FunctionParserToUtcTimestamp(SerializedPlanParser * plan_parser_) : FunctionParserUtcTimestampTransform(plan_parser_) { } - ~FunctionParserToUtcTimestamp() = default; + ~FunctionParserToUtcTimestamp() override = default; static constexpr auto name = "to_utc_timestamp"; String getCHFunctionName(const substrait::Expression_ScalarFunction &) const override { return "to_utc_timestamp"; } diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/trimFunctions.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/trimFunctions.cpp index e07196b282e0c..93e5b652ee3d4 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/trimFunctions.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/trimFunctions.cpp @@ -33,7 +33,7 @@ class SparkFunctionTrimParser : public FunctionParser return func.arguments().size() == 1 ? "trimBoth" : "trimBothSpark"; } - const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAGPtr & actions_dag) const override + const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { DB::ActionsDAG::NodeRawConstPtrs parsed_args; auto ch_function_name = getCHFunctionName(substrait_func); @@ -70,7 +70,7 @@ class SparkFunctionLtrimParser : public FunctionParser return func.arguments().size() == 1 ? "trimLeft" : "trimLeftSpark"; } - const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAGPtr & actions_dag) const override + const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { DB::ActionsDAG::NodeRawConstPtrs parsed_args; auto ch_function_name = getCHFunctionName(substrait_func); @@ -106,7 +106,7 @@ class SparkFunctionRtrimParser : public FunctionParser return func.arguments().size() == 1 ? "trimRight" : "trimRightSpark"; } - const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAGPtr & actions_dag) const override + const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { DB::ActionsDAG::NodeRawConstPtrs parsed_args; auto ch_function_name = getCHFunctionName(substrait_func); diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/trunc.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/trunc.cpp index 625d67a7e1c65..433e1af6f0c36 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/trunc.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/trunc.cpp @@ -45,7 +45,7 @@ class FunctionParserTrunc : public FunctionParser const ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - ActionsDAGPtr & actions_dag) const override + ActionsDAG & actions_dag) const override { auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); if (parsed_args.size() != 2) diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/tuple.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/tuple.cpp index 3228efb0ed887..b024ef486e0e3 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/tuple.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/tuple.cpp @@ -30,7 +30,7 @@ class SparkFunctionNamedStructParser : public FunctionParser String getName () const override { return name; } String getCHFunctionName(const substrait::Expression_ScalarFunction &) const override { return "tuple"; } - const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAGPtr & actions_dag) const override + const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { DB::ActionsDAG::NodeRawConstPtrs parsed_args; const auto & args = substrait_func.arguments(); diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/tupleElement.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/tupleElement.cpp index 6cf0acff0d04f..179aa7860484d 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/tupleElement.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/tupleElement.cpp @@ -34,7 +34,7 @@ namespace local_engine static constexpr auto name = #substrait_name; \ String getName () const override { return name; } \ String getCHFunctionName(const substrait::Expression_ScalarFunction &) const override { return #ch_name; } \ - const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAGPtr & actions_dag) const override \ + const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override \ { \ DB::ActionsDAG::NodeRawConstPtrs parsed_args; \ auto ch_function_name = getCHFunctionName(substrait_func); \ diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/unixTimestamp.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/unixTimestamp.cpp index 9488b89be67a1..0b1cee76fe6b3 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/unixTimestamp.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/unixTimestamp.cpp @@ -45,7 +45,7 @@ class FunctionParserUnixTimestamp : public FunctionParser const ActionsDAG::Node * parse( const substrait::Expression_ScalarFunction & substrait_func, - ActionsDAGPtr & actions_dag) const override + ActionsDAG & actions_dag) const override { /* spark function: unix_timestamp(expr, fmt) diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/utcTimestampTransform.h b/cpp-ch/local-engine/Parser/scalar_function_parser/utcTimestampTransform.h index b3b639c562bd5..387f7b6a36477 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/utcTimestampTransform.h +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/utcTimestampTransform.h @@ -39,7 +39,7 @@ class FunctionParserUtcTimestampTransform : public FunctionParser explicit FunctionParserUtcTimestampTransform(SerializedPlanParser * plan_parser_) : FunctionParser(plan_parser_) { } ~FunctionParserUtcTimestampTransform() override = default; - const ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAGPtr & actions_dag) const override + const ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, ActionsDAG & actions_dag) const override { /// Convert timezone value to clickhouse backend supported, i.e. GMT+8 -> Etc/GMT-8, +08:00 -> Etc/GMT-8 if (substrait_func.arguments_size() != 2) diff --git a/cpp-ch/local-engine/Shuffle/SelectorBuilder.cpp b/cpp-ch/local-engine/Shuffle/SelectorBuilder.cpp index 6804770c34c15..b9bd02c3ef687 100644 --- a/cpp-ch/local-engine/Shuffle/SelectorBuilder.cpp +++ b/cpp-ch/local-engine/Shuffle/SelectorBuilder.cpp @@ -338,7 +338,7 @@ void RangeSelectorBuilder::initActionsDAG(const DB::Block & block) exprs.emplace_back(expression); auto projection_actions_dag = plan_parser.expressionsToActionsDAG(exprs, block, block); - projection_expression_actions = std::make_unique(projection_actions_dag); + projection_expression_actions = std::make_unique(std::move(projection_actions_dag)); has_init_actions_dag = true; } diff --git a/cpp-ch/local-engine/Storages/Mergetree/SparkMergeTreeWriter.cpp b/cpp-ch/local-engine/Storages/Mergetree/SparkMergeTreeWriter.cpp index da92eeba83ce4..caee87cb9416c 100644 --- a/cpp-ch/local-engine/Storages/Mergetree/SparkMergeTreeWriter.cpp +++ b/cpp-ch/local-engine/Storages/Mergetree/SparkMergeTreeWriter.cpp @@ -112,9 +112,10 @@ bool SparkMergeTreeWriter::useLocalStorage() const void SparkMergeTreeWriter::write(const DB::Block & block) { auto new_block = removeColumnSuffix(block); - if (auto converter = ActionsDAG::makeConvertingActions( - new_block.getColumnsWithTypeAndName(), header.getColumnsWithTypeAndName(), DB::ActionsDAG::MatchColumnsMode::Position)) - ExpressionActions(converter).execute(new_block); + auto converter = ActionsDAG::makeConvertingActions( + new_block.getColumnsWithTypeAndName(), header.getColumnsWithTypeAndName(), DB::ActionsDAG::MatchColumnsMode::Position); + const ExpressionActions expression_actions{std::move(converter)}; + expression_actions.execute(new_block); bool has_part = chunkToPart(squashing->add({new_block.getColumns(), new_block.rows()})); diff --git a/cpp-ch/local-engine/Storages/Parquet/ColumnIndexFilter.cpp b/cpp-ch/local-engine/Storages/Parquet/ColumnIndexFilter.cpp index 817de7f27ef85..1063cba8a02fc 100644 --- a/cpp-ch/local-engine/Storages/Parquet/ColumnIndexFilter.cpp +++ b/cpp-ch/local-engine/Storages/Parquet/ColumnIndexFilter.cpp @@ -787,13 +787,13 @@ const ColumnIndexFilter::AtomMap ColumnIndexFilter::atom_map{ return true; }}}; -ColumnIndexFilter::ColumnIndexFilter(const DB::ActionsDAGPtr & filter_dag, DB::ContextPtr context) +ColumnIndexFilter::ColumnIndexFilter(const DB::ActionsDAG & filter_dag, DB::ContextPtr context) { - const auto inverted_dag = DB::KeyCondition::cloneASTWithInversionPushDown({filter_dag->getOutputs().at(0)}, context); + const auto inverted_dag = DB::KeyCondition::cloneASTWithInversionPushDown({filter_dag.getOutputs().at(0)}, context); - assert(inverted_dag->getOutputs().size() == 1); + assert(inverted_dag.getOutputs().size() == 1); - const auto * inverted_dag_filter_node = inverted_dag->getOutputs()[0]; + const auto * inverted_dag_filter_node = inverted_dag.getOutputs()[0]; DB::RPNBuilder builder( inverted_dag_filter_node, diff --git a/cpp-ch/local-engine/Storages/Parquet/ColumnIndexFilter.h b/cpp-ch/local-engine/Storages/Parquet/ColumnIndexFilter.h index 8ffeb7a228dd8..f5c5cc56168fa 100644 --- a/cpp-ch/local-engine/Storages/Parquet/ColumnIndexFilter.h +++ b/cpp-ch/local-engine/Storages/Parquet/ColumnIndexFilter.h @@ -196,7 +196,7 @@ class ColumnIndexFilter static const AtomMap atom_map; /// Construct key condition from ActionsDAG nodes - ColumnIndexFilter(const DB::ActionsDAGPtr & filter_dag, DB::ContextPtr context); + ColumnIndexFilter(const DB::ActionsDAG & filter_dag, DB::ContextPtr context); private: static bool extractAtomFromTree(const DB::RPNBuilderTreeNode & node, RPNElement & out); diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.cpp index 5b872244eab5c..d4e9f1eb8d4b3 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.cpp @@ -76,11 +76,11 @@ SubstraitFileSource::SubstraitFileSource( } } -void SubstraitFileSource::setKeyCondition(const DB::ActionsDAGPtr & filter_actions_dag, DB::ContextPtr context_) +void SubstraitFileSource::setKeyCondition(const std::optional & filter_actions_dag, DB::ContextPtr context_) { setKeyConditionImpl(filter_actions_dag, context_, to_read_header); if (filter_actions_dag) - column_index_filter = std::make_shared(filter_actions_dag, context_); + column_index_filter = std::make_shared(filter_actions_dag.value(), context_); } DB::Chunk SubstraitFileSource::generate() diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.h b/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.h index 650ec5d967a0f..571e4097107ac 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.h +++ b/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.h @@ -124,7 +124,7 @@ class SubstraitFileSource : public DB::SourceWithKeyCondition String getName() const override { return "SubstraitFileSource"; } - void setKeyCondition(const DB::ActionsDAGPtr & filter_actions_dag, DB::ContextPtr context_) override; + void setKeyCondition(const std::optional & filter_actions_dag, DB::ContextPtr context_) override; protected: DB::Chunk generate() override; diff --git a/cpp-ch/local-engine/tests/benchmark_local_engine.cpp b/cpp-ch/local-engine/tests/benchmark_local_engine.cpp index cf9ecf37dd30d..22b55ecf7d217 100644 --- a/cpp-ch/local-engine/tests/benchmark_local_engine.cpp +++ b/cpp-ch/local-engine/tests/benchmark_local_engine.cpp @@ -823,20 +823,20 @@ QueryPlanPtr joinPlan(QueryPlanPtr left, QueryPlanPtr right, String left_key, St auto left_keys = left->getCurrentDataStream().header.getNamesAndTypesList(); join->addJoinedColumnsAndCorrectTypes(left_keys, true); - ActionsDAGPtr left_convert_actions = nullptr; - ActionsDAGPtr right_convert_actions = nullptr; + std::optional left_convert_actions; + std::optional right_convert_actions; std::tie(left_convert_actions, right_convert_actions) = join->createConvertingActions(left_columns, right_columns); if (right_convert_actions) { - auto converting_step = std::make_unique(right->getCurrentDataStream(), right_convert_actions); + auto converting_step = std::make_unique(right->getCurrentDataStream(), std::move(*right_convert_actions)); converting_step->setStepDescription("Convert joined columns"); right->addStep(std::move(converting_step)); } if (left_convert_actions) { - auto converting_step = std::make_unique(right->getCurrentDataStream(), right_convert_actions); + auto converting_step = std::make_unique(right->getCurrentDataStream(), std::move(*right_convert_actions)); converting_step->setStepDescription("Convert joined columns"); left->addStep(std::move(converting_step)); } diff --git a/cpp-ch/local-engine/tests/benchmark_parquet_read.cpp b/cpp-ch/local-engine/tests/benchmark_parquet_read.cpp index 52d534a23a489..5cfe51389f2fb 100644 --- a/cpp-ch/local-engine/tests/benchmark_parquet_read.cpp +++ b/cpp-ch/local-engine/tests/benchmark_parquet_read.cpp @@ -183,7 +183,7 @@ substrait::ReadRel::LocalFiles createLocalFiles(const std::string & filename, co return files; } -void doRead(const substrait::ReadRel::LocalFiles & files, const DB::ActionsDAGPtr & pushDown, const DB::Block & header) +void doRead(const substrait::ReadRel::LocalFiles & files, const std::optional & pushDown, const DB::Block & header) { const auto builder = std::make_unique(); const auto source @@ -215,7 +215,7 @@ void BM_ColumnIndexRead_Filter_ReturnAllResult(benchmark::State & state) const std::string filter1 = "l_shipdate is not null AND l_shipdate <= toDate32('1998-09-01')"; const substrait::ReadRel::LocalFiles files = createLocalFiles(filename, true); const AnotherRowType schema = local_engine::test::readParquetSchema(filename); - const ActionsDAGPtr pushDown = local_engine::test::parseFilter(filter1, schema); + auto pushDown = local_engine::test::parseFilter(filter1, schema); const Block header = {toBlockRowType(schema)}; for (auto _ : state) @@ -232,7 +232,7 @@ void BM_ColumnIndexRead_Filter_ReturnHalfResult(benchmark::State & state) const std::string filter1 = "l_orderkey is not null AND l_orderkey > 300977829"; const substrait::ReadRel::LocalFiles files = createLocalFiles(filename, true); const AnotherRowType schema = local_engine::test::readParquetSchema(filename); - const ActionsDAGPtr pushDown = local_engine::test::parseFilter(filter1, schema); + auto pushDown = local_engine::test::parseFilter(filter1, schema); const Block header = {toBlockRowType(schema)}; for (auto _ : state) diff --git a/cpp-ch/local-engine/tests/benchmark_spark_divide_function.cpp b/cpp-ch/local-engine/tests/benchmark_spark_divide_function.cpp index 1fe077f2a7b6b..7f1a7309e7d44 100644 --- a/cpp-ch/local-engine/tests/benchmark_spark_divide_function.cpp +++ b/cpp-ch/local-engine/tests/benchmark_spark_divide_function.cpp @@ -66,59 +66,59 @@ static std::string join(const ActionsDAG::NodeRawConstPtrs & v, char c) return res; } -static const ActionsDAG::Node * addFunction(ActionsDAGPtr & actions_dag, const String & function, const DB::ActionsDAG::NodeRawConstPtrs & args) +static const ActionsDAG::Node * addFunction(ActionsDAG & actions_dag, const String & function, const DB::ActionsDAG::NodeRawConstPtrs & args) { auto function_builder = FunctionFactory::instance().get(function, local_engine::SerializedPlanParser::global_context); std::string args_name = join(args, ','); auto result_name = function + "(" + args_name + ")"; - return &actions_dag->addFunction(function_builder, args, result_name); + return &actions_dag.addFunction(function_builder, args, result_name); } static void BM_CHDivideFunction(benchmark::State & state) { - ActionsDAGPtr dag = std::make_shared(); + ActionsDAG dag; Block block = createDataBlock("d1", "d2", 30000000); ColumnWithTypeAndName col1 = block.getByPosition(0); ColumnWithTypeAndName col2 = block.getByPosition(1); - const ActionsDAG::Node * left_arg = &dag->addColumn(col1); - const ActionsDAG::Node * right_arg = &dag->addColumn(col2); + const ActionsDAG::Node * left_arg = &dag.addColumn(col1); + const ActionsDAG::Node * right_arg = &dag.addColumn(col2); addFunction(dag, "divide", {left_arg, right_arg}); - ExpressionActions expr_actions(dag); + ExpressionActions expr_actions(std::move(dag)); for (auto _ : state) expr_actions.execute(block); } static void BM_SparkDivideFunction(benchmark::State & state) { - ActionsDAGPtr dag = std::make_shared(); + ActionsDAG dag; Block block = createDataBlock("d1", "d2", 30000000); ColumnWithTypeAndName col1 = block.getByPosition(0); ColumnWithTypeAndName col2 = block.getByPosition(1); - const ActionsDAG::Node * left_arg = &dag->addColumn(col1); - const ActionsDAG::Node * right_arg = &dag->addColumn(col2); + const ActionsDAG::Node * left_arg = &dag.addColumn(col1); + const ActionsDAG::Node * right_arg = &dag.addColumn(col2); addFunction(dag, "sparkDivide", {left_arg, right_arg}); - ExpressionActions expr_actions(dag); + ExpressionActions expr_actions(std::move(dag)); for (auto _ : state) expr_actions.execute(block); } static void BM_GlutenDivideFunctionParser(benchmark::State & state) { - ActionsDAGPtr dag = std::make_shared(); + ActionsDAG dag; Block block = createDataBlock("d1", "d2", 30000000); ColumnWithTypeAndName col1 = block.getByPosition(0); ColumnWithTypeAndName col2 = block.getByPosition(1); - const ActionsDAG::Node * left_arg = &dag->addColumn(col1); - const ActionsDAG::Node * right_arg = &dag->addColumn(col2); + const ActionsDAG::Node * left_arg = &dag.addColumn(col1); + const ActionsDAG::Node * right_arg = &dag.addColumn(col2); const ActionsDAG::Node * divide_arg = addFunction(dag, "divide", {left_arg, right_arg}); DataTypePtr float64_type = std::make_shared(); ColumnWithTypeAndName col_zero(float64_type->createColumnConst(1, 0), float64_type, toString(0)); ColumnWithTypeAndName col_null(float64_type->createColumnConst(1, Field{}), float64_type, "null"); - const ActionsDAG::Node * zero_arg = &dag->addColumn(col_zero); - const ActionsDAG::Node * null_arg = &dag->addColumn(col_null); + const ActionsDAG::Node * zero_arg = &dag.addColumn(col_zero); + const ActionsDAG::Node * null_arg = &dag.addColumn(col_null); const ActionsDAG::Node * equals_arg = addFunction(dag, "equals", {right_arg, zero_arg}); const ActionsDAG::Node * if_arg = addFunction(dag, "if", {equals_arg, null_arg, divide_arg}); - ExpressionActions expr_actions(dag); + ExpressionActions expr_actions(std::move(dag)); for (auto _ : state) expr_actions.execute(block); } diff --git a/cpp-ch/local-engine/tests/gluten_test_util.cpp b/cpp-ch/local-engine/tests/gluten_test_util.cpp index 1f1bd99836967..2d558ebe47445 100644 --- a/cpp-ch/local-engine/tests/gluten_test_util.cpp +++ b/cpp-ch/local-engine/tests/gluten_test_util.cpp @@ -41,7 +41,7 @@ extern const int LOGICAL_ERROR; namespace local_engine::test { using namespace DB; -ActionsDAGPtr parseFilter(const std::string & filter, const AnotherRowType & name_and_types) +std::optional parseFilter(const std::string & filter, const AnotherRowType & name_and_types) { using namespace DB; diff --git a/cpp-ch/local-engine/tests/gluten_test_util.h b/cpp-ch/local-engine/tests/gluten_test_util.h index 34e05b8b188b7..996b27bf884de 100644 --- a/cpp-ch/local-engine/tests/gluten_test_util.h +++ b/cpp-ch/local-engine/tests/gluten_test_util.h @@ -63,7 +63,7 @@ DB::DataTypePtr toDataType(const parquet::ColumnDescriptor & type); AnotherRowType readParquetSchema(const std::string & file); -DB::ActionsDAGPtr parseFilter(const std::string & filter, const AnotherRowType & name_and_types); +std::optional parseFilter(const std::string & filter, const AnotherRowType & name_and_types); } diff --git a/cpp-ch/local-engine/tests/gtest_ch_join.cpp b/cpp-ch/local-engine/tests/gtest_ch_join.cpp index 3202fb235a5f5..93b567f3b8779 100644 --- a/cpp-ch/local-engine/tests/gtest_ch_join.cpp +++ b/cpp-ch/local-engine/tests/gtest_ch_join.cpp @@ -102,21 +102,21 @@ TEST(TestJoin, simple) std::cerr << "after join:\n"; for (const auto & key : left_keys) std::cerr << key.dump() << std::endl; - ActionsDAGPtr left_convert_actions = nullptr; - ActionsDAGPtr right_convert_actions = nullptr; + std::optional left_convert_actions; + std::optional right_convert_actions; std::tie(left_convert_actions, right_convert_actions) = join->createConvertingActions(left.getColumnsWithTypeAndName(), right.getColumnsWithTypeAndName()); if (right_convert_actions) { - auto converting_step = std::make_unique(right_plan.getCurrentDataStream(), right_convert_actions); + auto converting_step = std::make_unique(right_plan.getCurrentDataStream(), std::move(*right_convert_actions)); converting_step->setStepDescription("Convert joined columns"); right_plan.addStep(std::move(converting_step)); } if (left_convert_actions) { - auto converting_step = std::make_unique(right_plan.getCurrentDataStream(), right_convert_actions); + auto converting_step = std::make_unique(right_plan.getCurrentDataStream(), std::move(*right_convert_actions)); converting_step->setStepDescription("Convert joined columns"); left_plan.addStep(std::move(converting_step)); } @@ -134,10 +134,10 @@ TEST(TestJoin, simple) auto query_plan = QueryPlan(); query_plan.unitePlans(std::move(join_step), {std::move(plans)}); std::cerr << query_plan.getCurrentDataStream().header.dumpStructure() << std::endl; - ActionsDAGPtr project = std::make_shared(query_plan.getCurrentDataStream().header.getNamesAndTypesList()); - project->project( + ActionsDAG project{query_plan.getCurrentDataStream().header.getNamesAndTypesList()}; + project.project( {NameWithAlias("colA", "colA"), NameWithAlias("colB", "colB"), NameWithAlias("colD", "colD"), NameWithAlias("colC", "colC")}); - QueryPlanStepPtr project_step = std::make_unique(query_plan.getCurrentDataStream(), project); + QueryPlanStepPtr project_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(project)); query_plan.addStep(std::move(project_step)); auto pipeline = query_plan.buildQueryPipeline(QueryPlanOptimizationSettings(), BuildQueryPipelineSettings()); auto executable_pipe = QueryPipelineBuilder::getPipeline(std::move(*pipeline)); diff --git a/cpp-ch/local-engine/tests/gtest_parquet_columnindex.cpp b/cpp-ch/local-engine/tests/gtest_parquet_columnindex.cpp index fbd7fbc63c270..45aaf3db6f851 100644 --- a/cpp-ch/local-engine/tests/gtest_parquet_columnindex.cpp +++ b/cpp-ch/local-engine/tests/gtest_parquet_columnindex.cpp @@ -359,7 +359,7 @@ void testCondition(const std::string & exp, const std::vector & expected static const AnotherRowType name_and_types = buildTestRowType(); static const local_engine::ColumnIndexStore column_index_store = buildTestColumnIndexStore(); const local_engine::ColumnIndexFilter filter( - local_engine::test::parseFilter(exp, name_and_types), local_engine::SerializedPlanParser::global_context); + local_engine::test::parseFilter(exp, name_and_types).value(), local_engine::SerializedPlanParser::global_context); assertRows(filter.calculateRowRanges(column_index_store, TOTALSIZE), expectedRows); } @@ -479,7 +479,7 @@ TEST(ColumnIndex, FilteringWithNotFoundColumnName) // COLUMN5 is not found in the column_index_store, const AnotherRowType upper_name_and_types{{"COLUMN5", BIGINT()}}; const local_engine::ColumnIndexFilter filter_upper( - local_engine::test::parseFilter("COLUMN5 in (7, 20)", upper_name_and_types), + local_engine::test::parseFilter("COLUMN5 in (7, 20)", upper_name_and_types).value(), local_engine::SerializedPlanParser::global_context); assertRows( filter_upper.calculateRowRanges(column_index_store, TOTALSIZE), @@ -489,7 +489,7 @@ TEST(ColumnIndex, FilteringWithNotFoundColumnName) { const AnotherRowType lower_name_and_types{{"column5", BIGINT()}}; const local_engine::ColumnIndexFilter filter_lower( - local_engine::test::parseFilter("column5 in (7, 20)", lower_name_and_types), + local_engine::test::parseFilter("column5 in (7, 20)", lower_name_and_types).value(), local_engine::SerializedPlanParser::global_context); assertRows(filter_lower.calculateRowRanges(column_index_store, TOTALSIZE), {}); } @@ -1053,7 +1053,7 @@ TEST(ColumnIndex, VectorizedParquetRecordReader) static const AnotherRowType name_and_types{{"11", BIGINT()}}; const auto filterAction = local_engine::test::parseFilter("`11` = 10 or `11` = 50", name_and_types); auto column_index_filter - = std::make_shared(filterAction, local_engine::SerializedPlanParser::global_context); + = std::make_shared(filterAction.value(), local_engine::SerializedPlanParser::global_context); Block blockHeader({{BIGINT(), "11"}, {STRING(), "18"}}); From 7f8aefb4aea09f0fb8da8238e51b188d5ab565ef Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Wed, 31 Jul 2024 08:43:20 +0800 Subject: [PATCH 28/61] [VL] Daily Update Velox Version (2024_07_31) (#6643) --- ep/build-velox/src/get_velox.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 774ec07886337..d8dfa09d45755 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_07_30 +VELOX_BRANCH=2024_07_31 VELOX_HOME="" OS=`uname -s` From 257178002b575452fbe6b445883a6d5de086bf58 Mon Sep 17 00:00:00 2001 From: Rong Ma Date: Wed, 31 Jul 2024 10:10:01 +0800 Subject: [PATCH 29/61] [VL] Reduce spill in sort-based shuffle (#6639) --- cpp/core/shuffle/LocalPartitionWriter.cc | 146 +++++++++++--------- cpp/core/shuffle/LocalPartitionWriter.h | 7 +- cpp/core/shuffle/Spill.cc | 16 +++ cpp/core/shuffle/Spill.h | 10 ++ cpp/velox/shuffle/VeloxSortShuffleWriter.cc | 6 +- cpp/velox/shuffle/VeloxSortShuffleWriter.h | 2 +- 6 files changed, 112 insertions(+), 75 deletions(-) diff --git a/cpp/core/shuffle/LocalPartitionWriter.cc b/cpp/core/shuffle/LocalPartitionWriter.cc index 4383e64892372..fc5d758f8c8b2 100644 --- a/cpp/core/shuffle/LocalPartitionWriter.cc +++ b/cpp/core/shuffle/LocalPartitionWriter.cc @@ -32,12 +32,12 @@ class LocalPartitionWriter::LocalSpiller { public: LocalSpiller( std::shared_ptr os, - const std::string& spillFile, + std::string spillFile, uint32_t compressionThreshold, arrow::MemoryPool* pool, arrow::util::Codec* codec) : os_(os), - spillFile_(spillFile), + spillFile_(std::move(spillFile)), compressionThreshold_(compressionThreshold), pool_(pool), codec_(codec), @@ -69,13 +69,17 @@ class LocalPartitionWriter::LocalSpiller { return arrow::Status::OK(); } - arrow::Result> finish() { - if (finished_) { - return arrow::Status::Invalid("Calling toBlockPayload() on a finished SpillEvictor."); - } + arrow::Result> finish(bool close) { + ARROW_RETURN_IF(finished_, arrow::Status::Invalid("Calling finish() on a finished LocalSpiller.")); + ARROW_RETURN_IF(os_->closed(), arrow::Status::Invalid("Spill file os has been closed.")); + finished_ = true; - RETURN_NOT_OK(os_->Close()); - diskSpill_->setSpillFile(std::move(spillFile_)); + if (close) { + RETURN_NOT_OK(os_->Close()); + } + diskSpill_->setSpillFile(spillFile_); + diskSpill_->setSpillTime(spillTime_); + diskSpill_->setCompressTime(compressTime_); return std::move(diskSpill_); } @@ -83,14 +87,6 @@ class LocalPartitionWriter::LocalSpiller { return finished_; } - int64_t getSpillTime() const { - return spillTime_; - } - - int64_t getCompressTime() const { - return compressTime_; - } - private: std::shared_ptr os_; std::string spillFile_; @@ -442,9 +438,30 @@ arrow::Status LocalPartitionWriter::stop(ShuffleWriterMetrics* metrics) { } stopped_ = true; - RETURN_NOT_OK(finishSpill()); + if (useSpillFileAsDataFile_) { + RETURN_NOT_OK(finishSpill(false)); + // The last spill has been written to data file. + auto spill = std::move(spills_.back()); + spills_.pop_back(); + + // Merge the remaining partitions from spills. + if (spills_.size() > 0) { + for (auto pid = lastEvictPid_ + 1; pid < numPartitions_; ++pid) { + auto bytesEvicted = totalBytesEvicted_; + RETURN_NOT_OK(mergeSpills(pid)); + partitionLengths_[pid] = totalBytesEvicted_ - bytesEvicted; + } + } - if (!useSpillFileAsDataFile_) { + for (auto pid = 0; pid < numPartitions_; ++pid) { + while (auto payload = spill->nextPayload(pid)) { + partitionLengths_[pid] += payload->rawSize(); + } + } + writeTime_ = spill->spillTime(); + compressTime_ += spill->compressTime(); + } else { + RETURN_NOT_OK(finishSpill(true)); // Open final data file. // If options_.bufferedWrite is set, it will acquire 16KB memory that can trigger spill. RETURN_NOT_OK(openDataFile()); @@ -473,33 +490,24 @@ arrow::Status LocalPartitionWriter::stop(ShuffleWriterMetrics* metrics) { ARROW_ASSIGN_OR_RAISE(endInFinalFile, dataFileOs_->Tell()); partitionLengths_[pid] = endInFinalFile - startInFinalFile; } - - for (const auto& spill : spills_) { - for (auto pid = 0; pid < numPartitions_; ++pid) { - if (spill->hasNextPayload(pid)) { - return arrow::Status::Invalid("Merging from spill is not exhausted."); - } - } - } - - ARROW_ASSIGN_OR_RAISE(totalBytesWritten_, dataFileOs_->Tell()); - - // Close Final file. Clear buffered resources. - RETURN_NOT_OK(clearResource()); - } else { - auto spill = std::move(spills_.back()); + } + // Close Final file. Clear buffered resources. + RETURN_NOT_OK(clearResource()); + // Check all spills are merged. + auto s = 0; + for (const auto& spill : spills_) { + compressTime_ += spill->compressTime(); + spillTime_ += spill->spillTime(); for (auto pid = 0; pid < numPartitions_; ++pid) { - uint64_t length = 0; - while (auto payload = spill->nextPayload(pid)) { - length += payload->rawSize(); + if (spill->hasNextPayload(pid)) { + return arrow::Status::Invalid( + "Merging from spill " + std::to_string(s) + " is not exhausted. pid: " + std::to_string(pid)); } - partitionLengths_[pid] = length; } - totalBytesWritten_ = std::filesystem::file_size(dataFile_); - writeTime_ = spillTime_; - spillTime_ = 0; - DLOG(INFO) << "Use spill file as data file: " << dataFile_; + ++s; } + spills_.clear(); + // Populate shuffle writer metrics. RETURN_NOT_OK(populateMetrics(metrics)); return arrow::Status::OK(); @@ -508,27 +516,29 @@ arrow::Status LocalPartitionWriter::stop(ShuffleWriterMetrics* metrics) { arrow::Status LocalPartitionWriter::requestSpill(bool isFinal) { if (!spiller_ || spiller_->finished()) { std::string spillFile; - if (isFinal && useSpillFileAsDataFile()) { + std::shared_ptr os; + if (isFinal) { + RETURN_NOT_OK(openDataFile()); spillFile = dataFile_; + os = dataFileOs_; + useSpillFileAsDataFile_ = true; } else { ARROW_ASSIGN_OR_RAISE(spillFile, createTempShuffleFile(nextSpilledFileDir())); + ARROW_ASSIGN_OR_RAISE(auto raw, arrow::io::FileOutputStream::Open(spillFile, true)); + ARROW_ASSIGN_OR_RAISE(os, arrow::io::BufferedOutputStream::Create(16384, pool_, raw)); } - ARROW_ASSIGN_OR_RAISE(auto raw, arrow::io::FileOutputStream::Open(spillFile, true)); - ARROW_ASSIGN_OR_RAISE(auto os, arrow::io::BufferedOutputStream::Create(16384, pool_, raw)); spiller_ = std::make_unique( os, std::move(spillFile), options_.compressionThreshold, payloadPool_.get(), codec_.get()); } return arrow::Status::OK(); } -arrow::Status LocalPartitionWriter::finishSpill() { +arrow::Status LocalPartitionWriter::finishSpill(bool close) { // Finish the spiller. No compression, no spill. if (spiller_ && !spiller_->finished()) { auto spiller = std::move(spiller_); spills_.emplace_back(); - ARROW_ASSIGN_OR_RAISE(spills_.back(), spiller->finish()); - spillTime_ += spiller->getSpillTime(); - compressTime_ += spiller->getCompressTime(); + ARROW_ASSIGN_OR_RAISE(spills_.back(), spiller->finish(close)); } return arrow::Status::OK(); } @@ -543,18 +553,29 @@ arrow::Status LocalPartitionWriter::evict( rawPartitionLengths_[partitionId] += inMemoryPayload->getBufferSize(); if (evictType == Evict::kSortSpill) { - if (partitionId < lastEvictPid_) { - RETURN_NOT_OK(finishSpill()); + if (lastEvictPid_ != -1 && (partitionId < lastEvictPid_ || (isFinal && !dataFileOs_))) { + lastEvictPid_ = -1; + RETURN_NOT_OK(finishSpill(true)); } - lastEvictPid_ = partitionId; - RETURN_NOT_OK(requestSpill(isFinal)); auto payloadType = codec_ ? Payload::Type::kCompressed : Payload::Type::kUncompressed; ARROW_ASSIGN_OR_RAISE( auto payload, inMemoryPayload->toBlockPayload(payloadType, payloadPool_.get(), codec_ ? codec_.get() : nullptr)); - RETURN_NOT_OK(spiller_->spill(partitionId, std::move(payload))); + if (!isFinal) { + RETURN_NOT_OK(spiller_->spill(partitionId, std::move(payload))); + } else { + if (spills_.size() > 0) { + for (auto pid = lastEvictPid_ + 1; pid <= partitionId; ++pid) { + auto bytesEvicted = totalBytesEvicted_; + RETURN_NOT_OK(mergeSpills(pid)); + partitionLengths_[pid] = totalBytesEvicted_ - bytesEvicted; + } + } + RETURN_NOT_OK(spiller_->spill(partitionId, std::move(payload))); + } + lastEvictPid_ = partitionId; return arrow::Status::OK(); } @@ -586,8 +607,8 @@ arrow::Status LocalPartitionWriter::evict( arrow::Status LocalPartitionWriter::evict(uint32_t partitionId, std::unique_ptr blockPayload, bool stop) { rawPartitionLengths_[partitionId] += blockPayload->rawSize(); - if (partitionId < lastEvictPid_) { - RETURN_NOT_OK(finishSpill()); + if (lastEvictPid_ != -1 && partitionId < lastEvictPid_) { + RETURN_NOT_OK(finishSpill(true)); } lastEvictPid_ = partitionId; @@ -598,7 +619,7 @@ arrow::Status LocalPartitionWriter::evict(uint32_t partitionId, std::unique_ptr< arrow::Status LocalPartitionWriter::reclaimFixedSize(int64_t size, int64_t* actual) { // Finish last spiller. - RETURN_NOT_OK(finishSpill()); + RETURN_NOT_OK(finishSpill(true)); int64_t reclaimed = 0; // Reclaim memory from payloadCache. @@ -629,7 +650,7 @@ arrow::Status LocalPartitionWriter::reclaimFixedSize(int64_t size, int64_t* actu // This is not accurate. When the evicted partition buffers are not copied, the merged ones // are resized from the original buffers thus allocated from partitionBufferPool. reclaimed += beforeSpill - payloadPool_->bytes_allocated(); - RETURN_NOT_OK(finishSpill()); + RETURN_NOT_OK(finishSpill(true)); } *actual = reclaimed; return arrow::Status::OK(); @@ -646,18 +667,9 @@ arrow::Status LocalPartitionWriter::populateMetrics(ShuffleWriterMetrics* metric metrics->totalEvictTime += spillTime_; metrics->totalWriteTime += writeTime_; metrics->totalBytesEvicted += totalBytesEvicted_; - metrics->totalBytesWritten += totalBytesWritten_; + metrics->totalBytesWritten += std::filesystem::file_size(dataFile_); metrics->partitionLengths = std::move(partitionLengths_); metrics->rawPartitionLengths = std::move(rawPartitionLengths_); return arrow::Status::OK(); } - -bool LocalPartitionWriter::useSpillFileAsDataFile() { - if (!payloadCache_ && !merger_ && !spiller_ && spills_.size() == 0) { - useSpillFileAsDataFile_ = true; - return true; - } - return false; -} - } // namespace gluten diff --git a/cpp/core/shuffle/LocalPartitionWriter.h b/cpp/core/shuffle/LocalPartitionWriter.h index a29f04fb748f9..efd7b4df3f4f0 100644 --- a/cpp/core/shuffle/LocalPartitionWriter.h +++ b/cpp/core/shuffle/LocalPartitionWriter.h @@ -83,7 +83,7 @@ class LocalPartitionWriter : public PartitionWriter { arrow::Status requestSpill(bool isFinal); - arrow::Status finishSpill(); + arrow::Status finishSpill(bool close); std::string nextSpilledFileDir(); @@ -95,8 +95,6 @@ class LocalPartitionWriter : public PartitionWriter { arrow::Status populateMetrics(ShuffleWriterMetrics* metrics); - bool useSpillFileAsDataFile(); - std::string dataFile_; std::vector localDirs_; @@ -113,10 +111,9 @@ class LocalPartitionWriter : public PartitionWriter { std::shared_ptr dataFileOs_; int64_t totalBytesEvicted_{0}; - int64_t totalBytesWritten_{0}; std::vector partitionLengths_; std::vector rawPartitionLengths_; - uint32_t lastEvictPid_{0}; + int32_t lastEvictPid_{-1}; }; } // namespace gluten diff --git a/cpp/core/shuffle/Spill.cc b/cpp/core/shuffle/Spill.cc index 0603b5edfd182..0bbe667ab4d87 100644 --- a/cpp/core/shuffle/Spill.cc +++ b/cpp/core/shuffle/Spill.cc @@ -86,7 +86,23 @@ void Spill::setSpillFile(const std::string& spillFile) { spillFile_ = spillFile; } +void Spill::setSpillTime(int64_t spillTime) { + spillTime_ = spillTime; +} + +void Spill::setCompressTime(int64_t compressTime) { + compressTime_ = compressTime; +} + std::string Spill::spillFile() const { return spillFile_; } + +int64_t Spill::spillTime() const { + return spillTime_; +} + +int64_t Spill::compressTime() const { + return compressTime_; +} } // namespace gluten diff --git a/cpp/core/shuffle/Spill.h b/cpp/core/shuffle/Spill.h index 71cb3d0515e1e..7ee60ef299fe5 100644 --- a/cpp/core/shuffle/Spill.h +++ b/cpp/core/shuffle/Spill.h @@ -52,8 +52,16 @@ class Spill final { void setSpillFile(const std::string& spillFile); + void setSpillTime(int64_t spillTime); + + void setCompressTime(int64_t compressTime); + std::string spillFile() const; + int64_t spillTime() const; + + int64_t compressTime() const; + private: struct PartitionPayload { uint32_t partitionId{}; @@ -65,6 +73,8 @@ class Spill final { std::list partitionPayloads_{}; std::shared_ptr inputStream_{}; std::string spillFile_; + int64_t spillTime_; + int64_t compressTime_; arrow::io::InputStream* rawIs_; diff --git a/cpp/velox/shuffle/VeloxSortShuffleWriter.cc b/cpp/velox/shuffle/VeloxSortShuffleWriter.cc index d7db69659d253..c0d9b467d98c8 100644 --- a/cpp/velox/shuffle/VeloxSortShuffleWriter.cc +++ b/cpp/velox/shuffle/VeloxSortShuffleWriter.cc @@ -205,7 +205,7 @@ void VeloxSortShuffleWriter::insertRows(facebook::velox::row::CompactRow& row, u } } -arrow::Status VeloxSortShuffleWriter::maybeSpill(int32_t nextRows) { +arrow::Status VeloxSortShuffleWriter::maybeSpill(uint32_t nextRows) { if ((uint64_t)offset_ + nextRows > std::numeric_limits::max()) { RETURN_NOT_OK(evictAllPartitions()); } @@ -213,9 +213,12 @@ arrow::Status VeloxSortShuffleWriter::maybeSpill(int32_t nextRows) { } arrow::Status VeloxSortShuffleWriter::evictAllPartitions() { + VELOX_CHECK(offset_ > 0); EvictGuard evictGuard{evictState_}; auto numRecords = offset_; + // offset_ is used for checking spillable data. + offset_ = 0; int32_t begin = 0; { ScopedTimer timer(&sortTime_); @@ -257,7 +260,6 @@ arrow::Status VeloxSortShuffleWriter::evictAllPartitions() { pageCursor_ = 0; // Reset and reallocate array_ to minimal size. Allocate array_ can trigger spill. - offset_ = 0; initArray(); } return arrow::Status::OK(); diff --git a/cpp/velox/shuffle/VeloxSortShuffleWriter.h b/cpp/velox/shuffle/VeloxSortShuffleWriter.h index 747593ae457d9..69b8b25030955 100644 --- a/cpp/velox/shuffle/VeloxSortShuffleWriter.h +++ b/cpp/velox/shuffle/VeloxSortShuffleWriter.h @@ -71,7 +71,7 @@ class VeloxSortShuffleWriter final : public VeloxShuffleWriter { void insertRows(facebook::velox::row::CompactRow& row, uint32_t offset, uint32_t rows); - arrow::Status maybeSpill(int32_t nextRows); + arrow::Status maybeSpill(uint32_t nextRows); arrow::Status evictAllPartitions(); From decd8b4cf3eb0612b50984c9c81ddf19c77d7d81 Mon Sep 17 00:00:00 2001 From: PHILO-HE Date: Wed, 31 Jul 2024 10:48:30 +0800 Subject: [PATCH 30/61] [GLUTEN-6612] Fix ParquetFileFormat issue caused by the setting of local property isNativeApplicable (#6627) --- .../delta/ClickhouseOptimisticTransaction.scala | 2 +- .../delta/ClickhouseOptimisticTransaction.scala | 2 +- .../delta/ClickhouseOptimisticTransaction.scala | 2 +- .../datasources/GlutenWriterColumnarRules.scala | 12 +++++------- .../execution/datasources/FileFormatWriter.scala | 6 +++--- .../datasources/orc/OrcFileFormat.scala | 4 ++-- .../datasources/parquet/ParquetFileFormat.scala | 16 ++++++---------- .../sql/hive/execution/HiveFileFormat.scala | 4 ++-- .../execution/datasources/FileFormatWriter.scala | 6 +++--- .../datasources/orc/OrcFileFormat.scala | 4 ++-- .../datasources/parquet/ParquetFileFormat.scala | 14 +++++--------- .../sql/hive/execution/HiveFileFormat.scala | 4 ++-- 12 files changed, 33 insertions(+), 43 deletions(-) diff --git a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala index 4133b5c605b84..3314465c5022d 100644 --- a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala +++ b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala @@ -177,7 +177,7 @@ class ClickhouseOptimisticTransaction( // 1. insert FakeRowAdaptor // 2. DeltaInvariantCheckerExec transform // 3. DeltaTaskStatisticsTracker collect null count / min values / max values - // 4. set the parameters 'staticPartitionWriteOnly', 'isNativeAppliable', + // 4. set the parameters 'staticPartitionWriteOnly', 'isNativeApplicable', // 'nativeFormat' in the LocalProperty of the sparkcontext super.writeFiles(inputData, writeOptions, additionalConstraints) } diff --git a/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala index 4133b5c605b84..3314465c5022d 100644 --- a/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala +++ b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala @@ -177,7 +177,7 @@ class ClickhouseOptimisticTransaction( // 1. insert FakeRowAdaptor // 2. DeltaInvariantCheckerExec transform // 3. DeltaTaskStatisticsTracker collect null count / min values / max values - // 4. set the parameters 'staticPartitionWriteOnly', 'isNativeAppliable', + // 4. set the parameters 'staticPartitionWriteOnly', 'isNativeApplicable', // 'nativeFormat' in the LocalProperty of the sparkcontext super.writeFiles(inputData, writeOptions, additionalConstraints) } diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala index 9e79c4f2e9845..6eec68efece3b 100644 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala @@ -185,7 +185,7 @@ class ClickhouseOptimisticTransaction( // 1. insert FakeRowAdaptor // 2. DeltaInvariantCheckerExec transform // 3. DeltaTaskStatisticsTracker collect null count / min values / max values - // 4. set the parameters 'staticPartitionWriteOnly', 'isNativeAppliable', + // 4. set the parameters 'staticPartitionWriteOnly', 'isNativeApplicable', // 'nativeFormat' in the LocalProperty of the sparkcontext super.writeFiles(inputData, writeOptions, additionalConstraints) } diff --git a/gluten-core/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRules.scala b/gluten-core/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRules.scala index 7063c3f67b800..20b00601531f9 100644 --- a/gluten-core/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRules.scala +++ b/gluten-core/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRules.scala @@ -163,6 +163,10 @@ object GlutenWriterColumnarRules { BackendsApiManager.getSettings.enableNativeWriteFiles() => injectFakeRowAdaptor(rc, rc.child) case rc @ DataWritingCommandExec(cmd, child) => + // These properties can be set by the same thread in last query submission. + session.sparkContext.setLocalProperty("isNativeApplicable", null) + session.sparkContext.setLocalProperty("nativeFormat", null) + session.sparkContext.setLocalProperty("staticPartitionWriteOnly", null) if (BackendsApiManager.getSettings.supportNativeWrite(child.output.toStructType.fields)) { val format = getNativeFormat(cmd) session.sparkContext.setLocalProperty( @@ -170,7 +174,7 @@ object GlutenWriterColumnarRules { BackendsApiManager.getSettings.staticPartitionWriteOnly().toString) // FIXME: We should only use context property if having no other approaches. // Should see if there is another way to pass these options. - session.sparkContext.setLocalProperty("isNativeAppliable", format.isDefined.toString) + session.sparkContext.setLocalProperty("isNativeApplicable", format.isDefined.toString) session.sparkContext.setLocalProperty("nativeFormat", format.getOrElse("")) if (format.isDefined) { injectFakeRowAdaptor(rc, child) @@ -178,12 +182,6 @@ object GlutenWriterColumnarRules { rc.withNewChildren(rc.children.map(apply)) } } else { - session.sparkContext.setLocalProperty( - "staticPartitionWriteOnly", - BackendsApiManager.getSettings.staticPartitionWriteOnly().toString) - session.sparkContext.setLocalProperty("isNativeAppliable", "false") - session.sparkContext.setLocalProperty("nativeFormat", "") - rc.withNewChildren(rc.children.map(apply)) } case plan: SparkPlan => plan.withNewChildren(plan.children.map(apply)) diff --git a/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala index a5c857103910e..96a044c0cbbe9 100644 --- a/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala +++ b/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -148,9 +148,9 @@ object FileFormatWriter extends Logging { numStaticPartitionCols: Int = 0): Set[String] = { val nativeEnabled = - "true".equals(sparkSession.sparkContext.getLocalProperty("isNativeAppliable")) + "true" == sparkSession.sparkContext.getLocalProperty("isNativeApplicable") val staticPartitionWriteOnly = - "true".equals(sparkSession.sparkContext.getLocalProperty("staticPartitionWriteOnly")) + "true" == sparkSession.sparkContext.getLocalProperty("staticPartitionWriteOnly") if (nativeEnabled) { logInfo("Use Gluten partition write for hive") @@ -257,7 +257,7 @@ object FileFormatWriter extends Logging { } val nativeFormat = sparkSession.sparkContext.getLocalProperty("nativeFormat") - if ("parquet".equals(nativeFormat)) { + if ("parquet" == nativeFormat) { (GlutenParquetWriterInjects.getInstance().executeWriterWrappedSparkPlan(wrapped), None) } else { (GlutenOrcWriterInjects.getInstance().executeWriterWrappedSparkPlan(wrapped), None) diff --git a/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala b/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala index 34873c46b09ea..619fa64ace6d8 100644 --- a/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala +++ b/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala @@ -83,7 +83,7 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable options: Map[String, String], files: Seq[FileStatus]): Option[StructType] = { // Why if (false)? Such code requires comments when being written. - if ("true".equals(sparkSession.sparkContext.getLocalProperty("isNativeAppliable")) && false) { + if ("true" == sparkSession.sparkContext.getLocalProperty("isNativeApplicable") && false) { GlutenOrcWriterInjects .getInstance() .inferSchema(sparkSession, Map.empty[String, String], files) @@ -109,7 +109,7 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable .asInstanceOf[JobConf] .setOutputFormat(classOf[org.apache.orc.mapred.OrcOutputFormat[OrcStruct]]) - if ("true".equals(sparkSession.sparkContext.getLocalProperty("isNativeAppliable"))) { + if ("true" == sparkSession.sparkContext.getLocalProperty("isNativeApplicable")) { // pass compression to job conf so that the file extension can be aware of it. val nativeConf = GlutenOrcWriterInjects diff --git a/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index c6b383136590b..42a63c7ebcd1d 100644 --- a/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -83,7 +83,7 @@ class ParquetFileFormat extends FileFormat with DataSourceRegister with Logging job: Job, options: Map[String, String], dataSchema: StructType): OutputWriterFactory = { - if ("true".equals(sparkSession.sparkContext.getLocalProperty("isNativeAppliable"))) { + if ("true" == sparkSession.sparkContext.getLocalProperty("isNativeApplicable")) { // pass compression to job conf so that the file extension can be aware of it. val conf = ContextUtil.getConfiguration(job) @@ -201,7 +201,7 @@ class ParquetFileFormat extends FileFormat with DataSourceRegister with Logging parameters: Map[String, String], files: Seq[FileStatus]): Option[StructType] = { // Why if (false)? Such code requires comments when being written. - if ("true".equals(sparkSession.sparkContext.getLocalProperty("isNativeAppliable")) && false) { + if ("true" == sparkSession.sparkContext.getLocalProperty("isNativeApplicable") && false) { GlutenParquetWriterInjects.getInstance().inferSchema(sparkSession, parameters, files) } else { // the vanilla spark case ParquetUtils.inferSchema(sparkSession, parameters, files) @@ -210,14 +210,10 @@ class ParquetFileFormat extends FileFormat with DataSourceRegister with Logging /** Returns whether the reader will return the rows as batch or not. */ override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = { - if ("true".equals(sparkSession.sparkContext.getLocalProperty("isNativeAppliable"))) { - true - } else { - val conf = sparkSession.sessionState.conf - conf.parquetVectorizedReaderEnabled && conf.wholeStageEnabled && - schema.length <= conf.wholeStageMaxNumFields && - schema.forall(_.dataType.isInstanceOf[AtomicType]) - } + val conf = sparkSession.sessionState.conf + conf.parquetVectorizedReaderEnabled && conf.wholeStageEnabled && + schema.length <= conf.wholeStageMaxNumFields && + schema.forall(_.dataType.isInstanceOf[AtomicType]) } override def vectorTypes( diff --git a/shims/spark32/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala b/shims/spark32/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala index 162dd342bcf01..eb0f6a5d97dff 100644 --- a/shims/spark32/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala +++ b/shims/spark32/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala @@ -100,9 +100,9 @@ class HiveFileFormat(fileSinkConf: FileSinkDesc) // Avoid referencing the outer object. val fileSinkConfSer = fileSinkConf val outputFormat = fileSinkConf.tableInfo.getOutputFileFormatClassName - if ("true".equals(sparkSession.sparkContext.getLocalProperty("isNativeAppliable"))) { + if ("true" == sparkSession.sparkContext.getLocalProperty("isNativeApplicable")) { val nativeFormat = sparkSession.sparkContext.getLocalProperty("nativeFormat") - val isParquetFormat = nativeFormat.equals("parquet") + val isParquetFormat = nativeFormat == "parquet" val compressionCodec = if (fileSinkConf.compressed) { // hive related configurations fileSinkConf.compressCodec diff --git a/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala index ebf45e76e74ec..f5e932337c024 100644 --- a/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala +++ b/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -140,9 +140,9 @@ object FileFormatWriter extends Logging { numStaticPartitionCols: Int = 0): Set[String] = { val nativeEnabled = - "true".equals(sparkSession.sparkContext.getLocalProperty("isNativeAppliable")) + "true" == sparkSession.sparkContext.getLocalProperty("isNativeApplicable") val staticPartitionWriteOnly = - "true".equals(sparkSession.sparkContext.getLocalProperty("staticPartitionWriteOnly")) + "true" == sparkSession.sparkContext.getLocalProperty("staticPartitionWriteOnly") if (nativeEnabled) { logInfo("Use Gluten partition write for hive") @@ -277,7 +277,7 @@ object FileFormatWriter extends Logging { } val nativeFormat = sparkSession.sparkContext.getLocalProperty("nativeFormat") - if ("parquet".equals(nativeFormat)) { + if ("parquet" == nativeFormat) { (GlutenParquetWriterInjects.getInstance().executeWriterWrappedSparkPlan(wrapped), None) } else { (GlutenOrcWriterInjects.getInstance().executeWriterWrappedSparkPlan(wrapped), None) diff --git a/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala b/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala index 49ac28d73322f..9891f6851d009 100644 --- a/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala +++ b/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala @@ -66,7 +66,7 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable options: Map[String, String], files: Seq[FileStatus]): Option[StructType] = { // Why if (false)? Such code requires comments when being written. - if ("true".equals(sparkSession.sparkContext.getLocalProperty("isNativeAppliable")) && false) { + if ("true" == sparkSession.sparkContext.getLocalProperty("isNativeApplicable") && false) { GlutenOrcWriterInjects.getInstance().inferSchema(sparkSession, options, files) } else { // the vanilla spark case OrcUtils.inferSchema(sparkSession, files, options) @@ -88,7 +88,7 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable .asInstanceOf[JobConf] .setOutputFormat(classOf[org.apache.orc.mapred.OrcOutputFormat[OrcStruct]]) - if ("true".equals(sparkSession.sparkContext.getLocalProperty("isNativeAppliable"))) { + if ("true" == sparkSession.sparkContext.getLocalProperty("isNativeApplicable")) { // pass compression to job conf so that the file extension can be aware of it. val nativeConf = GlutenOrcWriterInjects diff --git a/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index b0573f68e46d5..403e31c1cb302 100644 --- a/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -75,7 +75,7 @@ class ParquetFileFormat extends FileFormat with DataSourceRegister with Logging job: Job, options: Map[String, String], dataSchema: StructType): OutputWriterFactory = { - if ("true".equals(sparkSession.sparkContext.getLocalProperty("isNativeAppliable"))) { + if ("true" == sparkSession.sparkContext.getLocalProperty("isNativeApplicable")) { // pass compression to job conf so that the file extension can be aware of it. val conf = ContextUtil.getConfiguration(job) @@ -197,7 +197,7 @@ class ParquetFileFormat extends FileFormat with DataSourceRegister with Logging parameters: Map[String, String], files: Seq[FileStatus]): Option[StructType] = { // Why if (false)? Such code requires comments when being written. - if ("true".equals(sparkSession.sparkContext.getLocalProperty("isNativeAppliable")) && false) { + if ("true" == sparkSession.sparkContext.getLocalProperty("isNativeApplicable") && false) { GlutenParquetWriterInjects.getInstance().inferSchema(sparkSession, parameters, files) } else { // the vanilla spark case ParquetUtils.inferSchema(sparkSession, parameters, files) @@ -206,13 +206,9 @@ class ParquetFileFormat extends FileFormat with DataSourceRegister with Logging /** Returns whether the reader will return the rows as batch or not. */ override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = { - if ("true".equals(sparkSession.sparkContext.getLocalProperty("isNativeAppliable"))) { - true - } else { - val conf = sparkSession.sessionState.conf - ParquetUtils.isBatchReadSupportedForSchema(conf, schema) && conf.wholeStageEnabled && - !WholeStageCodegenExec.isTooManyFields(conf, schema) - } + val conf = sparkSession.sessionState.conf + ParquetUtils.isBatchReadSupportedForSchema(conf, schema) && conf.wholeStageEnabled && + !WholeStageCodegenExec.isTooManyFields(conf, schema) } override def vectorTypes( diff --git a/shims/spark33/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala b/shims/spark33/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala index 7a824c43670d5..b9c1622cbee5c 100644 --- a/shims/spark33/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala +++ b/shims/spark33/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala @@ -97,9 +97,9 @@ class HiveFileFormat(fileSinkConf: FileSinkDesc) // Avoid referencing the outer object. val fileSinkConfSer = fileSinkConf val outputFormat = fileSinkConf.tableInfo.getOutputFileFormatClassName - if ("true".equals(sparkSession.sparkContext.getLocalProperty("isNativeAppliable"))) { + if ("true" == sparkSession.sparkContext.getLocalProperty("isNativeApplicable")) { val nativeFormat = sparkSession.sparkContext.getLocalProperty("nativeFormat") - val isParquetFormat = nativeFormat.equals("parquet") + val isParquetFormat = nativeFormat == "parquet" val compressionCodec = if (fileSinkConf.compressed) { // hive related configurations fileSinkConf.compressCodec From cff5de5f4d90e4a2ec89c096cc1f0d1db95abc85 Mon Sep 17 00:00:00 2001 From: Mingliang Zhu Date: Wed, 31 Jul 2024 11:44:39 +0800 Subject: [PATCH 31/61] [VL] Support Sum(Literal)/Count(Literal) with empty input schema (#6631) --- .../org/apache/gluten/backendsapi/velox/VeloxBackend.scala | 6 +++--- .../gluten/execution/ScalarFunctionsValidateSuite.scala | 6 ++++++ 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala index 990991c71660f..0eb6126876b55 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala @@ -438,13 +438,13 @@ object VeloxBackendSettings extends BackendSettingsApi { plan match { case exec: HashAggregateExec if exec.aggregateExpressions.nonEmpty => - // Check Sum(1) or Count(1). + // Check Sum(Literal) or Count(Literal). exec.aggregateExpressions.forall( expression => { val aggFunction = expression.aggregateFunction aggFunction match { - case _: Sum | _: Count => - aggFunction.children.size == 1 && aggFunction.children.head.equals(Literal(1)) + case Sum(Literal(_, _), _) => true + case Count(Seq(Literal(_, _))) => true case _ => false } }) diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/ScalarFunctionsValidateSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/ScalarFunctionsValidateSuite.scala index 13ade14b59438..897c1c5f58d5e 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/ScalarFunctionsValidateSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/ScalarFunctionsValidateSuite.scala @@ -754,6 +754,12 @@ abstract class ScalarFunctionsValidateSuite extends FunctionsValidateTest { } } + test("Test sum/count function") { + runQueryAndCompare("""SELECT sum(2),count(2) from lineitem""".stripMargin) { + checkGlutenOperatorMatch[BatchScanExecTransformer] + } + } + test("Test spark_partition_id function") { runQueryAndCompare("""SELECT spark_partition_id(), l_orderkey | from lineitem limit 100""".stripMargin) { From e8583c31d5abd2a02858868482686cd099907716 Mon Sep 17 00:00:00 2001 From: Chang chen Date: Wed, 31 Jul 2024 13:48:13 +0800 Subject: [PATCH 32/61] [GLUTEN-6067][CH][MINOR][UT] Pass backends-clickhouse ut in Spark 3.5 (#6623) * 1. move tpch parquet ut to tpch package 2. move tpcds ut to tpcds package 3. pass ut in spark 3.5 * we don't support 3.4 1. isSparkVersionGE("3.5") 2. isSparkVersionLE("3.3") --- .../GlutenClickHouseDecimalSuite.scala | 25 +++---- .../GlutenClickHouseHiveTableSuite.scala | 8 ++- ...lutenClickHouseNativeWriteTableSuite.scala | 4 +- .../GlutenClickHouseTPCDSAbstractSuite.scala | 55 ++++++++-------- .../GlutenClickHouseTPCHBucketSuite.scala | 34 +++++----- ...ClickHouseWholeStageTransformerSuite.scala | 21 ++++-- .../GlutenClickhouseCountDistinctSuite.scala | 4 +- .../GlutenClickHouseTPCHMetricsSuite.scala | 22 +++---- .../parquet/GlutenParquetFilterSuite.scala | 44 ++++++++----- ...GlutenClickHouseTPCDSParquetAQESuite.scala | 30 +++++---- ...eTPCDSParquetColumnarShuffleAQESuite.scala | 28 ++++---- ...ouseTPCDSParquetColumnarShuffleSuite.scala | 26 ++++---- ...kHouseTPCDSParquetGraceHashJoinSuite.scala | 14 ++-- .../GlutenClickHouseTPCDSParquetRFSuite.scala | 4 +- ...kHouseTPCDSParquetSortMergeJoinSuite.scala | 21 +++--- .../GlutenClickHouseTPCDSParquetSuite.scala | 52 ++++++++------- ...tenClickHouseDatetimeExpressionSuite.scala | 4 +- ...seTPCHColumnarShuffleParquetAQESuite.scala | 19 +++--- ...ckHouseTPCHParquetAQEConcurrentSuite.scala | 4 +- .../GlutenClickHouseTPCHParquetAQESuite.scala | 13 ++-- ...utenClickHouseTPCHParquetBucketSuite.scala | 54 +++++++-------- .../GlutenClickHouseTPCHParquetRFSuite.scala | 9 ++- ...enClickHouseTPCHSaltNullParquetSuite.scala | 66 +++++++++---------- .../spark/gluten/NativeWriteChecker.scala | 2 +- 24 files changed, 298 insertions(+), 265 deletions(-) rename backends-clickhouse/src/test/scala/org/apache/gluten/execution/{ => tpcds}/GlutenClickHouseTPCDSParquetAQESuite.scala (90%) rename backends-clickhouse/src/test/scala/org/apache/gluten/execution/{ => tpcds}/GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite.scala (93%) rename backends-clickhouse/src/test/scala/org/apache/gluten/execution/{ => tpcds}/GlutenClickHouseTPCDSParquetColumnarShuffleSuite.scala (91%) rename backends-clickhouse/src/test/scala/org/apache/gluten/execution/{ => tpcds}/GlutenClickHouseTPCDSParquetGraceHashJoinSuite.scala (93%) rename backends-clickhouse/src/test/scala/org/apache/gluten/execution/{ => tpcds}/GlutenClickHouseTPCDSParquetRFSuite.scala (93%) rename backends-clickhouse/src/test/scala/org/apache/gluten/execution/{ => tpcds}/GlutenClickHouseTPCDSParquetSortMergeJoinSuite.scala (93%) rename backends-clickhouse/src/test/scala/org/apache/gluten/execution/{ => tpcds}/GlutenClickHouseTPCDSParquetSuite.scala (88%) rename backends-clickhouse/src/test/scala/org/apache/gluten/execution/{ => tpch}/GlutenClickHouseDatetimeExpressionSuite.scala (98%) rename backends-clickhouse/src/test/scala/org/apache/gluten/execution/{ => tpch}/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala (94%) rename backends-clickhouse/src/test/scala/org/apache/gluten/execution/{ => tpch}/GlutenClickHouseTPCHParquetAQEConcurrentSuite.scala (96%) rename backends-clickhouse/src/test/scala/org/apache/gluten/execution/{ => tpch}/GlutenClickHouseTPCHParquetAQESuite.scala (98%) rename backends-clickhouse/src/test/scala/org/apache/gluten/execution/{ => tpch}/GlutenClickHouseTPCHParquetBucketSuite.scala (95%) rename backends-clickhouse/src/test/scala/org/apache/gluten/execution/{ => tpch}/GlutenClickHouseTPCHParquetRFSuite.scala (91%) rename backends-clickhouse/src/test/scala/org/apache/gluten/execution/{ => tpch}/GlutenClickHouseTPCHSaltNullParquetSuite.scala (98%) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDecimalSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDecimalSuite.scala index 7320b7c051522..cf1bdd296c013 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDecimalSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDecimalSuite.scala @@ -66,10 +66,10 @@ class GlutenClickHouseDecimalSuite private val decimalTable: String = "decimal_table" private val decimalTPCHTables: Seq[(DecimalType, Seq[Int])] = Seq.apply( (DecimalType.apply(9, 4), Seq()), - // 1: ch decimal avg is float (DecimalType.apply(18, 8), Seq()), - // 1: ch decimal avg is float, 3/10: all value is null and compare with limit - (DecimalType.apply(38, 19), Seq(3, 10)) + // 3/10: all value is null and compare with limit + // 1 Spark 3.5 + (DecimalType.apply(38, 19), if (isSparkVersionLE("3.3")) Seq(3, 10) else Seq(1, 3, 10)) ) private def createDecimalTables(dataType: DecimalType): Unit = { @@ -343,19 +343,14 @@ class GlutenClickHouseDecimalSuite decimalTPCHTables.foreach { dt => { + val fallBack = (sql_num == 16 || sql_num == 21) + val compareResult = !dt._2.contains(sql_num) + val native = if (fallBack) "fallback" else "native" + val compare = if (compareResult) "compare" else "noCompare" + val PrecisionLoss = s"allowPrecisionLoss=$allowPrecisionLoss" val decimalType = dt._1 test(s"""TPCH Decimal(${decimalType.precision},${decimalType.scale}) - | Q$sql_num[allowPrecisionLoss=$allowPrecisionLoss]""".stripMargin) { - var noFallBack = true - var compareResult = true - if (sql_num == 16 || sql_num == 21) { - noFallBack = false - } - - if (dt._2.contains(sql_num)) { - compareResult = false - } - + | Q$sql_num[$PrecisionLoss,$native,$compare]""".stripMargin) { spark.sql(s"use decimal_${decimalType.precision}_${decimalType.scale}") withSQLConf( (SQLConf.DECIMAL_OPERATIONS_ALLOW_PREC_LOSS.key, allowPrecisionLoss)) { @@ -363,7 +358,7 @@ class GlutenClickHouseDecimalSuite sql_num, tpchQueries, compareResult = compareResult, - noFallBack = noFallBack) { _ => {} } + noFallBack = !fallBack) { _ => {} } } spark.sql(s"use default") } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseHiveTableSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseHiveTableSuite.scala index 4e190c087920b..8599b3002e3af 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseHiveTableSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseHiveTableSuite.scala @@ -1051,8 +1051,12 @@ class GlutenClickHouseHiveTableSuite spark.sql( s"CREATE FUNCTION my_add as " + s"'org.apache.hadoop.hive.contrib.udf.example.UDFExampleAdd2' USING JAR '$jarUrl'") - runQueryAndCompare("select MY_ADD(id, id+1) from range(10)")( - checkGlutenOperatorMatch[ProjectExecTransformer]) + if (isSparkVersionLE("3.3")) { + runQueryAndCompare("select MY_ADD(id, id+1) from range(10)")( + checkGlutenOperatorMatch[ProjectExecTransformer]) + } else { + runQueryAndCompare("select MY_ADD(id, id+1) from range(10)", noFallBack = false)(_ => {}) + } } test("GLUTEN-4333: fix CSE in aggregate operator") { diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeWriteTableSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeWriteTableSuite.scala index 578c432927477..0f642dfa86647 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeWriteTableSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeWriteTableSuite.scala @@ -603,7 +603,7 @@ class GlutenClickHouseNativeWriteTableSuite ("timestamp_field", "timestamp") ) def excludeTimeFieldForORC(format: String): Seq[String] = { - if (format.equals("orc") && isSparkVersionGE("3.4")) { + if (format.equals("orc") && isSparkVersionGE("3.5")) { // FIXME:https://github.com/apache/incubator-gluten/pull/6507 fields.keys.filterNot(_.equals("timestamp_field")).toSeq } else { @@ -913,7 +913,7 @@ class GlutenClickHouseNativeWriteTableSuite (table_name, create_sql, insert_sql) }, (table_name, _) => - if (isSparkVersionGE("3.4")) { + if (isSparkVersionGE("3.5")) { compareResultsAgainstVanillaSpark( s"select * from $table_name", compareResult = true, diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSAbstractSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSAbstractSuite.scala index 9787182ed93f5..6ca587bebc283 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSAbstractSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSAbstractSuite.scala @@ -18,7 +18,7 @@ package org.apache.gluten.execution import org.apache.gluten.GlutenConfig import org.apache.gluten.benchmarks.GenTPCDSTableScripts -import org.apache.gluten.utils.UTSystemParameters +import org.apache.gluten.utils.{Arm, UTSystemParameters} import org.apache.spark.SparkConf import org.apache.spark.internal.Logging @@ -46,8 +46,8 @@ abstract class GlutenClickHouseTPCDSAbstractSuite rootPath + "../../../../gluten-core/src/test/resources/tpcds-queries/tpcds.queries.original" protected val queriesResults: String = rootPath + "tpcds-decimal-queries-output" - /** Return values: (sql num, is fall back, skip fall back assert) */ - def tpcdsAllQueries(isAqe: Boolean): Seq[(String, Boolean, Boolean)] = + /** Return values: (sql num, is fall back) */ + def tpcdsAllQueries(isAqe: Boolean): Seq[(String, Boolean)] = Range .inclusive(1, 99) .flatMap( @@ -57,25 +57,24 @@ abstract class GlutenClickHouseTPCDSAbstractSuite } else { Seq("q" + "%d".format(queryNum)) } - val noFallBack = queryNum match { - case i if !isAqe && (i == 10 || i == 16 || i == 35 || i == 94) => - // q10 smj + existence join - // q16 smj + left semi + not condition - // q35 smj + existence join - // Q94 BroadcastHashJoin, LeftSemi, NOT condition - (false, false) - case i if isAqe && (i == 16 || i == 94) => - (false, false) - case other => (true, false) - } - sqlNums.map((_, noFallBack._1, noFallBack._2)) + val native = !fallbackSets(isAqe).contains(queryNum) + sqlNums.map((_, native)) }) - // FIXME "q17", stddev_samp inconsistent results, CH return NaN, Spark return null + protected def fallbackSets(isAqe: Boolean): Set[Int] = { + val more = if (isSparkVersionGE("3.5")) Set(44, 67, 70) else Set.empty[Int] + + // q16 smj + left semi + not condition + // Q94 BroadcastHashJoin, LeftSemi, NOT condition + if (isAqe) { + Set(16, 94) | more + } else { + // q10, q35 smj + existence join + Set(10, 16, 35, 94) | more + } + } protected def excludedTpcdsQueries: Set[String] = Set( - "q61", // inconsistent results - "q66", // inconsistent results - "q67" // inconsistent results + "q66" // inconsistent results ) def executeTPCDSTest(isAqe: Boolean): Unit = { @@ -83,11 +82,12 @@ abstract class GlutenClickHouseTPCDSAbstractSuite s => if (excludedTpcdsQueries.contains(s._1)) { ignore(s"TPCDS ${s._1.toUpperCase()}") { - runTPCDSQuery(s._1, noFallBack = s._2, skipFallBackAssert = s._3) { df => } + runTPCDSQuery(s._1, noFallBack = s._2) { df => } } } else { - test(s"TPCDS ${s._1.toUpperCase()}") { - runTPCDSQuery(s._1, noFallBack = s._2, skipFallBackAssert = s._3) { df => } + val tag = if (s._2) "Native" else "Fallback" + test(s"TPCDS[$tag] ${s._1.toUpperCase()}") { + runTPCDSQuery(s._1, noFallBack = s._2) { df => } } }) } @@ -152,7 +152,7 @@ abstract class GlutenClickHouseTPCDSAbstractSuite } override protected def afterAll(): Unit = { - ClickhouseSnapshot.clearAllFileStatusCache + ClickhouseSnapshot.clearAllFileStatusCache() DeltaLog.clearCache() try { @@ -183,11 +183,10 @@ abstract class GlutenClickHouseTPCDSAbstractSuite tpcdsQueries: String = tpcdsQueries, queriesResults: String = queriesResults, compareResult: Boolean = true, - noFallBack: Boolean = true, - skipFallBackAssert: Boolean = false)(customCheck: DataFrame => Unit): Unit = { + noFallBack: Boolean = true)(customCheck: DataFrame => Unit): Unit = { val sqlFile = tpcdsQueries + "/" + queryNum + ".sql" - val sql = Source.fromFile(new File(sqlFile), "UTF-8").mkString + val sql = Arm.withResource(Source.fromFile(new File(sqlFile), "UTF-8"))(_.mkString) val df = spark.sql(sql) if (compareResult) { @@ -212,13 +211,13 @@ abstract class GlutenClickHouseTPCDSAbstractSuite // using WARN to guarantee printed log.warn(s"query: $queryNum, finish comparing with saved result") } else { - val start = System.currentTimeMillis(); + val start = System.currentTimeMillis() val ret = df.collect() // using WARN to guarantee printed log.warn(s"query: $queryNum skipped comparing, time cost to collect: ${System .currentTimeMillis() - start} ms, ret size: ${ret.length}") } - WholeStageTransformerSuite.checkFallBack(df, noFallBack, skipFallBackAssert) + WholeStageTransformerSuite.checkFallBack(df, noFallBack) customCheck(df) } } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHBucketSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHBucketSuite.scala index 59912e72222aa..e05cf7274feff 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHBucketSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHBucketSuite.scala @@ -234,10 +234,10 @@ class GlutenClickHouseTPCHBucketSuite val plans = collect(df.queryExecution.executedPlan) { case scanExec: BasicScanExecTransformer => scanExec } - assert(!(plans(0).asInstanceOf[FileSourceScanExecTransformer].bucketedScan)) - assert(plans(0).metrics("numFiles").value === 2) - assert(plans(0).metrics("pruningTime").value === -1) - assert(plans(0).metrics("numOutputRows").value === 591673) + assert(!plans.head.asInstanceOf[FileSourceScanExecTransformer].bucketedScan) + assert(plans.head.metrics("numFiles").value === 2) + assert(plans.head.metrics("pruningTime").value === pruningTimeValueSpark) + assert(plans.head.metrics("numOutputRows").value === 591673) }) } @@ -291,7 +291,7 @@ class GlutenClickHouseTPCHBucketSuite } if (sparkVersion.equals("3.2")) { - assert(!(plans(11).asInstanceOf[FileSourceScanExecTransformer].bucketedScan)) + assert(!plans(11).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) } else { assert(plans(11).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) } @@ -327,14 +327,14 @@ class GlutenClickHouseTPCHBucketSuite .isInstanceOf[InputIteratorTransformer]) if (sparkVersion.equals("3.2")) { - assert(!(plans(2).asInstanceOf[FileSourceScanExecTransformer].bucketedScan)) + assert(!plans(2).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) } else { assert(plans(2).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) } assert(plans(2).metrics("numFiles").value === 2) assert(plans(2).metrics("numOutputRows").value === 3111) - assert(!(plans(3).asInstanceOf[FileSourceScanExecTransformer].bucketedScan)) + assert(!plans(3).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) assert(plans(3).metrics("numFiles").value === 2) assert(plans(3).metrics("numOutputRows").value === 72678) }) @@ -366,12 +366,12 @@ class GlutenClickHouseTPCHBucketSuite } // bucket join assert( - plans(0) + plans.head .asInstanceOf[HashJoinLikeExecTransformer] .left .isInstanceOf[ProjectExecTransformer]) assert( - plans(0) + plans.head .asInstanceOf[HashJoinLikeExecTransformer] .right .isInstanceOf[ProjectExecTransformer]) @@ -409,10 +409,10 @@ class GlutenClickHouseTPCHBucketSuite val plans = collect(df.queryExecution.executedPlan) { case scanExec: BasicScanExecTransformer => scanExec } - assert(!(plans(0).asInstanceOf[FileSourceScanExecTransformer].bucketedScan)) - assert(plans(0).metrics("numFiles").value === 2) - assert(plans(0).metrics("pruningTime").value === -1) - assert(plans(0).metrics("numOutputRows").value === 11618) + assert(!plans.head.asInstanceOf[FileSourceScanExecTransformer].bucketedScan) + assert(plans.head.metrics("numFiles").value === 2) + assert(plans.head.metrics("pruningTime").value === pruningTimeValueSpark) + assert(plans.head.metrics("numOutputRows").value === 11618) }) } @@ -425,12 +425,12 @@ class GlutenClickHouseTPCHBucketSuite } // bucket join assert( - plans(0) + plans.head .asInstanceOf[HashJoinLikeExecTransformer] .left .isInstanceOf[FilterExecTransformerBase]) assert( - plans(0) + plans.head .asInstanceOf[HashJoinLikeExecTransformer] .right .isInstanceOf[ProjectExecTransformer]) @@ -585,7 +585,7 @@ class GlutenClickHouseTPCHBucketSuite def checkResult(df: DataFrame, exceptedResult: Seq[Row]): Unit = { // check the result val result = df.collect() - assert(result.size == exceptedResult.size) + assert(result.length == exceptedResult.size) val sortedRes = result.map { s => Row.fromSeq(s.toSeq.map { @@ -786,7 +786,7 @@ class GlutenClickHouseTPCHBucketSuite |order by l_orderkey, l_returnflag, t |limit 10 |""".stripMargin - runSql(SQL7, false)( + runSql(SQL7, noFallBack = false)( df => { checkResult( df, diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala index 9412326ae342c..4972861152fde 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala @@ -23,6 +23,7 @@ import org.apache.spark.{SPARK_VERSION_SHORT, SparkConf} import org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseConfig import org.apache.commons.io.FileUtils +import org.scalatest.Tag import java.io.File @@ -177,13 +178,23 @@ class GlutenClickHouseWholeStageTransformerSuite extends WholeStageTransformerSu super.beforeAll() } - protected val rootPath = this.getClass.getResource("/").getPath - protected val basePath = rootPath + "tests-working-home" - protected val warehouse = basePath + "/spark-warehouse" - protected val metaStorePathAbsolute = basePath + "/meta" - protected val hiveMetaStoreDB = metaStorePathAbsolute + "/metastore_db" + protected val rootPath: String = this.getClass.getResource("/").getPath + protected val basePath: String = rootPath + "tests-working-home" + protected val warehouse: String = basePath + "/spark-warehouse" + protected val metaStorePathAbsolute: String = basePath + "/meta" + protected val hiveMetaStoreDB: String = metaStorePathAbsolute + "/metastore_db" final override protected val resourcePath: String = "" // ch not need this override protected val fileFormat: String = "parquet" + + protected def testSparkVersionLE33(testName: String, testTag: Tag*)(testFun: => Any): Unit = { + if (isSparkVersionLE("3.3")) { + test(testName, testTag: _*)(testFun) + } else { + ignore(s"[$SPARK_VERSION_SHORT]-$testName", testTag: _*)(testFun) + } + } + + lazy val pruningTimeValueSpark: Int = if (isSparkVersionLE("3.3")) -1 else 0 } // scalastyle:off line.size.limit diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickhouseCountDistinctSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickhouseCountDistinctSuite.scala index 5887050d0aaa7..28ff5874fabd1 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickhouseCountDistinctSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickhouseCountDistinctSuite.scala @@ -105,9 +105,9 @@ class GlutenClickhouseCountDistinctSuite extends GlutenClickHouseWholeStageTrans val sql = s""" select count(distinct(a,b)) , try_add(c,b) from values (0, null,1), (0,null,2), (1, 1,4) as data(a,b,c) group by try_add(c,b) - """; + """ val df = spark.sql(sql) - WholeStageTransformerSuite.checkFallBack(df, noFallback = false) + WholeStageTransformerSuite.checkFallBack(df, noFallback = isSparkVersionGE("3.5")) } test("check count distinct with filter") { diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCHMetricsSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCHMetricsSuite.scala index 4b5a5b328cb3f..509967125a647 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCHMetricsSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCHMetricsSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.util.TaskResources import scala.collection.JavaConverters._ class GlutenClickHouseTPCHMetricsSuite extends GlutenClickHouseTPCHAbstractSuite { - private val parquetMaxBlockSize = 4096; + private val parquetMaxBlockSize = 4096 override protected val needCopyParquetToTablePath = true override protected val tablesPath: String = basePath + "/tpch-data" @@ -71,15 +71,15 @@ class GlutenClickHouseTPCHMetricsSuite extends GlutenClickHouseTPCHAbstractSuite assert(plans.size == 3) assert(plans(2).metrics("numFiles").value === 1) - assert(plans(2).metrics("pruningTime").value === -1) + assert(plans(2).metrics("pruningTime").value === pruningTimeValueSpark) assert(plans(2).metrics("filesSize").value === 19230111) assert(plans(1).metrics("numOutputRows").value === 4) assert(plans(1).metrics("outputVectors").value === 1) // Execute Sort operator, it will read the data twice. - assert(plans(0).metrics("numOutputRows").value === 4) - assert(plans(0).metrics("outputVectors").value === 1) + assert(plans.head.metrics("numOutputRows").value === 4) + assert(plans.head.metrics("outputVectors").value === 1) } } @@ -139,15 +139,15 @@ class GlutenClickHouseTPCHMetricsSuite extends GlutenClickHouseTPCHAbstractSuite assert(plans.size == 3) assert(plans(2).metrics("numFiles").value === 1) - assert(plans(2).metrics("pruningTime").value === -1) + assert(plans(2).metrics("pruningTime").value === pruningTimeValueSpark) assert(plans(2).metrics("filesSize").value === 19230111) assert(plans(1).metrics("numOutputRows").value === 4) assert(plans(1).metrics("outputVectors").value === 1) // Execute Sort operator, it will read the data twice. - assert(plans(0).metrics("numOutputRows").value === 4) - assert(plans(0).metrics("outputVectors").value === 1) + assert(plans.head.metrics("numOutputRows").value === 4) + assert(plans.head.metrics("outputVectors").value === 1) } } } @@ -165,7 +165,7 @@ class GlutenClickHouseTPCHMetricsSuite extends GlutenClickHouseTPCHAbstractSuite ) assert(nativeMetricsList.size == 1) - val nativeMetricsData = nativeMetricsList(0) + val nativeMetricsData = nativeMetricsList.head assert(nativeMetricsData.metricsDataList.size() == 3) assert(nativeMetricsData.metricsDataList.get(0).getName.equals("kRead")) @@ -287,7 +287,7 @@ class GlutenClickHouseTPCHMetricsSuite extends GlutenClickHouseTPCHAbstractSuite assert(joinPlan.metrics("inputBytes").value == 1920000) } - val wholeStageTransformer2 = allWholeStageTransformers(0) + val wholeStageTransformer2 = allWholeStageTransformers.head GlutenClickHouseMetricsUTUtils.executeMetricsUpdater( wholeStageTransformer2, @@ -325,7 +325,7 @@ class GlutenClickHouseTPCHMetricsSuite extends GlutenClickHouseTPCHAbstractSuite ) assert(nativeMetricsList.size == 1) - val nativeMetricsData = nativeMetricsList(0) + val nativeMetricsData = nativeMetricsList.head assert(nativeMetricsData.metricsDataList.size() == 5) assert(nativeMetricsData.metricsDataList.get(0).getName.equals("kRead")) @@ -399,7 +399,7 @@ class GlutenClickHouseTPCHMetricsSuite extends GlutenClickHouseTPCHAbstractSuite ) assert(nativeMetricsListFinal.size == 1) - val nativeMetricsDataFinal = nativeMetricsListFinal(0) + val nativeMetricsDataFinal = nativeMetricsListFinal.head assert(nativeMetricsDataFinal.metricsDataList.size() == 3) assert(nativeMetricsDataFinal.metricsDataList.get(0).getName.equals("kRead")) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/parquet/GlutenParquetFilterSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/parquet/GlutenParquetFilterSuite.scala index a1b5801daddf2..b4e4cea9173b6 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/parquet/GlutenParquetFilterSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/parquet/GlutenParquetFilterSuite.scala @@ -460,26 +460,34 @@ class GlutenParquetFilterSuite "orders1" -> Nil) ) + def runTest(i: Int): Unit = withDataFrame(tpchSQL(i + 1, tpchQueriesResourceFolder)) { + df => + val scans = df.queryExecution.executedPlan + .collect { case scan: FileSourceScanExecTransformer => scan } + assertResult(result(i).size)(scans.size) + scans.zipWithIndex + .foreach { + case (scan, fileIndex) => + val tableName = scan.tableIdentifier + .map(_.table) + .getOrElse(scan.relation.options("path").split("/").last) + val predicates = scan.filterExprs() + val expected = result(i)(s"$tableName$fileIndex") + assertResult(expected.size)(predicates.size) + if (expected.isEmpty) assert(predicates.isEmpty) + else compareExpressions(expected.reduceLeft(And), predicates.reduceLeft(And)) + } + } + tpchQueries.zipWithIndex.foreach { case (q, i) => - test(q) { - withDataFrame(tpchSQL(i + 1, tpchQueriesResourceFolder)) { - df => - val scans = df.queryExecution.executedPlan - .collect { case scan: FileSourceScanExecTransformer => scan } - assertResult(result(i).size)(scans.size) - scans.zipWithIndex - .foreach { - case (scan, fileIndex) => - val tableName = scan.tableIdentifier - .map(_.table) - .getOrElse(scan.relation.options("path").split("/").last) - val predicates = scan.filterExprs() - val expected = result(i)(s"$tableName$fileIndex") - assertResult(expected.size)(predicates.size) - if (expected.isEmpty) assert(predicates.isEmpty) - else compareExpressions(expected.reduceLeft(And), predicates.reduceLeft(And)) - } + if (q == "q2" || q == "q9") { + testSparkVersionLE33(q) { + runTest(i) + } + } else { + test(q) { + runTest(i) } } } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSParquetAQESuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetAQESuite.scala similarity index 90% rename from backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSParquetAQESuite.scala rename to backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetAQESuite.scala index 1960e3002a8ba..389d617f10eb0 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSParquetAQESuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetAQESuite.scala @@ -14,7 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.execution +package org.apache.gluten.execution.tpcds + +import org.apache.gluten.execution._ import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.expressions.DynamicPruningExpression @@ -46,7 +48,7 @@ class GlutenClickHouseTPCDSParquetAQESuite val result = runSql(""" |select count(c_customer_sk) from customer |""".stripMargin) { _ => } - assert(result(0).getLong(0) == 100000L) + assertResult(100000L)(result.head.getLong(0)) } test("test reading from partitioned table") { @@ -55,7 +57,7 @@ class GlutenClickHouseTPCDSParquetAQESuite | from store_sales | where ss_quantity between 1 and 20 |""".stripMargin) { _ => } - assert(result(0).getLong(0) == 550458L) + assertResult(550458L)(result.head.getLong(0)) } test("test reading from partitioned table with partition column filter") { @@ -66,7 +68,7 @@ class GlutenClickHouseTPCDSParquetAQESuite | where ss_quantity between 1 and 20 | and ss_sold_date_sk = 2452635 |""".stripMargin, - true, + compareResult = true, _ => {} ) } @@ -76,8 +78,8 @@ class GlutenClickHouseTPCDSParquetAQESuite |select avg(cs_item_sk), avg(cs_order_number) | from catalog_sales |""".stripMargin) { _ => } - assert(result(0).getDouble(0) == 8998.463336886734) - assert(result(0).getDouble(1) == 80037.12727449503) + assertResult(8998.463336886734)(result.head.getDouble(0)) + assertResult(80037.12727449503)(result.head.getDouble(1)) } test("Gluten-1235: Fix missing reading from the broadcasted value when executing DPP") { @@ -96,7 +98,7 @@ class GlutenClickHouseTPCDSParquetAQESuite |""".stripMargin compareResultsAgainstVanillaSpark( testSql, - true, + compareResult = true, df => { val foundDynamicPruningExpr = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -107,11 +109,11 @@ class GlutenClickHouseTPCDSParquetAQESuite .asInstanceOf[FileSourceScanExecTransformer] .partitionFilters .exists(_.isInstanceOf[DynamicPruningExpression])) - assert( + assertResult(1823)( foundDynamicPruningExpr(1) .asInstanceOf[FileSourceScanExecTransformer] .selectedPartitions - .size == 1823) + .length) } ) } @@ -126,7 +128,7 @@ class GlutenClickHouseTPCDSParquetAQESuite } // On Spark 3.2, there are 15 AdaptiveSparkPlanExec, // and on Spark 3.3, there are 5 AdaptiveSparkPlanExec and 10 ReusedSubqueryExec - assert(subqueryAdaptiveSparkPlan.filter(_ == true).size == 15) + assertResult(15)(subqueryAdaptiveSparkPlan.count(_ == true)) } } @@ -141,12 +143,12 @@ class GlutenClickHouseTPCDSParquetAQESuite } => f } - assert(foundDynamicPruningExpr.nonEmpty == true) + assert(foundDynamicPruningExpr.nonEmpty) val reusedExchangeExec = collectWithSubqueries(df.queryExecution.executedPlan) { case r: ReusedExchangeExec => r } - assert(reusedExchangeExec.nonEmpty == true) + assert(reusedExchangeExec.nonEmpty) } } @@ -164,7 +166,7 @@ class GlutenClickHouseTPCDSParquetAQESuite } => f } - assert(foundDynamicPruningExpr.nonEmpty == true) + assert(foundDynamicPruningExpr.nonEmpty) val reusedExchangeExec = collectWithSubqueries(df.queryExecution.executedPlan) { case r: ReusedExchangeExec => r @@ -194,6 +196,6 @@ class GlutenClickHouseTPCDSParquetAQESuite |ORDER BY channel | LIMIT 100 ; |""".stripMargin - compareResultsAgainstVanillaSpark(testSql, true, df => {}) + compareResultsAgainstVanillaSpark(testSql, compareResult = true, df => {}) } } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite.scala similarity index 93% rename from backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite.scala rename to backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite.scala index 66f1adfb6282c..1fd8983f5876a 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite.scala @@ -14,7 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.execution +package org.apache.gluten.execution.tpcds + +import org.apache.gluten.execution._ import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.expressions.DynamicPruningExpression @@ -48,7 +50,7 @@ class GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite | from store_sales | where ss_quantity between 1 and 20 |""".stripMargin) { _ => } - assert(result(0).getLong(0) == 550458L) + assertResult(550458L)(result.head.getLong(0)) } test("test reading from partitioned table with partition column filter") { @@ -59,7 +61,7 @@ class GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite | where ss_quantity between 1 and 20 | and ss_sold_date_sk = 2452635 |""".stripMargin, - true, + compareResult = true, _ => {} ) } @@ -69,8 +71,8 @@ class GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite |select avg(cs_item_sk), avg(cs_order_number) | from catalog_sales |""".stripMargin) { _ => } - assert(result(0).getDouble(0) == 8998.463336886734) - assert(result(0).getDouble(1) == 80037.12727449503) + assertResult(8998.463336886734)(result.head.getDouble(0)) + assertResult(80037.12727449503)(result.head.getDouble(1)) } test("Gluten-1235: Fix missing reading from the broadcasted value when executing DPP") { @@ -89,7 +91,7 @@ class GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite |""".stripMargin compareResultsAgainstVanillaSpark( testSql, - true, + compareResult = true, df => { val foundDynamicPruningExpr = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -100,11 +102,11 @@ class GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite .asInstanceOf[FileSourceScanExecTransformer] .partitionFilters .exists(_.isInstanceOf[DynamicPruningExpression])) - assert( + assertResult(1823)( foundDynamicPruningExpr(1) .asInstanceOf[FileSourceScanExecTransformer] .selectedPartitions - .size == 1823) + .length) } ) } @@ -119,7 +121,7 @@ class GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite } // On Spark 3.2, there are 15 AdaptiveSparkPlanExec, // and on Spark 3.3, there are 5 AdaptiveSparkPlanExec and 10 ReusedSubqueryExec - assert(subqueryAdaptiveSparkPlan.filter(_ == true).size == 15) + assertResult(15)(subqueryAdaptiveSparkPlan.count(_ == true)) } } @@ -145,12 +147,12 @@ class GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite } => f } - assert(foundDynamicPruningExpr.nonEmpty == true) + assert(foundDynamicPruningExpr.nonEmpty) val reusedExchangeExec = collectWithSubqueries(df.queryExecution.executedPlan) { case r: ReusedExchangeExec => r } - assert(reusedExchangeExec.nonEmpty == true) + assert(reusedExchangeExec.nonEmpty) } } @@ -168,7 +170,7 @@ class GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite } => f } - assert(foundDynamicPruningExpr.nonEmpty == true) + assert(foundDynamicPruningExpr.nonEmpty) val reusedExchangeExec = collectWithSubqueries(df.queryExecution.executedPlan) { case r: ReusedExchangeExec => r @@ -198,7 +200,7 @@ class GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite |ORDER BY channel | LIMIT 100 ; |""".stripMargin - compareResultsAgainstVanillaSpark(testSql, true, df => {}) + compareResultsAgainstVanillaSpark(testSql, compareResult = true, df => {}) } test("GLUTEN-1620: fix 'attribute binding failed.' when executing hash agg with aqe") { diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSParquetColumnarShuffleSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetColumnarShuffleSuite.scala similarity index 91% rename from backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSParquetColumnarShuffleSuite.scala rename to backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetColumnarShuffleSuite.scala index ca3db077285f8..4675de249c6df 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSParquetColumnarShuffleSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetColumnarShuffleSuite.scala @@ -14,7 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.execution +package org.apache.gluten.execution.tpcds + +import org.apache.gluten.execution.{FileSourceScanExecTransformer, GlutenClickHouseTPCDSAbstractSuite} import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.expressions.DynamicPruningExpression @@ -45,7 +47,7 @@ class GlutenClickHouseTPCDSParquetColumnarShuffleSuite extends GlutenClickHouseT | from store_sales | where ss_quantity between 1 and 20 |""".stripMargin) { _ => } - assert(result(0).getLong(0) == 550458L) + assertResult(550458L)(result.head.getLong(0)) } test("test reading from partitioned table with partition column filter") { @@ -56,7 +58,7 @@ class GlutenClickHouseTPCDSParquetColumnarShuffleSuite extends GlutenClickHouseT | where ss_quantity between 1 and 20 | and ss_sold_date_sk = 2452635 |""".stripMargin, - true, + compareResult = true, _ => {} ) } @@ -66,8 +68,8 @@ class GlutenClickHouseTPCDSParquetColumnarShuffleSuite extends GlutenClickHouseT |select avg(cs_item_sk), avg(cs_order_number) | from catalog_sales |""".stripMargin) { _ => } - assert(result(0).getDouble(0) == 8998.463336886734) - assert(result(0).getDouble(1) == 80037.12727449503) + assertResult(8998.463336886734)(result.head.getDouble(0)) + assertResult(80037.12727449503)(result.head.getDouble(1)) } test("Gluten-1235: Fix missing reading from the broadcasted value when executing DPP") { @@ -86,7 +88,7 @@ class GlutenClickHouseTPCDSParquetColumnarShuffleSuite extends GlutenClickHouseT |""".stripMargin compareResultsAgainstVanillaSpark( testSql, - true, + compareResult = true, df => { val foundDynamicPruningExpr = df.queryExecution.executedPlan.collect { case f: FileSourceScanExecTransformer => f @@ -97,11 +99,11 @@ class GlutenClickHouseTPCDSParquetColumnarShuffleSuite extends GlutenClickHouseT .asInstanceOf[FileSourceScanExecTransformer] .partitionFilters .exists(_.isInstanceOf[DynamicPruningExpression])) - assert( + assertResult(1823)( foundDynamicPruningExpr(1) .asInstanceOf[FileSourceScanExecTransformer] .selectedPartitions - .size == 1823) + .length) } ) } @@ -144,13 +146,13 @@ class GlutenClickHouseTPCDSParquetColumnarShuffleSuite extends GlutenClickHouseT } case _ => false } - assert(foundDynamicPruningExpr.nonEmpty == true) + assert(foundDynamicPruningExpr.nonEmpty) val reuseExchange = df.queryExecution.executedPlan.find { case r: ReusedExchangeExec => true case _ => false } - assert(reuseExchange.nonEmpty == true) + assert(reuseExchange.nonEmpty) } } @@ -168,7 +170,7 @@ class GlutenClickHouseTPCDSParquetColumnarShuffleSuite extends GlutenClickHouseT } case _ => false } - assert(foundDynamicPruningExpr.nonEmpty == true) + assert(foundDynamicPruningExpr.nonEmpty) val reuseExchange = df.queryExecution.executedPlan.find { case r: ReusedExchangeExec => true @@ -199,6 +201,6 @@ class GlutenClickHouseTPCDSParquetColumnarShuffleSuite extends GlutenClickHouseT |ORDER BY channel | LIMIT 100 ; |""".stripMargin - compareResultsAgainstVanillaSpark(testSql, true, df => {}) + compareResultsAgainstVanillaSpark(testSql, compareResult = true, df => {}) } } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSParquetGraceHashJoinSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetGraceHashJoinSuite.scala similarity index 93% rename from backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSParquetGraceHashJoinSuite.scala rename to backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetGraceHashJoinSuite.scala index a7b3518cc9815..716ea5761d2d3 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSParquetGraceHashJoinSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetGraceHashJoinSuite.scala @@ -14,7 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.execution +package org.apache.gluten.execution.tpcds + +import org.apache.gluten.execution._ import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.expressions.DynamicPruningExpression @@ -34,7 +36,7 @@ class GlutenClickHouseTPCDSParquetGraceHashJoinSuite extends GlutenClickHouseTPC .set("spark.gluten.sql.columnar.backend.ch.runtime_settings.max_bytes_in_join", "314572800") } - executeTPCDSTest(false); + executeTPCDSTest(false) test("Gluten-1235: Fix missing reading from the broadcasted value when executing DPP") { val testSql = @@ -52,7 +54,7 @@ class GlutenClickHouseTPCDSParquetGraceHashJoinSuite extends GlutenClickHouseTPC |""".stripMargin compareResultsAgainstVanillaSpark( testSql, - true, + compareResult = true, df => { val foundDynamicPruningExpr = df.queryExecution.executedPlan.collect { case f: FileSourceScanExecTransformer => f @@ -63,11 +65,11 @@ class GlutenClickHouseTPCDSParquetGraceHashJoinSuite extends GlutenClickHouseTPC .asInstanceOf[FileSourceScanExecTransformer] .partitionFilters .exists(_.isInstanceOf[DynamicPruningExpression])) - assert( + assertResult(1823)( foundDynamicPruningExpr(1) .asInstanceOf[FileSourceScanExecTransformer] .selectedPartitions - .size == 1823) + .length) } ) } @@ -86,7 +88,7 @@ class GlutenClickHouseTPCDSParquetGraceHashJoinSuite extends GlutenClickHouseTPC } case _ => false } - assert(foundDynamicPruningExpr.nonEmpty == true) + assert(foundDynamicPruningExpr.nonEmpty) val reuseExchange = df.queryExecution.executedPlan.find { case r: ReusedExchangeExec => true diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSParquetRFSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetRFSuite.scala similarity index 93% rename from backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSParquetRFSuite.scala rename to backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetRFSuite.scala index 27137c6d92661..657a6e32146ac 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSParquetRFSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetRFSuite.scala @@ -14,7 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.execution +package org.apache.gluten.execution.tpcds + +import org.apache.gluten.execution.GlutenClickHouseTPCDSAbstractSuite import org.apache.spark.SparkConf diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSParquetSortMergeJoinSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetSortMergeJoinSuite.scala similarity index 93% rename from backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSParquetSortMergeJoinSuite.scala rename to backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetSortMergeJoinSuite.scala index 3ec4e31a41096..7e480361bfe19 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSParquetSortMergeJoinSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetSortMergeJoinSuite.scala @@ -14,8 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.execution +package org.apache.gluten.execution.tpcds +import org.apache.gluten.execution.{CHSortMergeJoinExecTransformer, GlutenClickHouseTPCDSAbstractSuite} import org.apache.gluten.test.FallbackUtil import org.apache.spark.SparkConf @@ -64,7 +65,7 @@ class GlutenClickHouseTPCDSParquetSortMergeJoinSuite extends GlutenClickHouseTPC |i.i_current_price > 1.0 """.stripMargin compareResultsAgainstVanillaSpark( testSql, - true, + compareResult = true, df => { val smjTransformers = df.queryExecution.executedPlan.collect { case f: CHSortMergeJoinExecTransformer => f @@ -83,7 +84,7 @@ class GlutenClickHouseTPCDSParquetSortMergeJoinSuite extends GlutenClickHouseTPC """.stripMargin compareResultsAgainstVanillaSpark( testSql, - true, + compareResult = true, df => { val smjTransformers = df.queryExecution.executedPlan.collect { case f: CHSortMergeJoinExecTransformer => f @@ -102,7 +103,7 @@ class GlutenClickHouseTPCDSParquetSortMergeJoinSuite extends GlutenClickHouseTPC """.stripMargin compareResultsAgainstVanillaSpark( testSql, - true, + compareResult = true, df => { val smjTransformers = df.queryExecution.executedPlan.collect { case f: CHSortMergeJoinExecTransformer => f @@ -124,7 +125,7 @@ class GlutenClickHouseTPCDSParquetSortMergeJoinSuite extends GlutenClickHouseTPC val smjTransformers = df.queryExecution.executedPlan.collect { case f: CHSortMergeJoinExecTransformer => f } - assert(smjTransformers.size == 0) + assert(smjTransformers.isEmpty) assert(FallbackUtil.hasFallback(df.queryExecution.executedPlan)) } } @@ -140,18 +141,18 @@ class GlutenClickHouseTPCDSParquetSortMergeJoinSuite extends GlutenClickHouseTPC val smjTransformers = df.queryExecution.executedPlan.collect { case f: CHSortMergeJoinExecTransformer => f } - assert(smjTransformers.size == 0) + assert(smjTransformers.isEmpty) assert(FallbackUtil.hasFallback(df.queryExecution.executedPlan)) } } - val createItem = + val createItem: String = """CREATE TABLE myitem ( | i_current_price DECIMAL(7,2), | i_category STRING) |USING parquet""".stripMargin - val insertItem = + val insertItem: String = """insert into myitem values |(null,null), |(null,null), @@ -174,7 +175,7 @@ class GlutenClickHouseTPCDSParquetSortMergeJoinSuite extends GlutenClickHouseTPC """.stripMargin compareResultsAgainstVanillaSpark( testSql, - true, + compareResult = true, df => { val smjTransformers = df.queryExecution.executedPlan.collect { case f: CHSortMergeJoinExecTransformer => f @@ -206,7 +207,7 @@ class GlutenClickHouseTPCDSParquetSortMergeJoinSuite extends GlutenClickHouseTPC spark.sql(testSql).show() compareResultsAgainstVanillaSpark( testSql, - true, + compareResult = true, df => { val smjTransformers = df.queryExecution.executedPlan.collect { case f: CHSortMergeJoinExecTransformer => f diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSParquetSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetSuite.scala similarity index 88% rename from backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSParquetSuite.scala rename to backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetSuite.scala index e20ea35e50db3..d0b270d2aae52 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSParquetSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetSuite.scala @@ -14,7 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.execution +package org.apache.gluten.execution.tpcds + +import org.apache.gluten.execution._ import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.expressions.DynamicPruningExpression @@ -47,7 +49,7 @@ class GlutenClickHouseTPCDSParquetSuite extends GlutenClickHouseTPCDSAbstractSui val result = runSql(""" |select count(c_customer_sk) from customer |""".stripMargin) { _ => } - assert(result(0).getLong(0) == 100000L) + assertResult(100000L)(result.head.getLong(0)) } test("test reading from partitioned table") { @@ -56,7 +58,7 @@ class GlutenClickHouseTPCDSParquetSuite extends GlutenClickHouseTPCDSAbstractSui | from store_sales | where ss_quantity between 1 and 20 |""".stripMargin) { _ => } - assert(result(0).getLong(0) == 550458L) + assertResult(550458L)(result.head.getLong(0)) } test("test reading from partitioned table with partition column filter") { @@ -67,7 +69,7 @@ class GlutenClickHouseTPCDSParquetSuite extends GlutenClickHouseTPCDSAbstractSui | where ss_quantity between 1 and 20 | and ss_sold_date_sk = 2452635 |""".stripMargin, - true, + compareResult = true, _ => {} ) } @@ -77,8 +79,8 @@ class GlutenClickHouseTPCDSParquetSuite extends GlutenClickHouseTPCDSAbstractSui |select avg(cs_item_sk), avg(cs_order_number) | from catalog_sales |""".stripMargin) { _ => } - assert(result(0).getDouble(0) == 8998.463336886734) - assert(result(0).getDouble(1) == 80037.12727449503) + assertResult(8998.463336886734)(result.head.getDouble(0)) + assertResult(80037.12727449503)(result.head.getDouble(1)) } test("test union all operator with two tables") { @@ -89,7 +91,7 @@ class GlutenClickHouseTPCDSParquetSuite extends GlutenClickHouseTPCDSAbstractSui | select ws_sold_date_sk as date_sk from web_sales |) |""".stripMargin) { _ => } - assert(result(0).getLong(0) == 791809) + assertResult(791809)(result.head.getLong(0)) } test("test union all operator with three tables") { @@ -103,7 +105,7 @@ class GlutenClickHouseTPCDSParquetSuite extends GlutenClickHouseTPCDSAbstractSui | ) |) |""".stripMargin) { _ => } - assert(result(0).getLong(0) == 791909) + assertResult(791909)(result.head.getLong(0)) } test("test union operator with two tables") { @@ -114,7 +116,7 @@ class GlutenClickHouseTPCDSParquetSuite extends GlutenClickHouseTPCDSAbstractSui | select ws_sold_date_sk as date_sk from web_sales |) |""".stripMargin) { _ => } - assert(result(0).getLong(0) == 73049) + assertResult(73049)(result.head.getLong(0)) } test("Test join with mixed condition 1") { @@ -134,7 +136,7 @@ class GlutenClickHouseTPCDSParquetSuite extends GlutenClickHouseTPCDSAbstractSui | ORDER BY ext_price DESC, i_brand, i_brand_id, i_manufact_id, i_manufact | LIMIT 100; |""".stripMargin - compareResultsAgainstVanillaSpark(testSql, true, _ => {}) + compareResultsAgainstVanillaSpark(testSql, compareResult = true, _ => {}) } test("Gluten-1235: Fix missing reading from the broadcasted value when executing DPP") { @@ -153,7 +155,7 @@ class GlutenClickHouseTPCDSParquetSuite extends GlutenClickHouseTPCDSAbstractSui |""".stripMargin compareResultsAgainstVanillaSpark( testSql, - true, + compareResult = true, df => { val foundDynamicPruningExpr = df.queryExecution.executedPlan.collect { case f: FileSourceScanExecTransformer => f @@ -164,11 +166,11 @@ class GlutenClickHouseTPCDSParquetSuite extends GlutenClickHouseTPCDSAbstractSui .asInstanceOf[FileSourceScanExecTransformer] .partitionFilters .exists(_.isInstanceOf[DynamicPruningExpression])) - assert( + assertResult(1823)( foundDynamicPruningExpr(1) .asInstanceOf[FileSourceScanExecTransformer] .selectedPartitions - .size == 1823) + .length) } ) } @@ -200,13 +202,13 @@ class GlutenClickHouseTPCDSParquetSuite extends GlutenClickHouseTPCDSAbstractSui } case _ => false } - assert(foundDynamicPruningExpr.nonEmpty == true) + assert(foundDynamicPruningExpr.nonEmpty) val reuseExchange = df.queryExecution.executedPlan.find { case r: ReusedExchangeExec => true case _ => false } - assert(reuseExchange.nonEmpty == true) + assert(reuseExchange.nonEmpty) } } @@ -224,7 +226,7 @@ class GlutenClickHouseTPCDSParquetSuite extends GlutenClickHouseTPCDSAbstractSui } case _ => false } - assert(foundDynamicPruningExpr.nonEmpty == true) + assert(foundDynamicPruningExpr.nonEmpty) val reuseExchange = df.queryExecution.executedPlan.find { case r: ReusedExchangeExec => true @@ -255,7 +257,7 @@ class GlutenClickHouseTPCDSParquetSuite extends GlutenClickHouseTPCDSAbstractSui |ORDER BY channel | LIMIT 100 ; |""".stripMargin - compareResultsAgainstVanillaSpark(testSql, true, df => {}) + compareResultsAgainstVanillaSpark(testSql, compareResult = true, df => {}) } test("Bug-382 collec_list failure") { @@ -264,7 +266,7 @@ class GlutenClickHouseTPCDSParquetSuite extends GlutenClickHouseTPCDSAbstractSui |select cc_call_center_id, collect_list(cc_call_center_sk) from call_center group by cc_call_center_id |order by cc_call_center_id |""".stripMargin - compareResultsAgainstVanillaSpark(sql, true, df => {}) + compareResultsAgainstVanillaSpark(sql, compareResult = true, df => {}) } test("collec_set") { @@ -275,7 +277,7 @@ class GlutenClickHouseTPCDSParquetSuite extends GlutenClickHouseTPCDSAbstractSui |lateral view explode(set) as b |order by a, b |""".stripMargin - compareResultsAgainstVanillaSpark(sql, true, _ => {}) + compareResultsAgainstVanillaSpark(sql, compareResult = true, _ => {}) } test("GLUTEN-1626: test 'roundHalfup'") { @@ -286,7 +288,7 @@ class GlutenClickHouseTPCDSParquetSuite extends GlutenClickHouseTPCDSAbstractSui |from store_sales |group by a order by a |""".stripMargin - compareResultsAgainstVanillaSpark(sql0, true, _ => {}) + compareResultsAgainstVanillaSpark(sql0, compareResult = true, _ => {}) val sql1 = """ @@ -295,7 +297,7 @@ class GlutenClickHouseTPCDSParquetSuite extends GlutenClickHouseTPCDSAbstractSui |from store_sales |group by a order by a |""".stripMargin - compareResultsAgainstVanillaSpark(sql1, true, _ => {}) + compareResultsAgainstVanillaSpark(sql1, compareResult = true, _ => {}) val sql2 = """ @@ -304,7 +306,7 @@ class GlutenClickHouseTPCDSParquetSuite extends GlutenClickHouseTPCDSAbstractSui |from catalog_sales |group by a order by a |""".stripMargin - compareResultsAgainstVanillaSpark(sql2, true, _ => {}) + compareResultsAgainstVanillaSpark(sql2, compareResult = true, _ => {}) val sql3 = """ @@ -313,7 +315,7 @@ class GlutenClickHouseTPCDSParquetSuite extends GlutenClickHouseTPCDSAbstractSui |from catalog_sales |group by a order by a |""".stripMargin - compareResultsAgainstVanillaSpark(sql3, true, _ => {}) + compareResultsAgainstVanillaSpark(sql3, compareResult = true, _ => {}) val sql4 = """ @@ -322,7 +324,7 @@ class GlutenClickHouseTPCDSParquetSuite extends GlutenClickHouseTPCDSAbstractSui |from web_sales |group by a order by a |""".stripMargin - compareResultsAgainstVanillaSpark(sql4, true, _ => {}) + compareResultsAgainstVanillaSpark(sql4, compareResult = true, _ => {}) val sql5 = """ @@ -331,7 +333,7 @@ class GlutenClickHouseTPCDSParquetSuite extends GlutenClickHouseTPCDSAbstractSui |from web_sales |group by a order by a |""".stripMargin - compareResultsAgainstVanillaSpark(sql5, true, _ => {}) + compareResultsAgainstVanillaSpark(sql5, compareResult = true, _ => {}) } } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDatetimeExpressionSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseDatetimeExpressionSuite.scala similarity index 98% rename from backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDatetimeExpressionSuite.scala rename to backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseDatetimeExpressionSuite.scala index a1749efb18b2a..b3196286e128f 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDatetimeExpressionSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseDatetimeExpressionSuite.scala @@ -14,7 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.execution +package org.apache.gluten.execution.tpch + +import org.apache.gluten.execution.GlutenClickHouseTPCHAbstractSuite import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.util.DateTimeTestUtils diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala similarity index 94% rename from backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala rename to backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala index 6caac99181fa4..c2e2f9f5565f2 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala @@ -14,9 +14,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.execution +package org.apache.gluten.execution.tpch import org.apache.gluten.GlutenConfig +import org.apache.gluten.execution._ import org.apache.gluten.extension.GlutenPlan import org.apache.spark.SparkConf @@ -65,7 +66,7 @@ class GlutenClickHouseTPCHColumnarShuffleParquetAQESuite assert(plans.size == 5) assert(plans(4).metrics("numFiles").value === 1) - assert(plans(4).metrics("pruningTime").value === -1) + assert(plans(4).metrics("pruningTime").value === pruningTimeValueSpark) assert(plans(4).metrics("filesSize").value === 19230111) assert(plans(4).metrics("numOutputRows").value === 600572) @@ -80,8 +81,8 @@ class GlutenClickHouseTPCHColumnarShuffleParquetAQESuite assert(plans(1).metrics("numOutputRows").value === 8) assert(plans(1).metrics("outputVectors").value === 2) - assert(plans(0).metrics("numInputRows").value === 4) - assert(plans(0).metrics("numOutputRows").value === 4) + assert(plans.head.metrics("numInputRows").value === 4) + assert(plans.head.metrics("numOutputRows").value === 4) } } @@ -97,7 +98,7 @@ class GlutenClickHouseTPCHColumnarShuffleParquetAQESuite assert(plans.size == 3) assert(plans(2).metrics("numFiles").value === 1) - assert(plans(2).metrics("pruningTime").value === -1) + assert(plans(2).metrics("pruningTime").value === pruningTimeValueSpark) assert(plans(2).metrics("filesSize").value === 19230111) assert(plans(1).metrics("numInputRows").value === 591673) @@ -105,8 +106,8 @@ class GlutenClickHouseTPCHColumnarShuffleParquetAQESuite assert(plans(1).metrics("outputVectors").value === 1) // Execute Sort operator, it will read the data twice. - assert(plans(0).metrics("numOutputRows").value === 8) - assert(plans(0).metrics("outputVectors").value === 2) + assert(plans.head.metrics("numOutputRows").value === 8) + assert(plans.head.metrics("outputVectors").value === 2) } } } @@ -147,8 +148,8 @@ class GlutenClickHouseTPCHColumnarShuffleParquetAQESuite assert(inputIteratorTransformers(1).metrics("numInputRows").value === 3111) assert(inputIteratorTransformers(1).metrics("numOutputRows").value === 3111) - assert(inputIteratorTransformers(0).metrics("numInputRows").value === 15224) - assert(inputIteratorTransformers(0).metrics("numOutputRows").value === 15224) + assert(inputIteratorTransformers.head.metrics("numInputRows").value === 15224) + assert(inputIteratorTransformers.head.metrics("numOutputRows").value === 15224) } } } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHParquetAQEConcurrentSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetAQEConcurrentSuite.scala similarity index 96% rename from backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHParquetAQEConcurrentSuite.scala rename to backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetAQEConcurrentSuite.scala index 9f4befbb01a9b..8c706f6836399 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHParquetAQEConcurrentSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetAQEConcurrentSuite.scala @@ -14,7 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.execution +package org.apache.gluten.execution.tpch + +import org.apache.gluten.execution.GlutenClickHouseTPCHAbstractSuite import org.apache.spark.SparkConf import org.apache.spark.sql.DataFrame diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHParquetAQESuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetAQESuite.scala similarity index 98% rename from backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHParquetAQESuite.scala rename to backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetAQESuite.scala index c3e64a94146de..1d8389b481439 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHParquetAQESuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetAQESuite.scala @@ -14,7 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.execution +package org.apache.gluten.execution.tpch + +import org.apache.gluten.execution._ import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.optimizer.BuildLeft @@ -345,9 +347,7 @@ class GlutenClickHouseTPCHParquetAQESuite |order by t1.l_orderkey, t2.o_orderkey, t2.o_year, t1.l_cnt, t2.o_cnt |limit 100 | - |""".stripMargin, - true, - true + |""".stripMargin )(df => {}) runQueryAndCompare( @@ -366,10 +366,7 @@ class GlutenClickHouseTPCHParquetAQESuite |order by t1.l_orderkey, t2.o_orderkey, t2.o_year |limit 100 | - |""".stripMargin, - true, - true - )(df => {}) + |""".stripMargin)(df => {}) } } } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHParquetBucketSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetBucketSuite.scala similarity index 95% rename from backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHParquetBucketSuite.scala rename to backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetBucketSuite.scala index c164fae708f8f..614e0124b9ff2 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHParquetBucketSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetBucketSuite.scala @@ -14,7 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.execution +package org.apache.gluten.execution.tpch + +import org.apache.gluten.execution._ import org.apache.spark.SparkConf import org.apache.spark.sql.DataFrame @@ -259,10 +261,10 @@ class GlutenClickHouseTPCHParquetBucketSuite val plans = collect(df.queryExecution.executedPlan) { case scanExec: BasicScanExecTransformer => scanExec } - assert(!(plans(0).asInstanceOf[FileSourceScanExecTransformer].bucketedScan)) - assert(plans(0).metrics("numFiles").value === 4) - assert(plans(0).metrics("pruningTime").value === -1) - assert(plans(0).metrics("numOutputRows").value === 600572) + assert(!plans.head.asInstanceOf[FileSourceScanExecTransformer].bucketedScan) + assert(plans.head.metrics("numFiles").value === 4) + assert(plans.head.metrics("pruningTime").value === pruningTimeValueSpark) + assert(plans.head.metrics("numOutputRows").value === 600572) } ) } @@ -319,7 +321,7 @@ class GlutenClickHouseTPCHParquetBucketSuite } if (sparkVersion.equals("3.2")) { - assert(!(plans(11).asInstanceOf[FileSourceScanExecTransformer].bucketedScan)) + assert(!plans(11).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) } else { assert(plans(11).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) } @@ -359,14 +361,14 @@ class GlutenClickHouseTPCHParquetBucketSuite .isInstanceOf[InputIteratorTransformer]) if (sparkVersion.equals("3.2")) { - assert(!(plans(2).asInstanceOf[FileSourceScanExecTransformer].bucketedScan)) + assert(!plans(2).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) } else { assert(plans(2).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) } assert(plans(2).metrics("numFiles").value === 4) assert(plans(2).metrics("numOutputRows").value === 15000) - assert(!(plans(3).asInstanceOf[FileSourceScanExecTransformer].bucketedScan)) + assert(!plans(3).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) assert(plans(3).metrics("numFiles").value === 4) assert(plans(3).metrics("numOutputRows").value === 150000) } @@ -404,12 +406,12 @@ class GlutenClickHouseTPCHParquetBucketSuite } // bucket join assert( - plans(0) + plans.head .asInstanceOf[HashJoinLikeExecTransformer] .left .isInstanceOf[ProjectExecTransformer]) assert( - plans(0) + plans.head .asInstanceOf[HashJoinLikeExecTransformer] .right .isInstanceOf[ProjectExecTransformer]) @@ -453,10 +455,10 @@ class GlutenClickHouseTPCHParquetBucketSuite val plans = collect(df.queryExecution.executedPlan) { case scanExec: BasicScanExecTransformer => scanExec } - assert(!(plans(0).asInstanceOf[FileSourceScanExecTransformer].bucketedScan)) - assert(plans(0).metrics("numFiles").value === 4) - assert(plans(0).metrics("pruningTime").value === -1) - assert(plans(0).metrics("numOutputRows").value === 600572) + assert(!plans.head.asInstanceOf[FileSourceScanExecTransformer].bucketedScan) + assert(plans.head.metrics("numFiles").value === 4) + assert(plans.head.metrics("pruningTime").value === pruningTimeValueSpark) + assert(plans.head.metrics("numOutputRows").value === 600572) } ) } @@ -472,12 +474,12 @@ class GlutenClickHouseTPCHParquetBucketSuite } // bucket join assert( - plans(0) + plans.head .asInstanceOf[HashJoinLikeExecTransformer] .left .isInstanceOf[FilterExecTransformerBase]) assert( - plans(0) + plans.head .asInstanceOf[HashJoinLikeExecTransformer] .right .isInstanceOf[ProjectExecTransformer]) @@ -654,7 +656,7 @@ class GlutenClickHouseTPCHParquetBucketSuite |""".stripMargin compareResultsAgainstVanillaSpark( SQL, - true, + compareResult = true, df => {} ) } @@ -675,7 +677,7 @@ class GlutenClickHouseTPCHParquetBucketSuite |""".stripMargin compareResultsAgainstVanillaSpark( SQL, - true, + compareResult = true, df => { checkHashAggregateCount(df, 1) } ) @@ -690,7 +692,7 @@ class GlutenClickHouseTPCHParquetBucketSuite |""".stripMargin compareResultsAgainstVanillaSpark( SQL1, - true, + compareResult = true, df => { checkHashAggregateCount(df, 1) } ) @@ -702,7 +704,7 @@ class GlutenClickHouseTPCHParquetBucketSuite |""".stripMargin compareResultsAgainstVanillaSpark( SQL2, - true, + compareResult = true, df => { checkHashAggregateCount(df, 1) } ) @@ -716,7 +718,7 @@ class GlutenClickHouseTPCHParquetBucketSuite |""".stripMargin compareResultsAgainstVanillaSpark( SQL3, - true, + compareResult = true, df => { checkHashAggregateCount(df, 2) } ) @@ -731,7 +733,7 @@ class GlutenClickHouseTPCHParquetBucketSuite |""".stripMargin compareResultsAgainstVanillaSpark( SQL4, - true, + compareResult = true, df => { checkHashAggregateCount(df, 4) } ) @@ -745,7 +747,7 @@ class GlutenClickHouseTPCHParquetBucketSuite |""".stripMargin compareResultsAgainstVanillaSpark( SQL5, - true, + compareResult = true, df => { checkHashAggregateCount(df, 4) } ) @@ -755,7 +757,7 @@ class GlutenClickHouseTPCHParquetBucketSuite |""".stripMargin compareResultsAgainstVanillaSpark( SQL6, - true, + compareResult = true, df => { // there is a shuffle between two phase hash aggregate. checkHashAggregateCount(df, 2) @@ -773,7 +775,7 @@ class GlutenClickHouseTPCHParquetBucketSuite |""".stripMargin compareResultsAgainstVanillaSpark( SQL7, - true, + compareResult = true, df => { checkHashAggregateCount(df, 1) } @@ -790,7 +792,7 @@ class GlutenClickHouseTPCHParquetBucketSuite |""".stripMargin compareResultsAgainstVanillaSpark( SQL, - true, + compareResult = true, df => { checkHashAggregateCount(df, 0) val plans = collect(df.queryExecution.executedPlan) { case agg: SortAggregateExec => agg } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHParquetRFSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetRFSuite.scala similarity index 91% rename from backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHParquetRFSuite.scala rename to backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetRFSuite.scala index 83e847a707ff7..eb4118689fef3 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHParquetRFSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetRFSuite.scala @@ -14,7 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.execution +package org.apache.gluten.execution.tpch + +import org.apache.gluten.execution._ import org.apache.spark.SparkConf @@ -60,7 +62,10 @@ class GlutenClickHouseTPCHParquetRFSuite extends GlutenClickHouseTPCHSaltNullPar } assert(filterExecs.size == 4) assert( - filterExecs(0).asInstanceOf[FilterExecTransformer].toString.contains("might_contain")) + filterExecs.head + .asInstanceOf[FilterExecTransformer] + .toString + .contains("might_contain")) } } ) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSaltNullParquetSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala similarity index 98% rename from backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSaltNullParquetSuite.scala rename to backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala index 694a9f253becb..d903304367d06 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSaltNullParquetSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala @@ -14,9 +14,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.execution +package org.apache.gluten.execution.tpch import org.apache.gluten.GlutenConfig +import org.apache.gluten.execution._ import org.apache.gluten.extension.GlutenPlan import org.apache.spark.{SparkConf, SparkException} @@ -41,7 +42,7 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr override protected val queriesResults: String = rootPath + "queries-output" protected val BACKEND_CONF_KEY = "spark.gluten.sql.columnar.backend.ch." - protected val BACKEND_RUNTIME_CINF_KEY = BACKEND_CONF_KEY + "runtime_config." + protected val BACKEND_RUNTIME_CINF_KEY: String = BACKEND_CONF_KEY + "runtime_config." override protected def sparkConf: SparkConf = { super.sparkConf @@ -205,7 +206,7 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr | show tables; |""".stripMargin) .collect() - assert(result.size == 8) + assertResult(8)(result.length) } test("TPCH Q1") { @@ -753,8 +754,7 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr runQueryAndCompare(query)(checkGlutenOperatorMatch[ProjectExecTransformer]) } - // see issue https://github.com/Kyligence/ClickHouse/issues/93 - ignore("TPCH Q22") { + test("TPCH Q22") { runTPCHQuery(22) { df => } } @@ -1253,7 +1253,7 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr |select n_regionkey, collect_list(if(n_regionkey=0, n_name, null)) as t from nation group by n_regionkey |order by n_regionkey |""".stripMargin - compareResultsAgainstVanillaSpark(sql, true, df => {}) + compareResultsAgainstVanillaSpark(sql, compareResult = true, df => {}) } test("collect_set") { @@ -1366,7 +1366,7 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr } } - test("test posexplode issue: https://github.com/oap-project/gluten/issues/1767") { + testSparkVersionLE33("test posexplode issue: https://github.com/oap-project/gluten/issues/1767") { spark.sql("create table test_1767 (id bigint, data map) using parquet") spark.sql("INSERT INTO test_1767 values(1, map('k', 'v'))") @@ -1855,7 +1855,7 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr | ) t1 |) t2 where rank = 1 """.stripMargin - compareResultsAgainstVanillaSpark(sql, true, { _ => }) + compareResultsAgainstVanillaSpark(sql, true, { _ => }, isSparkVersionLE("3.3")) } test("GLUTEN-1874 not null in both streams") { @@ -1873,7 +1873,7 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr | ) t1 |) t2 where rank = 1 """.stripMargin - compareResultsAgainstVanillaSpark(sql, true, { _ => }) + compareResultsAgainstVanillaSpark(sql, true, { _ => }, isSparkVersionLE("3.3")) } test("GLUTEN-2095: test cast(string as binary)") { @@ -2158,12 +2158,12 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr } test("GLUTEN-3149/GLUTEN-5580: Fix convert float to int") { - val tbl_create_sql = "create table test_tbl_3149(a int, b bigint) using parquet"; + val tbl_create_sql = "create table test_tbl_3149(a int, b bigint) using parquet" val tbl_insert_sql = "insert into test_tbl_3149 values(1, 0), (2, 171396196666200)" val select_sql_1 = "select cast(a * 1.0f/b as int) as x from test_tbl_3149 where a = 1" val select_sql_2 = "select cast(b/100 as int) from test_tbl_3149 where a = 2" spark.sql(tbl_create_sql) - spark.sql(tbl_insert_sql); + spark.sql(tbl_insert_sql) compareResultsAgainstVanillaSpark(select_sql_1, true, { _ => }) compareResultsAgainstVanillaSpark(select_sql_2, true, { _ => }) spark.sql("drop table test_tbl_3149") @@ -2223,12 +2223,12 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr test("GLUTEN-3134: Bug fix left join not match") { withSQLConf(("spark.sql.autoBroadcastJoinThreshold", "1B")) { val left_tbl_create_sql = - "create table test_tbl_left_3134(id bigint, name string) using parquet"; + "create table test_tbl_left_3134(id bigint, name string) using parquet" val right_tbl_create_sql = - "create table test_tbl_right_3134(id string, name string) using parquet"; + "create table test_tbl_right_3134(id string, name string) using parquet" val left_data_insert_sql = - "insert into test_tbl_left_3134 values(2, 'a'), (3, 'b'), (673, 'c')"; - val right_data_insert_sql = "insert into test_tbl_right_3134 values('673', 'c')"; + "insert into test_tbl_left_3134 values(2, 'a'), (3, 'b'), (673, 'c')" + val right_data_insert_sql = "insert into test_tbl_right_3134 values('673', 'c')" val join_select_sql_1 = "select a.id, b.cnt from " + "(select id from test_tbl_left_3134) as a " + "left join (select id, 12 as cnt from test_tbl_right_3134 group by id) as b on a.id = b.id" @@ -2254,9 +2254,7 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr } } - // Please see the issue: https://github.com/oap-project/gluten/issues/3731 - ignore( - "GLUTEN-3534: Fix incorrect logic of judging whether supports pre-project for the shuffle") { + test("GLUTEN-3534: Fix incorrect logic of judging whether supports pre-project for the shuffle") { withSQLConf(("spark.sql.autoBroadcastJoinThreshold", "-1")) { runQueryAndCompare( s""" @@ -2275,9 +2273,7 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr |order by t1.l_orderkey, t2.o_orderkey, t2.o_year, t1.l_cnt, t2.o_cnt |limit 100 | - |""".stripMargin, - true, - true + |""".stripMargin )(df => {}) runQueryAndCompare( @@ -2296,9 +2292,7 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr |order by t1.l_orderkey, t2.o_orderkey, t2.o_year |limit 100 | - |""".stripMargin, - true, - true + |""".stripMargin )(df => {}) } } @@ -2405,8 +2399,8 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr } test("GLUTEN-3521: Bug fix substring index start from 1") { - val tbl_create_sql = "create table test_tbl_3521(id bigint, name string) using parquet"; - val data_insert_sql = "insert into test_tbl_3521 values(1, 'abcdefghijk'), (2, '2023-10-32')"; + val tbl_create_sql = "create table test_tbl_3521(id bigint, name string) using parquet" + val data_insert_sql = "insert into test_tbl_3521 values(1, 'abcdefghijk'), (2, '2023-10-32')" val select_sql = "select id, substring(name, 0), substring(name, 0, 3), substring(name from 0), substring(name from 0 for 100) from test_tbl_3521" spark.sql(tbl_create_sql) @@ -2452,7 +2446,7 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr | ) t1 |) t2 where rank = 1 order by p_partkey limit 100 |""".stripMargin - runQueryAndCompare(sql)({ _ => }) + runQueryAndCompare(sql, noFallBack = isSparkVersionLE("3.3"))({ _ => }) } test("GLUTEN-4190: crush on flattening a const null column") { @@ -2485,9 +2479,9 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr } test("GLUTEN-4085: Fix unix_timestamp") { - val tbl_create_sql = "create table test_tbl_4085(id bigint, data string) using parquet"; + val tbl_create_sql = "create table test_tbl_4085(id bigint, data string) using parquet" val data_insert_sql = - "insert into test_tbl_4085 values(1, '2023-12-18'),(2, '2023-12-19'), (3, '2023-12-20')"; + "insert into test_tbl_4085 values(1, '2023-12-18'),(2, '2023-12-19'), (3, '2023-12-20')" val select_sql = "select id, unix_timestamp(to_date(data), 'yyyy-MM-dd') from test_tbl_4085" spark.sql(tbl_create_sql) @@ -2497,8 +2491,8 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr } test("GLUTEN-3951: Bug fix floor") { - val tbl_create_sql = "create table test_tbl_3951(d double) using parquet"; - val data_insert_sql = "insert into test_tbl_3951 values(1.0), (2.0), (2.5)"; + val tbl_create_sql = "create table test_tbl_3951(d double) using parquet" + val data_insert_sql = "insert into test_tbl_3951 values(1.0), (2.0), (2.5)" val select_sql = "select floor(d), floor(log10(d-1)), floor(log10(d-2)) from test_tbl_3951" spark.sql(tbl_create_sql) @@ -2559,7 +2553,7 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr } test("GLUTEN-4279: Bug fix hour diff") { - val tbl_create_sql = "create table test_tbl_4279(id bigint, data string) using parquet"; + val tbl_create_sql = "create table test_tbl_4279(id bigint, data string) using parquet" val tbl_insert_sql = "insert into test_tbl_4279 values(1, '2024-01-04 11:22:33'), " + "(2, '2024-01-04 11:22:33.456+08'), (3, '2024'), (4, '2024-01'), (5, '2024-01-04'), " + "(6, '2024-01-04 12'), (7, '2024-01-04 12:12'), (8, '11:22:33'), (9, '22:33')," + @@ -2636,10 +2630,10 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr test("Inequal join support") { withSQLConf(("spark.sql.autoBroadcastJoinThreshold", "-1")) { - spark.sql("create table ineq_join_t1 (key bigint, value bigint) using parquet"); - spark.sql("create table ineq_join_t2 (key bigint, value bigint) using parquet"); - spark.sql("insert into ineq_join_t1 values(1, 1), (2, 2), (3, 3), (4, 4), (5, 5)"); - spark.sql("insert into ineq_join_t2 values(2, 2), (2, 1), (3, 3), (4, 6), (5, 3)"); + spark.sql("create table ineq_join_t1 (key bigint, value bigint) using parquet") + spark.sql("create table ineq_join_t2 (key bigint, value bigint) using parquet") + spark.sql("insert into ineq_join_t1 values(1, 1), (2, 2), (3, 3), (4, 4), (5, 5)") + spark.sql("insert into ineq_join_t2 values(2, 2), (2, 1), (3, 3), (4, 6), (5, 3)") val sql1 = """ | select t1.key, t1.value, t2.key, t2.value from ineq_join_t1 as t1 diff --git a/backends-clickhouse/src/test/scala/org/apache/spark/gluten/NativeWriteChecker.scala b/backends-clickhouse/src/test/scala/org/apache/spark/gluten/NativeWriteChecker.scala index 590d221f0e3a0..fc30d151b675d 100644 --- a/backends-clickhouse/src/test/scala/org/apache/spark/gluten/NativeWriteChecker.scala +++ b/backends-clickhouse/src/test/scala/org/apache/spark/gluten/NativeWriteChecker.scala @@ -40,7 +40,7 @@ trait NativeWriteChecker override def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = { if (!nativeUsed) { val executedPlan = stripAQEPlan(qe.executedPlan) - nativeUsed = if (isSparkVersionGE("3.4")) { + nativeUsed = if (isSparkVersionGE("3.5")) { executedPlan.find(_.isInstanceOf[ColumnarWriteFilesExec]).isDefined } else { executedPlan.find(_.isInstanceOf[FakeRowAdaptor]).isDefined From df6fe9065e0204c4f3e30dcb59d6d98df8b943e6 Mon Sep 17 00:00:00 2001 From: Rui Mo Date: Wed, 31 Jul 2024 15:55:14 +0800 Subject: [PATCH 33/61] [VL] Support row type and fix subfield in filter push-down (#6618) --- .../gluten/execution/TestOperator.scala | 56 +++++++++++++- cpp/velox/substrait/SubstraitParser.cc | 14 +++- cpp/velox/substrait/SubstraitParser.h | 5 +- cpp/velox/substrait/SubstraitToVeloxPlan.cc | 73 ++++++++++++------- 4 files changed, 116 insertions(+), 32 deletions(-) diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/TestOperator.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/TestOperator.scala index dcae4920d01c2..0fb5fb54900b4 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/TestOperator.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/TestOperator.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.window.WindowExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{DecimalType, IntegerType, StringType, StructField, StructType} +import org.apache.spark.sql.types.{ArrayType, DecimalType, IntegerType, StringType, StructField, StructType} import java.util.concurrent.TimeUnit @@ -102,6 +102,33 @@ class TestOperator extends VeloxWholeStageTransformerSuite with AdaptiveSparkPla "where l_comment is null") { _ => } assert(df.isEmpty) checkLengthAndPlan(df, 0) + + // Struct of array. + val data = + Row(Row(Array("a", "b", "c"), null)) :: + Row(Row(Array("d", "e", "f"), Array(1, 2, 3))) :: + Row(Row(null, null)) :: Nil + + val schema = new StructType() + .add( + "struct", + new StructType() + .add("a0", ArrayType(StringType)) + .add("a1", ArrayType(IntegerType))) + + val dataFrame = spark.createDataFrame(JavaConverters.seqAsJavaList(data), schema) + + withTempPath { + path => + dataFrame.write.parquet(path.getCanonicalPath) + spark.read.parquet(path.getCanonicalPath).createOrReplaceTempView("view") + runQueryAndCompare("select * from view where struct is null") { + checkGlutenOperatorMatch[FileSourceScanExecTransformer] + } + runQueryAndCompare("select * from view where struct.a0 is null") { + checkGlutenOperatorMatch[FileSourceScanExecTransformer] + } + } } test("is_null_has_null") { @@ -119,6 +146,33 @@ class TestOperator extends VeloxWholeStageTransformerSuite with AdaptiveSparkPla "select l_orderkey from lineitem where l_comment is not null " + "and l_orderkey = 1") { _ => } checkLengthAndPlan(df, 6) + + // Struct of array. + val data = + Row(Row(Array("a", "b", "c"), null)) :: + Row(Row(Array("d", "e", "f"), Array(1, 2, 3))) :: + Row(Row(null, null)) :: Nil + + val schema = new StructType() + .add( + "struct", + new StructType() + .add("a0", ArrayType(StringType)) + .add("a1", ArrayType(IntegerType))) + + val dataFrame = spark.createDataFrame(JavaConverters.seqAsJavaList(data), schema) + + withTempPath { + path => + dataFrame.write.parquet(path.getCanonicalPath) + spark.read.parquet(path.getCanonicalPath).createOrReplaceTempView("view") + runQueryAndCompare("select * from view where struct is not null") { + checkGlutenOperatorMatch[FileSourceScanExecTransformer] + } + runQueryAndCompare("select * from view where struct.a0 is not null") { + checkGlutenOperatorMatch[FileSourceScanExecTransformer] + } + } } test("is_null and is_not_null coexist") { diff --git a/cpp/velox/substrait/SubstraitParser.cc b/cpp/velox/substrait/SubstraitParser.cc index 6eb62f854e0d0..006a20c232eaa 100644 --- a/cpp/velox/substrait/SubstraitParser.cc +++ b/cpp/velox/substrait/SubstraitParser.cc @@ -141,11 +141,21 @@ void SubstraitParser::parseColumnTypes( return; } -int32_t SubstraitParser::parseReferenceSegment(const ::substrait::Expression::ReferenceSegment& refSegment) { +bool SubstraitParser::parseReferenceSegment( + const ::substrait::Expression::ReferenceSegment& refSegment, + uint32_t& fieldIndex) { auto typeCase = refSegment.reference_type_case(); switch (typeCase) { case ::substrait::Expression::ReferenceSegment::ReferenceTypeCase::kStructField: { - return refSegment.struct_field().field(); + if (refSegment.struct_field().has_child()) { + // To parse subfield index is not supported. + return false; + } + fieldIndex = refSegment.struct_field().field(); + if (fieldIndex < 0) { + return false; + } + return true; } default: VELOX_NYI("Substrait conversion not supported for ReferenceSegment '{}'", std::to_string(typeCase)); diff --git a/cpp/velox/substrait/SubstraitParser.h b/cpp/velox/substrait/SubstraitParser.h index 1f766b91ca1b1..f42d05b4a21cc 100644 --- a/cpp/velox/substrait/SubstraitParser.h +++ b/cpp/velox/substrait/SubstraitParser.h @@ -50,8 +50,9 @@ class SubstraitParser { /// Parse Substrait Type to Velox type. static facebook::velox::TypePtr parseType(const ::substrait::Type& substraitType, bool asLowerCase = false); - /// Parse Substrait ReferenceSegment. - static int32_t parseReferenceSegment(const ::substrait::Expression::ReferenceSegment& refSegment); + /// Parse Substrait ReferenceSegment and extract the field index. Return false if the segment is not a valid unnested + /// field. + static bool parseReferenceSegment(const ::substrait::Expression::ReferenceSegment& refSegment, uint32_t& fieldIndex); /// Make names in the format of {prefix}_{index}. static std::vector makeNames(const std::string& prefix, int size); diff --git a/cpp/velox/substrait/SubstraitToVeloxPlan.cc b/cpp/velox/substrait/SubstraitToVeloxPlan.cc index 7b41f7071e842..d7de841191ed0 100644 --- a/cpp/velox/substrait/SubstraitToVeloxPlan.cc +++ b/cpp/velox/substrait/SubstraitToVeloxPlan.cc @@ -1530,8 +1530,7 @@ bool SubstraitToVeloxPlanConverter::fieldOrWithLiteral( if (arguments.size() == 1) { if (arguments[0].value().has_selection()) { // Only field exists. - fieldIndex = SubstraitParser::parseReferenceSegment(arguments[0].value().selection().direct_reference()); - return true; + return SubstraitParser::parseReferenceSegment(arguments[0].value().selection().direct_reference(), fieldIndex); } else { return false; } @@ -1546,13 +1545,17 @@ bool SubstraitToVeloxPlanConverter::fieldOrWithLiteral( for (const auto& param : arguments) { auto typeCase = param.value().rex_type_case(); switch (typeCase) { - case ::substrait::Expression::RexTypeCase::kSelection: - fieldIndex = SubstraitParser::parseReferenceSegment(param.value().selection().direct_reference()); + case ::substrait::Expression::RexTypeCase::kSelection: { + if (!SubstraitParser::parseReferenceSegment(param.value().selection().direct_reference(), fieldIndex)) { + return false; + } fieldExists = true; break; - case ::substrait::Expression::RexTypeCase::kLiteral: + } + case ::substrait::Expression::RexTypeCase::kLiteral: { literalExists = true; break; + } default: break; } @@ -1564,7 +1567,7 @@ bool SubstraitToVeloxPlanConverter::fieldOrWithLiteral( bool SubstraitToVeloxPlanConverter::childrenFunctionsOnSameField( const ::substrait::Expression_ScalarFunction& function) { // Get the column indices of the children functions. - std::vector colIndices; + std::vector colIndices; for (const auto& arg : function.arguments()) { if (arg.value().has_scalar_function()) { const auto& scalarFunction = arg.value().scalar_function(); @@ -1572,14 +1575,16 @@ bool SubstraitToVeloxPlanConverter::childrenFunctionsOnSameField( if (param.value().has_selection()) { const auto& field = param.value().selection(); VELOX_CHECK(field.has_direct_reference()); - int32_t colIdx = SubstraitParser::parseReferenceSegment(field.direct_reference()); + uint32_t colIdx; + if (!SubstraitParser::parseReferenceSegment(field.direct_reference(), colIdx)) { + return false; + } colIndices.emplace_back(colIdx); } } } else if (arg.value().has_singular_or_list()) { const auto& singularOrList = arg.value().singular_or_list(); - int32_t colIdx = getColumnIndexFromSingularOrList(singularOrList); - colIndices.emplace_back(colIdx); + colIndices.emplace_back(getColumnIndexFromSingularOrList(singularOrList)); } else { return false; } @@ -1711,8 +1716,9 @@ void SubstraitToVeloxPlanConverter::separateFilters( if (format == dwio::common::FileFormat::ORC && scalarFunction.arguments().size() > 0) { auto value = scalarFunction.arguments().at(0).value(); if (value.has_selection()) { - uint32_t fieldIndex = SubstraitParser::parseReferenceSegment(value.selection().direct_reference()); - if (!veloxTypeList.empty() && veloxTypeList.at(fieldIndex)->isDecimal()) { + uint32_t fieldIndex; + bool parsed = SubstraitParser::parseReferenceSegment(value.selection().direct_reference(), fieldIndex); + if (!parsed || (!veloxTypeList.empty() && veloxTypeList.at(fieldIndex)->isDecimal())) { remainingFunctions.emplace_back(scalarFunction); continue; } @@ -1870,14 +1876,20 @@ void SubstraitToVeloxPlanConverter::setFilterInfo( for (const auto& param : scalarFunction.arguments()) { auto typeCase = param.value().rex_type_case(); switch (typeCase) { - case ::substrait::Expression::RexTypeCase::kSelection: + case ::substrait::Expression::RexTypeCase::kSelection: { typeCases.emplace_back("kSelection"); - colIdx = SubstraitParser::parseReferenceSegment(param.value().selection().direct_reference()); + uint32_t index; + VELOX_CHECK( + SubstraitParser::parseReferenceSegment(param.value().selection().direct_reference(), index), + "Failed to parse the column index from the selection."); + colIdx = index; break; - case ::substrait::Expression::RexTypeCase::kLiteral: + } + case ::substrait::Expression::RexTypeCase::kLiteral: { typeCases.emplace_back("kLiteral"); substraitLit = param.value().literal(); break; + } default: VELOX_NYI("Substrait conversion not supported for arg type '{}'", std::to_string(typeCase)); } @@ -2177,18 +2189,17 @@ void SubstraitToVeloxPlanConverter::constructSubfieldFilters( VELOX_CHECK(value == filterInfo.upperBounds_[0].value().value(), "invalid state of bool equal"); filters[common::Subfield(inputName)] = std::make_unique(value, nullAllowed); } - } else if constexpr (KIND == facebook::velox::TypeKind::ARRAY || KIND == facebook::velox::TypeKind::MAP) { - // Only IsNotNull and IsNull are supported for array and map types. - if (rangeSize == 0) { - if (!nullAllowed) { - filters[common::Subfield(inputName)] = std::make_unique(); - } else if (isNull) { - filters[common::Subfield(inputName)] = std::make_unique(); - } else { - VELOX_NYI( - "Only IsNotNull and IsNull are supported in constructSubfieldFilters for input type '{}'.", - inputType->toString()); - } + } else if constexpr ( + KIND == facebook::velox::TypeKind::ARRAY || KIND == facebook::velox::TypeKind::MAP || + KIND == facebook::velox::TypeKind::ROW) { + // Only IsNotNull and IsNull are supported for complex types. + VELOX_CHECK_EQ(rangeSize, 0, "Only IsNotNull and IsNull are supported for complex type."); + if (!nullAllowed) { + filters[common::Subfield(inputName)] = std::make_unique(); + } else if (isNull) { + filters[common::Subfield(inputName)] = std::make_unique(); + } else { + VELOX_NYI("Only IsNotNull and IsNull are supported for input type '{}'.", inputType->toString()); } } else { using NativeType = typename RangeTraits::NativeType; @@ -2393,6 +2404,10 @@ connector::hive::SubfieldFilters SubstraitToVeloxPlanConverter::mapToFilters( constructSubfieldFilters( colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); break; + case TypeKind::ROW: + constructSubfieldFilters( + colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); + break; default: VELOX_NYI( "Subfield filters creation not supported for input type '{}' in mapToFilters", inputType->toString()); @@ -2494,7 +2509,11 @@ uint32_t SubstraitToVeloxPlanConverter::getColumnIndexFromSingularOrList( } else { VELOX_FAIL("Unsupported type in IN pushdown."); } - return SubstraitParser::parseReferenceSegment(selection.direct_reference()); + uint32_t index; + VELOX_CHECK( + SubstraitParser::parseReferenceSegment(selection.direct_reference(), index), + "Failed to parse column index from SingularOrList."); + return index; } void SubstraitToVeloxPlanConverter::setFilterInfo( From b36d790f47b4df80caffb37ccba60b5b61b5b8c4 Mon Sep 17 00:00:00 2001 From: Wenzheng Liu Date: Wed, 31 Jul 2024 22:06:08 +0800 Subject: [PATCH 34/61] [GLUTEN-6610] Update clickhouse.md due to upgrade to clang-18 (#6654) --- docs/get-started/ClickHouse.md | 4 ++-- .../ClickHouse/CLion-Configuration-1.png | Bin 162137 -> 165676 bytes ep/build-clickhouse/src/build_clickhouse.sh | 4 ++-- ep/build-clickhouse/src/install_ubuntu.sh | 4 ++-- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/get-started/ClickHouse.md b/docs/get-started/ClickHouse.md index ab90f2312c543..12a662270c029 100644 --- a/docs/get-started/ClickHouse.md +++ b/docs/get-started/ClickHouse.md @@ -31,7 +31,7 @@ In general, we use IDEA for Gluten development and CLion for ClickHouse backend Install the software required for compilation, run `sudo ./ep/build-clickhouse/src/install_ubuntu.sh`. Under the hood, it will install the following software: -- Clang 16.0 +- Clang 18.0 - cmake 3.20 or higher version - ninja-build 1.8.2 @@ -66,7 +66,7 @@ Otherwise, do: ```shell export GLUTEN_SOURCE=/path/to/gluten export CH_SOURCE_DIR=/path/to/ClickHouse - cmake -G Ninja -S ${GLUTEN_SOURCE}/cpp-ch -B ${GLUTEN_SOURCE}/cpp-ch/build_ch -DCH_SOURCE_DIR=${CH_SOURCE_DIR} "-DCMAKE_C_COMPILER=$(command -v clang-16)" "-DCMAKE_CXX_COMPILER=$(command -v clang++-16)" "-DCMAKE_BUILD_TYPE=RelWithDebInfo" + cmake -G Ninja -S ${GLUTEN_SOURCE}/cpp-ch -B ${GLUTEN_SOURCE}/cpp-ch/build_ch -DCH_SOURCE_DIR=${CH_SOURCE_DIR} "-DCMAKE_C_COMPILER=$(command -v clang-18)" "-DCMAKE_CXX_COMPILER=$(command -v clang++-18)" "-DCMAKE_BUILD_TYPE=RelWithDebInfo" ``` Next, you need to compile Kyligence/Clickhouse. There are two options: diff --git a/docs/image/ClickHouse/CLion-Configuration-1.png b/docs/image/ClickHouse/CLion-Configuration-1.png index 14c739d870baf36243107ccf8e6202a9de0667de..a748a27e1d883e4c1571456161055976b93b94d6 100644 GIT binary patch literal 165676 zcmXt91yq#H*Iz=9KMY_AYyS~Nu|DCgE z_ng^z=H9t8w|`H_dnH*cbP{v`0D$#Y4x$18y!Z_OAWi`h;Uk#YVOj7W1V5g^_^}N4$03?%FIZjH^6V| zX0+9nHwSi5KCh+%K4*L(0YYv*LX~`P^mABepEn}0yT$y z+Vwjhp^QtB(#FO{5GooGOEbX;5DModS3bU5l5Kj~`XvqSCUvC`Gni=B|%QEuCH)IPESu`r)A}6sh*jB&4?_Ptrqr!i$MlGa@|n|6DOoL`{vyr zzMUhQpX+&$KtCx{(_@X>O7T#~w{oxy0+nO~Jc;N1Fe^~skIc;p3tX7U)#*m2Ff%iE zP|2O>%o-&>v&2S>8x2vm4$k-%*!qDzpM&ez)|4m1&yq?SPPIPorwY^3m23+0YntYx ze3gPWyHCiFXO_22VeZ4AZZV*u;d|fk>=w;&FH{<2i)BnkW#B+c9gRJ4JRytsnsgpXhhwA})o(c!8>N_losfrB7?V*WLU+c3UM) z{wHqd{B)f{O5yOGIlp6qZ2}$Nq|@8V+GLr%-B@|eXUB%KXU ze9qxM>hXGsQat!1iV# zD6l-)D8Y$^Yo3wMs5zuM=UE&jKv-R@+Dr4thL*dAAPgclo$8|Fy%wK`+nObxyR@bI zJkEzG^{-U+Fl{RgRUXuTvVQKAL2mXM zUpd}Wb0BmfnuH+7zKP~m3YP%F=w>>LktcH_fZ@(iPIuV%y1J7+gS%N;Y+s|Beqo7K z$s%NALc|V?xOd5z)Eu!GOZ|4tIPZ0Vfvs_^0kro`Oem(e2>SuOQ8`}ir~4$I8nEC z(+wCl<;@^DxrI*4!_(u{n2nzL8beNF^Y4aNiHcq-8+p$<7}aa9mqmyw*?gK%8t9wp z=n(PWDN2n1m@)TM->42;V2jvq4QKjnMp*b<-(n2IQX970zKMB{4RN#TYR;u3O*TRy zLs|ECEwwhA@l%s__brb-QIk!Nw-3K9fbK8RIvag&ip4WL2HrWmGMksi{ENKt|9KdIsby9-Z-AN3@TxLHhsbxm%l&ny!>=s=_|Z- zNssMwTRE1m6cqo`BmbWT|9CRZ*45rDPT_q%QbwQ%LYA)n%A~FPpJ({Ww98MMg@&6Z zVlv5Zn|?DYyS^im4P}f+IO`Bm0R=uc06(b1is=xhWdJmbux~ED3?I>l4xg6)E zSzN<5K~X&eapkpBX+n{^y32Y zNX*KUcNdMs{2JMom&z85VIVV+3~u~1a*JY^y0 z&uiF^k9JwNeacOzoKG9Gk` z015yDdU+ykwEtL+II_1~C_gPNJzmfGMB@6}Fd%cj%@X*tHT<+Ho}@mmrlv+FGGU}> z=BP>@1C;$li^)XStimG^9+~hPexLM=imEcVUP$^%YnlvFKCVy($3(WXc2>YGAd9vN z?&GnDvg32)z{Fd`p%$&kki?>Wx)`ZvF)Q3fx7!n$=6hE$EOMWQhVf&XsO4U`YG|tI zc=Wfdat&(wr%x%FiVV~gDeFni_X$qRi$1;v%Cc4p4+1?j`Zw^giN)oRcSi?@ha$pt zCmua?>w+}0-O*b9S{=w?t>~z$f^LNV8|j>zR&d8Oo%kuOrb(7&LGUP}sW>PtEw6kn zicHiwuc@KXXg*(K$@Qo(7e&%{+$7n&1Aad7^%e0+Nv^K0-%2|OE9he)-%~TF)S@!7 zv$H?q0jjF2ea?md3ljB`WzoxmkO0&SNB~noFIMC)nd_LyQi>q3|5h(z*(xR?b0icI z9K6+wO9vDDiU;a2IZc-6=HF)_ZC%F%bP{9YfsyQWD*#^@`R&c-XWXlYet(wljEmiS zuzMP=F_?DRR=t|P8Y=Z=^T+JKWsyeCE5laf8CQ6OW{XMwkp{ud&@TGK)Kt=txvZe0 z8xmZ~0|NsJi=$-Ef5{^}xc}@9Dt`0p3G3^08Fh%na^0hA3drpPDIQQ!VuYI*@!J@A zO4$t>P7JaJK-O;y2q*#oNO4!}@m_KQC|<~XF??LSqR?q^w=f$5!<@1XhSW8b8cn*m zqPOS>50pl#g9T}x3F*!B4-BLT0>6&j*M>p2+|A6)f-I z_E32!)Smz+nYuzeSVs$D2=tSDvc3@$P;_HPEQ@ARWM`3s+*ry{|7pX2_=JqYfx8hN zh-K+kyC|9+j%|aG^DLFzbjLbfm>`k!^&8XTh@Q2{*_f;)OJ?x1$P#F1fKWM(<(oeH zl_;qOaOB4xOwKeYw3gaVJ`~lS=_5~gGrkI6G5{FWEIt;d@Zn$7Yi4weU0=m#y~VXY zNLwTix7>Oj9>c})^~M)9Ae_)Iobrcs!GmRty62ccdb^D~Mb@IvwfmU3!T1-gLPcIZ zzj>TI6Web);^Qn^P?~?QwLg0@%}@ocn$f&6_*#Qo3qpEud$zs(LgP6yA(DfuaJ^iv zGLb{cATwMwj8nXhNYh&mFRtb})oi<#nb+l_^g}BZf`>;0mcH?uk6URx+e3Aw|C|a* zZpANY3CrG{r^M&IjotVd&u0x1Q*t zM0$`$3g_5Zzy-zhLfOhvUQY!Rj5BGMsIvQ+EA?l%f-uTK*H3@BCju?@uIH6^x0?s8_yoSe9dzd zLA*jOFyA4V`1t%o6Ck*Ed4HyQH?)7`!0*|_B?DS9fok#kXyV#8qK%E0&$Lvs1e6UW z{J`N{3u;b28BZTG3^0)pIjA~=LICJtaT1*f!Ec|%W^Z(mS|vO`#acoq0yAYuC12xxglyvN%q4ba*&ojN>o}*zdn2zu`^o6REq93i&($te`J${u_ai{ zNlDXi``iTjfYFp8e=+NuSTajWFD|NXNF?~;tZc;722cP7Vh3F^C3v6r*W$8QqN1ew z`ik;v{;eVO3VS&?*xX;({j}DRExoWNRz9NjqmGWw(tAzM)Abz=;nyD0gRjz1zQ9V0 zHM+!=QXT9=;I<5t+h?ZtRf`YdzwMx}RPI4&CPzkA(wWC2Lv87`zYVAt0+II9_qHvS z>DB-N*A?hQkz{Mo*&|&gVJ`xVIWmIBapM-S(GEd>*>U!I;1oFPA44SmXju`0TS%ZE zk~jB4G~p^;?RLn*TTo;=AhVuyc;0y*$RQvnvFk#nn8wE%@>5{<2feR8-G0{VC}=O$ zeK%IJO>(cCU(*&^HwgSo^fb2;MrP6ipmGZv>)}vNj5sQVkWBO9J;s--H5oy2Q zV>BhGp&Ky(71VZcaiBMl?jBw#H6BQuU$PtkDbHULgn$NIX;`UAl|vEzq`8$_dmg@y zp)1!5>;Hs3x2Xg%KXX0NJaIroc5fxUV5GSPipe0C45jGPNmqz;ev)FtycLt0rjXL!OFlYxO zXxErJ{Pg6Uo;Wdp0M$4cA-q*)?px)Dp~*Fcgb$p@Zeb=eL|%-tTh@Z7fyCTivE`)O!GtvGVA1Ea6V2ZNKNA=H z*ENBNlE2!)SoPv0=bKWs%a*2ON6>XBjJX^a00JLy{42HW?rswQh^4lB8xFvGrDSbu zE9Q2*yx8Q4pr)y*TxaxoVuxEciY(Ne^mAkKFE*SBnsW^my&)h+Km$Ek?$j|-Q36vL z!Xn5lWr&LsRK+z6HA1Hh#p69t#tH>N;8L(Ojr2Y%cLQNRcj< zbwqHdLh@UA5pVhVv&Z$yH{*1{#}&4gsbTG?Q{KK`pTeV~9ZUh$WLiJ~;B1RiCKrb$Q_7;=;4kB|0h@9j=FmY&T$_OYWIK1Mi(^Vl7Mwl=Ba5}9;e0nlmc_$KzMIC=#Bp-zn zl^C^7_Q@CR{}}f2QK`DyV9ykr@X!EJQtwDh+??%>u?|(SKdk0|dr8*&0g@lkv1ox9 zF70vXl*saSXe<+etx&LstO#tvo#{h+DFet7ofGL`if?vPuLzU z#bR!crXcoH^gtmpad!51$iGlT zy$z*sbO6T_!_C}E$2Y-V0`XKQN&D;L6}W=BIyt@n5bAT{afa=8(fppt@OnSWYk}o9 z&Is|edz?3-K+(RsRJcRGBHN)oegf}fYT>|Sk$c@Ysxp}`0IyCcF#SYA##&jnCyZTe zNlUY4Z=o1bweg)l4i_e^Y0-7&HHcq|fPbH&Gsa9pmk@@{h*E~^vz*+;0on=&pW+}b zQ(MVt86)oX0**UhA;*_u3s9iZLgXfqv zJm_G@(x7rPXtREI;mFo3x|Ksj8llsD9YE;j$Fx2Y_ye7Wa~%T2HOB%0XzazfXgH`; z^+g#n6=|jpEVvo8MXSyF1JUqo)A+B5#2-#h;_|{s<(IUTuC%5J?M&g-oa1cYc zdWE>CqB+`POhbOP@MvL91Lbtj>)^u@MPLjFXDqk_M>w>>QGF+kbXU2i>hcPI!Z+|k zI92)~z@H!n`9-HvFJ4(I8ZiS0W?*p{wMF1eCsl9`CKatcrx7WEvj~+;b5$6pk(8=i zq6feeglj<$P<)AN{+%P-KhO&3=uS3c?}MR*EYr>rda!G4aMoU$M*7>esUe|DXpJ)l z2S)%272Jx)XDvc4LFR9RukYUqCI%sohy^D+TNxP!nu!T@lh2>;sVr@5*cB<+M%zU8 zxACN5>@@jvAJa%ltUiARI9;C8_{I@llDE=HZ~3WdXaq+XNCnM)balNNFHlV}Jk1cp zje(L*th~+*rbhNBK=iL^zJr~WT01c`Th5#|XtJd8ne~%~Xu&kR)~2AMm$c(#l;@uX)EC+h+9q*m3*D6{TWrbEfGAm&i0z9=&H0BR-$|DQUB3| zleum*tk;h@G8!N?eX>|htUd#PSl-RtZA?&L=yb@?Ie6kQF`q&}nXB5sE<#nO}OTL7iySCirC@v?m z>P)`ZOc3(+>bb=3oYk&0XV-JnF|l!eJ<;IUb7xnw6y5&ljM}B1I{g^aduQl}Z6aS*?@Pae1GxogW)tdOd5#O=l zYRyoy^GN&6g>%(TCM;u)&(7 zH>rMg;nec9I9bDIJy`k`Jq=miS?XEQNSP!FcwAh+? zx@hWYP!XgN{_?W716%w)-jw(rYK@69c7-%IWH))S#~N7_PLgF87u zlwc!S8SAe78OKDHaqA#3{7dxNx{gW0OJCak@s z-2wD;W=wZxDo)Pw6CJzMa?_Faylx@n86?U>kIS2+FGh_4>zSUEwRill$9B#S4|JxA zb*7R`XzWI^>CcE*eig`TCE192n|6?rUhWG^(lKkWSQSym2|0h6Dp_z{a6J0S6lcZU zKd@F^TH|_mdBs;UmwYK`vELaIcc@=iYo<$!>SG~$7uMqQ*n5IUuNPo-{oHm^`CPVI z-<`}k!JD8i8Es`_gN?`sz$4|vB;viN{Ikzk-72H_hr;->4(|C2%6$~y5#TsGxsJ4c z^Pi-P_T=&inTQOx|(F;CjFQ0_6&&Duc3nx1s9{m zlYUXmY}GYjVU&Riq_h5g;G0o;X1NIba|Pgbz|K1uY)DnS8|Q3XIafP zKkDiV3JM<7E-$+Vjv7oTI;&hS)YYeoYE9m1w~IZEGJx$y6z)t&bTo=c zgnu;O5)F$|QZafpe+Q>cw|9jd=~fk`bNevpX{gt@?U1jC9SOVH!&@DAsicJicuk*< z>igpGIz6!|w})QD=SQ2UK{7^yQs@}5`lw{!Ct(3~;>}4a;o9n?{7rHyU5k^GQTcUj>N&Wrr!UF4z zlcq8|_f-~J$I~bo-0wf$9X;Jtw(N_$w<8W_@gZxWX)%^fwvixHBvXOdWfLRx%HZ|! z--$_%487+4g?mJ7dNaD3H)j@^d&}4DT?$7Ag2U|>X1R^^K7Fxou$-&JpKb;OYo#Ws zJJuCft8c<6J~s7oOTw@rbcZQay+Pz@q10(|mp6>RNh;4TZ}5NWdwAnv7?*2_P#o=V z7qdh~*rl8==aW9}P-boKzQb#!2`=U>XHqRKwfe=qbEqh!ssTUF9rQg5eZ88I-%$(VX0>wvZFC?FfRZO{|vAn6NsjCjyh*k%+=7Z&RR7eIgWC?$?@l+_PSOa0W{(VD}*fgt=RRcjgK!low0Ubl(?_lTRXrhHF_cWpIQk|A(-HfFx)v2ekk*i&{H zRAn&>W9U&#Z80oI-pLZb7$d)nR8AE+`5iSc{*XspTC|i|W)^0bWrt(!W5U^ww?pUhH9CQhs5?W6L+iHAc3FY^W!k5e?E`@9Dd{ zuXXdM*(P5EE#2?`Ao_ zmpsr|4m`rahD?a#kHz>wUys-K(Z)y+-`8d89XRdac#~CY`>pxHZ}yh@lYkJ_e6JmT z%5*N9TW40`C^B(%f$|{5ifoUS>@J5>emk#+uBj#T^!T1$G`fA#;EkkV(aKL#@N7=k zd#jzaBui*!u8ei7%;$lWl+T(`xx4UnY}Va;T-mN5Ek86sOO1h%Y2pN zRjjhN`N;uB-U5f&TK9#TMk1pEO36`*MW&$r`Y+B&`|B!RlK%>IB)O}k`9?={s7hoj zlYiDF)uvH=kI&oVTG|UmXc@^dPy`%SJJ$=QTwPp(MSq}k0*1rjTJ~wFxk&~P<4x%! zU^Dy2+w)2a5+@cfBtHJ6j!(nn)%3Q$K}gv(BUY;`Ij=myOdVVF{O_8wh{NYzb5yQ- z0!fLlaIa?;Qk%6aTIfaPJ&=3TBn`YsX(rnMz+oAFqJ1vT)h)-f z-NRx|2F`UY7I%qPu7{U6rNuUiiAkP;>|&?6!?Xy-0ZuH$3ToqY1FyT)qD^78a}_f$ z@WG_ltG#YKh7Btign^$d=&V6tWYAtOIA@yv|i4 zBi=hWvJQSBqnza%eHBh@SjN)E^s<8um-{-6*6#1#d09K29aiRdICF<9pLE{_r=m;G z9O?oNuKhBfgz39=OK5D1up`Z%`F)II1N!7ss&4jA|{ zOJwcrj~fkfjlR?0cBA|sk`J^L)CZ0jl-2hD~LXHUdBDa5|)Ue7cqIt;C)rA1PTeNTV|&BR+W-)iaJS$%V!j7T!T7CMMtp3_tAE zE(vKveLc`#(st@FW5v4yhn)9)!5QIX5+r*)?07gs*j4Olx0`HL-Fax~`arg4aze3T zC0NySmI}o*qDm%kxt)+Xh(zp+$)v)S8REW-Bd#U{H|NrAjE<2go`y--BK zY$xN9hH8@{8G^tul2x{NC{{qlfTh8!Fl`|mU=F=vya|WDVUANu_J~TdqR~$^9JHQ0 z!*WDvh$=$_dlYGsvxTrwLDARV+h-EO-o&puAenSWRv9T}C&7wJsh;agq5MR;lYh$z$amZl? zt#RUg09_y1&BXToBy$gJJyUHezy#%c)nvSkHNi3I5L3j01O{J=3TnD$mTOs06sp6W zAl6qvcW{=Jukv(`Ev`yqNDR#bexzb}_I#e@bn>HKLiQU*A$>nDLRp#KGB>@FK;_?1(HyiYsd zvWq_L*&Nxqwu8Z_#6#uqz7f2YQdwW`5&pffu&}(mysC9^P} zSd}*4UfxFN@Y@{j?d@#u@AmoicMqoX&UqLwzmA=hZe>f7e<9VkiNy&X?Mn3(ah8@HK}fL17kANvImtT7&1i^LQ)+Q9ZbHCR}rFFQlmO7}1xPduzDYqiUiQAq z5_9-6;5pa+!&DxYg40e(XH37nZ+S#@=5RhrtLyyJREHTxQyji*)xY?794J)p z>EYo~^jsz0s`Vx*XW$*oc*V~@sbkgP=^@nMk^Rbqt}CDhGXn{?q@*&ZDJO?^-VN*l zzsk5ViHU@?UyfT5{4lgP;L5=E)((nn0bt;^csx+G(>7c+>#O(@^)>Qx(Hk{NK)m{@ zV_}LtqsegR^pj&!w@EM~GZjj^X#mGwtA%TIaSl9NogsH!Pj9?GY`dzlA#QM+T;2F> zn_2S}Gd9utRda~ugy!S#(kBfO|9s!`wTC-wIcl_^c8BcsO&yB`ud#4HV|c~A=5YF9 zHTb}F(cPl`IJJu1+pTfyx4~m^3jb20#gqR#&yO{BbMt@PE)NtEhqcRC?CS~x^=ej% zdF`xscMcXWA4|-_Hrbm@CRV#@Q1?AA8J5=#Yqjpx!4Tr=6&jAfLFGIIxa%@c(a z`sz)x0zTK{F>CqAycWaN?@CB(OuGuhk8Gy@IG+j6X_S1#IpmI=ApBxD91|cLiP!(7 zZolEk*X?M5nHjI(kURe+rSM@d#+o)FdenN87_vet^>-|8ruyI^SHaHUkM+&h z7Z{_kc>1-YB2Sxxy*;^UIf3?ac(aa$g9ak=RrjzrCdb*iX3=45A&d~l-|e+^_cXpZ zgGzEs^uutz)?w-4&{ei}^V`x3Tu6$rQ8Z(CaqH3Ep^wGcq_8F%`TLi5QDkBs*~*_6 zy)9JFLgx~f+=p!j4w)4?i_a-$Yp#yV*TnW`Dl<&db`wi$JFwR`SW&B6YKl)Tb&}>C zu51*JzDTmkIX`t=`GTl`QAt7VYcIlEL2q?*td<=e$7+{uGD=;q&UyKwUQqc-thXWE@wHrN=(zT)%*>Hab@gR=?+x~c zqQB%Sw$Z!8rf_XnjLc4C!546MK46|44Nt6JRO?3B;a_qS9CJ2rYpla^zCbBysC?+` z)62JR7gWpw2Roc5f2L`KFD;#bwA80z+{oRydK?G~6s%xGt zTN-l=mXo`GXIB!XGuZ6@(fN(@vZL%?i1fF}8{_bS=9Arl4VQ`YPjLgx%)`~rop^+w z_-?v+SpIxkOG{C-?fY7H!JClM3@a9{v2!!g6FLs>LEh?h#OJ3Wd|8IjmHrla@@q2P zvhF0;*JBdK$j(cb%!^8sIKG$Jx`p}02ZN8;u6A?P#q0;%T@eE%z7HGCFm8;^p-l;4Dn!%J07Hf_slJxh_-t_v~9Hep;r3*Re+7+U%wlq0N^Tlab z7`k}BGpnyz3&@z`FVV?bO>TwOW;D34*Lg1{f2py$t8TVlelG(o65!*MrD#jpt`(tG z1koT>NDTtvh)#IHd#k;!hHqku-WVQdjHD@{X(J`EV?`|&0VRw@sm;;!eK!(ntGMkf z-)cneSPRF=(fKvUR{r65>CqOa{3JqJ+Xe^7AQ@RHo?wx6pd2Hkbkafg?PhC7hR<1b z>~-{NUgr4Eqc+)xw3u~D0LZ3ZeE08yISkC_;@n_y?U*z3_?g# zMpULUz1a$zt@?2CLqk2i8gdW9lOg_vQ}Hg{FezH`g`L`1yv>{)WR8F0=k} z6IobY#qkP-G!$xjEKnz}_Yg`;#L9|?O(8&M`koAxj$tJdRtZN+ZI$bDA+WUjFjLEb zJ7o|7i0q1{ATCU#vNzMV^Z-oSm0pf=|1bvtXY*SQP!f+nv4iYn47Fq}KDA-whV{gH zAD-{l$X!JfdDKIPiKysI`UM#a=#uX<;F69+JGeF2n{zUsi$O_V2~wfP5Ljy(fsClU ze1nfZ6sY**Ih;Ts8_KIy7<-b;sJ!g|pAoT-4Qhp5vsWDHWY5 z8G?jcV=%H&WIC9EolH;*7^x_r?B;~YEECb~qCmpv4l`(=C*JGWG7UpNLY)Top9QIOm{AabgcK0{pG#K(yOMFZVrkHr53ix@{4EFpo?nR$j*eJqL?ox_FV zZXZ+;+MFEYc4XG`XOdyrwTgV%sdE$s;fI0nZX5NQEml z&rJnv59aBLY7CXl{d*0q|LCn2qNm+FDPSzc=bj=lsg%i=Qn~Xw^Z*(FlgfFgg&{ z2#pd804dndD0vlnDFA)>f&t2qYFQ5%d$WpLOpI*7)L2H08XS`1btQ#@X+Mqh%O8zY z4UAw&=1%38!MVan(TxVs(R$zM;HHf+fY2R7P=Uvwcx-riDEv3$EowIgX)p&`+sxx* zIA&nZpa3FX7^KN|w%xL5WJDWB+NcAyEFqoXWnglny{M560Tdv^$N-oQT1ch2{|R8& zv)(VcqeyihJKqgtmNs7tcAXRQ2#O7sFnX6}%p$eQSm-R<^8);e6T=wsjW+cNHTdLo z3xw306wFNpHR|k;Oz0S0YV+dM@o&1 zzoO-s+79o?Hn1)a?yxS|mrv$XQk=7qiHUGhWA|S2OVv?b@<$-{|+x4nsXfgO* zVp0?+8^_q&=Hl}$lCJgi7p+nZWh$TJpXD%|Oh$uB0;E-> zyQ`))y^)ib`)xm>aX30DRMc*RlzW1to8;p6P2}W2d0QMJhjuoMY9FZ{J{ZIDv%n^B z*tE!Do#NHpb2n~}oRZ?NLAxS~+IhXNXz=MBtD&k>F&o*}ACKa115eBMrD~~Lgx>^- z><+k5Glt2mvZ#|O`HKKfROVhv-=w();q*!&}DqR7wE&Z?A$UHIvg~(*r&>nA>`aHsCnNF zbcY4pdheHtGdf3JNJP<|v{X zb$?{!m0xEuzYgY>53uXbmj#EJIUG3GHskTXzDQY_QkzDXWYKkNjBGr^P+6}L9#^w~ z76U_0i~=yUxX~oTku|ctF$}vfNgrGN+M-aygLJ3++8TtUJ`su4D!993yxJT+tWhxE zej@uO^ao?^z>!V;52?s?7)R~#zi#`uJJTQGn{B=ix~hze(XXUzY%J1JQ@bO`*>9X~ zrFu>6UzpfqyWf5^Ft93p(YtY9U41!<9F3uVtwdO(l9Rn3wNmo3V`5#VWQ71#Lm<`H zKCka@4{zhX=@hgnD!U!ang+j0+Ic~-CknWecK`qApI+5#k2PhI;XOQ zf5CYhBN|A$V!K33v*g+nm4&R;UT%lFK$hyo*XUwXh@UV&H;D0RiE_PfQ9N1Y~;wV?@6;?S~s{ljW~txe4{?pAi2+ zT;MT=9;b-k8Itl#oV(tslXIpTm4KZqssDc(ERBuyqiOzM!RC2v_p?guxe)}#sdu^YQ?AD9S6)Bk_8{@$Io#U?xR<)0qe+~7BmS(C`;nC6-2CeNQA zjkX8rok|B1nQTvrgJ%ZY?x+ zKj(6a{oB8@nV>8jJ3VgC@Gg~S+*Hit`nacw`9A_oxv)2?BJuT%K|Ra2BD(+Mh|L0y zK;Iqx%g4S;@}up4s|1b0-=W+pevd>)LyiAP@RjQ08#T?}@O~}U<-f-YnB!$6O6-4j zl=>is(vk;FLFj0V_N9MAcRzmLs{0?q;e3cHr;2HQZLZBD9K;YsLqw=-NmkT(x>u(e zD*HASt=$RY;dn<5*!ES}UX$Mrd!Hlu1#s>)r`uRi1Q#)9Vk_N48pf9R)=Ps}?+%Ww zR$eIGEow7-n%>4CmYVg|OncjD*aGQtp??5)jO3c@DRe zQ4aKxAax>H{lZ#{{&uU)k#1vo4gHF^E@sNp?*JfPoC64e{{j@eAQXqThJg~!=z>2a ze$CO515Xhf%h$S>_&hgjgE`|%qw#6pOZ%{fDdhMDm5sav=^zuf0UJI_Q9kLfK{_Rn zk(9)}cv$^lb+EPqep>cyi=*7!lfO|B)n(5w*@!U_rR<5$JDOc^lKSxI8q?rX)T@YY z6As#6K9Ek6yR90P9S!@55}6<)n^61X$QnW&keMR*i5bfL!pm`^InYQ0U)UnrpWH*}>>+;9 ze$$CdPs`GHR_iwgkE@DSvhM{<;}2o>-_Rt&2_ii??{?2Wa?rJaJ zP~qyckU@}Wh}r-T7J1>vMlU@t6ZH?ieUu(?=S_X6^T3ILtT+QY5?>QNFv>l42?n)z zY}E>qp1f9T48FJPe&yTrd`a05sQaHv62}ub?n@7PNKV14{2o7r8A-ZCZnR$@#c_s% z!=0Q^q!$SwzLAYAe+}@&uhNMO;W#UG^@;ujW`mEFd~tp0mHKVS!9>b#@S>0`15d;! z(fX$z@!JotL4BoJ@K;17L7H0fhRN-tMj#?zEnIS&lLoSPNsqb6Y%I}bY=qtTp;HVW z6a$e<%6$Bp2AOE`EIYd9=v}^Z(6#x!dbo=dU-g#^wQu9V=YHt^#GJ@#^NTd@U~lH+ zaL&%gLwI0aUYUjFZ(+kxs-VfoccYb^+F8QG)kiaRHUo(^gWUv2ugET5W%UGNc7U%n z$f!Bi-!M^_(qq~H2oW8OTH9k#Tjc>|&=)vy2uREQq5UVUtY&8BUj_ncQ7>uHO)ipt zAVfd?gp&J7Wy;ld#SCLP7eIk}O4ze47sYl#STq9rFZe_8ME}8%!@XCJ5OB(+m?smw! zAsJ?SVFONwHPy;ORjU9FUpYtWG8Lh9FKR0;9ys#zf;GJ^I-NZ-Drrf(9~PM*7{&Ar znbEc&YiFzh-!UMu z^%waQR3E=>#mCqqLF|Ahb^tj(CE~BLeZeq(I?mhusMR8#^_IF@1wQ$pW7)SQvebThDv?N;-=gD{ z1f$aPM)XZ(61w32A3*2_;e1qZ{@z8;cgxWK0T0&p_6L_+<|DA+*)WGz zbgkr9RKVgDQ#7)jN;chYVXVB0UZJge)zJQ*16clHVa38jHRa@GhtluaZkJi>njUkH z`g(Y;tWd2mRxOq>BuVZV7aK$;i3|Tw*-{~_XArMEUP_xX;7TAOz9kn~wczhhp)tz@ENUXH;@87?<7(~ihVi|vI2wx~4x_^GVFgQw+W#k5bPgsYI zFXQpvdeodFKK{36>3oHe-W&I~EL>jjB8O^^5}nPQagL8k$L8%s?i-(6wX&A2&Jo(F z+1Z4hrSZRIpB)y$PmMe9+uBxYmTn9e@86K}6=CDwY5(?L%*2xwYw)R*_C$9%SZw-f ztMT^7()sP`NN)m$MHIO(Va<1m=r1!B^g8?{r6nw835_lXv#695)%c&NHH-Djy*q9T zD`Ms7oOX=QLh$lc5=ZT&000T;|0C-wqT1lPuJINt?oP4d?(VLIQrs=U-JRkTC%9AG z-JRgYo#GC~9scxr*EjmlfJp}7X5D*i@3T)*Qp&SWc6lmG$NgKLEzwENW>axP3;Cuw z7J4BCRHi+@uhn~A8sl1%`Stax7E7Ap!EJUUCY)UPV#NmJaGKxcIN5z_AN#wGI7SH}|@5Pb=|ot02q zo~pNVcQUJ(M`XONw_^rk#MiOz7pLdDGn_s1+OBg2ylNZB5tPZ;aHAUZ+8nl*z#-;< zqxGd4^Z9MUgOyLH2=9NNMLr?Oti7Esf>r#VG@$G|Zx@0&dd9S*S)uY~+d6h&OS`x; zclZ_SeVr!An9(rCa9P0Dz__f$mJjEZIc!;-ddT5X5F6`{rGdb&exWm%gQBSo6$&k_3rw~zRojebJW)r)EoAXnJr9IGq5lu zM98umrzTCUbpeJ>%eLL$Z)&N3*=+<76sfvVrtc<1mSwd!jaSh7UUc^Bn9dfXw!7Ki zuSER)TT{}|U_2HZP^coZZcij!ZTuv(p-N}+6jgRG*Y(mj+x;?MeIXf>T)r@$B-d0B zQE2 zC`Dtsj&+jbXOo(PRb!$9nEk60nKm#KhN^P5a@?qqx0M%EcNJzU5QXw|GpoX+mMhCD zih^b1Va1BwTn50a`u@AsZ{q6fXqRVuB4Ya04K_ys3v|!HB`yy3M^#QtmG7!lClJDj z)1q5k*2f=VuY*OX0&X~-1GFFFeno#R+qw*8WTry`E;nSKE|uhPAqF{*_UBm{2eQ>$ zOhT#!vX?!9vqjw;Z4I6RFX@PGBf5k%o^CC9#{*>3Z9Jb`(|7S;x+TOoP_1u|~yqQx?I$lMcAT)!7hE2eME z?pA(MdNLvtrZ%*|ExIQZF0?JGh=2Z z#Z%o)x$WL6$INCvn$2J3@ZHRk;QD;3Fc7Htl%zOQjjK3vc}!yXLvXd^oB3-H$0#|$ zUS#c}nYv~jn-}&@^JCN5}@B4E!ktH5SSS1+7UX%3|oZ=dt zDm_wse5+1Z%d>xAb;arV!xE+{$ouN33ys;JrNiiU-@dcOeK}vs(lV=aW(taGmCXO4 zWJ;GmwbI~m^jZLy={q+->*SAvhr}!YuQ>tx_j4^>U8%D^GQW#P90fcQdqG^K%i2WR zNn(-1d+*M`x^ISFc77nsyXetaT?!@im8V0Mi>!7ZeIocTd)9)U$GZ`qyI`uTf1S?T zpH(tkk5^S41CjZLUl035eEk|vz_dAQ8r?>{L7qQU^f@|0GYF4Sjk5&YUJmnuabY(o zkx32JjCR7j=uWbl{oSvJVn*AfL|!%kW2sF1o~I+KCO1Jfx$*Jwy3KZ#-&=Q!QVkR( z`}}?0(bj!ogHW)hf9{w_XWm=R7As}&*hNN(LzBeZlGp(oaB#5UxrHy`PUgrq-qvG? zc*;(2eK%WShE)bQVbNxH0n&;S@h_GQ4G%*d6+#>Tb#WyoMmCe-l08^NK8 zoZ9(z?2Zd@qCA%Sg9ZZkD+xFBV;In$b3^!Oj603iWZ@X?=q6?MBl;OQ?E7&uGwK$k z1c;l-PjdYDF|7>^oai5ooRDSd2#H8ZyN2b3Hw0C`uaWZ1sI-t1W~7~FR3`uPVNqkI zp{rNIr#^x30~hxq?=Krh4iD3L5}rIpKPl}Ut&PAh%fL{bYW-Smy-p|7`Kvu-ytx!y zyrIv80Z?$-b-1|;C(X}4Oan3b>TQOS-wcA>cIm^V008f)g|FsP)_dbNO8G54rm_&X z-Hj}oC{`2N-GfKBZM@N9eSNOP5cQIc4WeOKk|?(3*Th^&sp#!fS4W&Gjv;d6si42{ z#s(!Y8OM2`B!G=Cnf08os>BIAQ9}@R#ehu(wRMR_W#qVXew4g-yD@`P$ zyEYyH!Kaw$LNmF+=(7BvJ|PyL_JhR>9C(;O@!8KeC!5dAY@7r^wpY(Sq42~AV#rNd zPtET4tJ**hU-k4th!Vlmgf2IHhe6QirM#CgWl4!e``ap=_9naE306QeEf6gMX^Ufh zKTS1Fo>;n>wUoJaV9mBfWhyfRZu9w#I~u}Rc}IPjWy6gtQkYqCYZIkq(88Vq%_6l} z&9zmGnyW>83${+t~Q*&H6|aG$epEL!~972Z8+W>T}FH;Yv=7 z#g00Zsi~<0`zXUOqqnwwId8$^t#2^nO=KGkT!?_^* zEDDfMA5|0_o3jPGvCD?h8kR7H(!AN1d0K4?vNS>Y`yUSCx&a>)X&dau(=3a;MRiY`6^49sD$D(4)w>NZbg=Rzk#pqBg2lwRXNNCT zQDc!N%{&dXsyr7@mcpP$BwVf(J!wqQ{v6Xrt81$4w0dTd&J$@aUQ(Wnp($F`g7PTUTFG zOAm^K5apAAsWs}C!f%i^eY=xI?P5bge=zLl&s(y6%=9-9{jZ%E8PmuPgIN|=Ub}Db z@}ZKySUmRU!06A56*@%v{NFwiNmqJG94kD%Y z2N-C1+Hq-KY*GROHHVzNYPbWUt%gd^0wF#Q#_X1hG(9RwC>NSJhKyR?=FC&SNz$Fz zgyBARwb0!>DxMFs9@R((6`{AZ-+~IdAnn)^082z3Eb(EyUZMIA@_b5iPs)E_1sFJ;FjIg7PU*(m%!evPwxBSCZ{<^Gnv$rBCFeq&FBKfh%>Pu<2Crq6`tLy z4sn{s0Q@ivq)_44T~R^ODdqiLhHnsqDAh19Jlw6?H@l-1kCcP;>O6Yy4gDaL>+|t4 zg7siAD7edU=c$XO+A&6YJn=Q_3&6#(u^J+jneg-4#?v|-3N|R|(8j*o`9N8H?0O=# z3AjF2;|cKf^&Q0(&bhdTus}%pxbaV$ZQILi_A~6?3PpxEEVbCwQJ^OVl+(3oZJ9oN z@U~4ZD@%VY;PEGpHr^knmky#}M+-5+$5al1W(y(5L5$0^?`rTI8=6>k2d*^q&d?Lq zzYnwc;~}H*@E9&NnQQd#R}w9V#LvxK+-C9nK?XWMFB8c{pH~T7S^?c`*U>f~uJ3xa zrRGt(H>YIb<;xcq>;*lx1;qf9H(x0}rsD$^7FfR5MR!mO=S<~FTG1^nxCs%*=GfaU z*EQxDUoX%u*w=6tRrgrZn$hquC8AeEBqL-F0~~5L>%MBvkMnH$a1ckNPF29)zblz<79Qm`zBgTwd^FJn}an z;{fdTSoSc;;ujcgzIEXLs`1kehPPH;kJ&yRR^i!~4mHcyof8;K9GS-*=C|L8@FCSf4m zxUexl6J@W^Rhjq)x(Sj$x?{B;oM*GYOSeW)c!P5uaVQWWmt$6*65x?GZA~6G)or?Z z2T8oKJ@WSm}he?rJ9*{ccu6IU8+JZ_d;L5hp z#O}D@Z;~vX2>bJAc7BQ5+3|^8tJmczHfwqkrFhf$_~{|na(V`Nq1>%_$d_6s@UL#cqM_ZqUS>7B+b(lj z-krMEk8f>lY0xU)xvYC$hFUJzE_?1M*K`&jQ{kger)%=^W`@&5D*CI{LPP@Tcpci$ zR!TKW!ur_Y3od0%>999i>@>|iLBqN6d0X0y(4uvi)9HdpG-5LRb?BdK@Rvq)g`(t;?y8+OmIcNlw`%WWMMm&@U%i`Jl`JFcv_UF{xRO4>NT zwnZk0Qew8PIbC|})WQ*s^DS}@W4nu><&3$ET#k*{bs z%^QS`bG*XjlWvm|>{53umccVKGooq~?1bnJdVkn=7DUN&<#_j_OMmMeNJCp;O61iz zyXC1;aS-^W9vrhr794R}qm+cHN^b#Jy1#y-_j|7HCXQY)st!839eICb@%K1iI~p>z z#J2qWh1a`&ge_|KIMDz1A5geQP~2gm{7&3Q%4@WOPk6cd-q{At7L{V$3=G&2>(=(| z4tH}sJpmI4lF}3m2P9OuC>4gyHkW;EusJ@2galtl>u!DlPB|qUI26?7AIE3gS6A$A zZcz_wsReIl82$?d%mj7PKOKd1QR5~CEx?&dQ(f4n-OiPl?xX@q=j<`oAK{l&|I*23 zsogl|D}9%UUkvL-S0rTga2cH>fL#7hlP$7aYa$VOjUN)%zRNYUQ(JZ9`}Vvz6nQ2Kj%^PmM(d?_5pdEY@)`$x-K;yP($69snc)< ziox9-Qpqx;p-2RCYC{C&Y%q&2hu&{2G(C;p1-d)EjuLRg24T#!|E_E#_}}l4G0|y~ zbv{onkJ+Z91R)?xsmJHdn$qIdL^MKSaY|_K&ZCH-*W@KreOzFSrV=E-JLauz0)I7Q zrGgS~f}_^vh%WpK{r4D-L$_}V4i*udPQiteK@}d+7{K%$gZ1e-Z3e&Mqq-V@e-sSg z6AT<~1^gF_l%GiBg@}yUScp=wv0>BXGy&R9`)~x4gU(f=TAyWlQYuKIh_Q-B&?-A* zgIpqC6j^z{eCNm@3}XsM4AX+s4akbEL-^tWf>KYgmEaIXA|(o#gIg`Y(Jl;0MVuZD zNKlpiQAD8Q!Ko{(j1I`?P=?r|Lt3%ebIS~uK@8U9Jt7?Ow&Cd8o#$i zfA~KluiN(BNEaF2*)^^<$#VF^!3ZO2t7z?~_3-)%KjR*prVy~T^7~%$D zL`Wh6(7nv zh-s;|@uw205=6;Vp7P65iezMwlmRMYkK;b|5@``9VpK;gZJ={R_(yq+cpBNJB4t^V znk^LTtjUZ3c3wFYrDBaArx>UM8g$AJDI&M0_083%CqMy)WLqmGATnV!8UG-!Rk}oB z1)41B>M$?3&4=2icbW@O7TjnQAi2;T%;9^UIWtoaUj7b1Tqsx-4HhlrWx5*+z?IG64bBbS$sZ@o zWI}KK)^jwVz*;{&Lih$Jh6qy+rvP0^a!`H~OQ=YRMN>d9DFsVT93zf^SUOVsVvU$b z03&XGo*q_DmF#BMreUocFuRBg6Ic7V1NaOWn0RE8ar`Unq`NzrXqoCPxOslIZxaA7 zP@pkkT2G<`cy}pAJXoJ7we=D_-5mgpJ_yNG_H5U3FZq^Qi5>{MVlg2rPXF|k_*Surc1=c#PW2p$;x;O^%;E=SgxorQ9|IC)rcw@WS>AM$cZY~84ep0r4 zPGbjtER|y6%qrzPs~FFvA}j<$|CN2G-(P!j7l)pNDZPby|Ig6QR3@ zBSUpqeEd!aN?Rz6J>E`ZE}9zGc~?9}Y1FiJZVN8n|2%Y|@!RP;R#1(fYO8SDZ8WtS zf3R+RPmG&1Y=KHxaeRW$UH>WfU;gwXBSLttqVY3Uo{@vtV})mwZh8jpg615vwvT7a zVgptXDiQoBqi&1a>i9}%80!xSiQP+qD{byajs1>605_zWQ4|yO_7jl>GQPA+x5T!* ztKwlXVW?)Pp|NvK5Zs$xexRZge3ssE%_C^2CDc7)DXZO^cdnpxKpC7@iF(rM2gy6) zjE8Yv<{8{3^=r-#dxx@C7f@WXTbv$)ijRvJcr@o19*{$f(RgzBmfhPG5H~Y%hyb|E z_vu}#H3ergd9wZjzy;~cm#2f#vID$F!Jq_*-t&}V>Gte6 zPhR`;)sU(-P4Ky4RCS^)|AyI_!y%*nFHEbNBj@!Mw)3L^^>b37J~*d{-eWq}+Ps;>g}; zV&O&tfI{k^en8stncT#-nZYVv;EaxUUbVZ`Ma2D zm1u!Qz{|?tpS8ntr|p}8-=+V{jtD$ifnvGoS8up9?O+!-QfX^qAz#1Vz9M8?M&i?x z&4#Ur`IE&Q8$ovVrDnhNGCTv1txdn%RR=+e+$ud=zIhM5I@`hg^y&54=8GMbGmkS)gihC%~?2$>k~9h$Jp-Xnx7};y-|cR4DU{ z++bnIccEsQ|Se@bLstf=G$8MN#x3~K)N?Ul#N0*%)*pKPbH1#i@xazD)IW(toI#6?Ik zXjK3__Q$&nIimAN)2s5d?_(6;4K@=RGmb|O)_{+K#GfWAzSp_jE+7Y2#s_YM(CK!$ zxnG{PS~BY^(A<=nHv)J>JRkl9hUjbQsV$1kdB5t_8)H}N=rk7_AN0ch$Z~utQb_f> zXz*~+9!8)-CbxzD-sWNrF%1n3I&2tKb@e(3YzwI-_V@SO&&(9fp$R%0XMJFXShFQ)JZc^*6RY&K=$`QIS5 znASX7<`cVbJa2pOZFG6Q&h9Ab=DcayUsQ|tl72pTB*Xt_tPP}xt)0NPaXzj_m%Mc- zg(mTSUy^3K6>^natHntYyo`+gyu;hlP@*AqM&LsrVC%2}w3Zum*CM721ZuoTP#+c4i@8!nC zd=K*nk&LC+lu$1yMV7iCueZ1NMnb$83;z0}f#1J)ftQM+2t)TXtlQ4O)!i~`_?+yN zP`T3Jo3Q&{&mP533M9*^G>-SDL0<_T=Gr^gPbmv?N!EATZFH*Iya46pP~@CjLy_ub ze1eZG*_|2|mTTazdWT=qzjL{7Tsj|o;9u>*RsumLoAK!?i3yJJ!gc!BBeIx~Wv68=5xQVRTpTy=>>OrZmyR%FyK2hZD%A|4iB zeveMj67Yxt_4M(sVlcxFK?GwwXa_uAidcZ` z7XrCLA7Qdu{jchJ+S(6mv)%?P^&AvP&}TT8efa_8MHJWsadNTr`)qdE*6z`@znux$ z(dn4)GQ8bvy`2u2Q6=JUK5ns~uyTkgz_0-lP_TMlJe_&$*R(a&q*ks7wWwC^q;MRf zO$Z%02XQb?T-L5UeA#(0C;m&+e@oO`=hgfIR(0pr6x8bnGwL?16J%>PJNLf4lRyW; zs1<3{Amyh-sM&B{gY0j8#l0fg0nU5{^6ZH}4XsAk7Fu3DvgW zCu>_8EVb$oGDOs|y2Q~s)P7Ye-$42!rnOqn%EG;Azn@{@Po2W#$yKo8M)8sG1en>B z2*^!B#Me%etiQEq#ptl@Y;1Iz?bcUrPh^ordAPFTLd5T<51!?j$Xk!gK#I79B|HA{ z0xZ6Us@%R|QDZzqW)OG@b-wG5(&S~HL&~*E|H1WXnj)Ih)#nX`sx|NnSLUCFVOp&% zM{*K)Ig{-NE!HJvzbEjph~Lgs_EMY1M+$B2zhAW`AI_U|Yyq~fXC==d&x;L`$X$GQ za$Kk+6DaYIG6;gSD;O(Hcm7XXSs9g%YF5Y;%dyf0=~;|he3Qa)Ezg0{I$18nLE-X| zzM=5hs}f?Ud74*;@k2z-&uNNbf5?(n^_YHUd!n_pL!3U_k)qG*|dR&PF*Q003l>6}Gi0TF6wLaB$G=i@Smgj}Qz z=0Lg}9Xm8W5ZWmGhziZ?E4gx%8?ps_5QX(>7#9NV>dMM?tp_7midwV3zw+n=3TI`!)~RYK7BI!fAdLb2WgF-Tif{>#}V z4yIghx;6(iu!W5HKbI~;glb}#YvXpdcWwFfw1SAx4^X=tqD`5q0Bv%xVk3WmFOg>e z{0O43Ad5E)hGP`dq7oco9AyBsA$Vzf1KCM5WWA~qaVQ94P!KDTeBZJi=H}RfaL2X_Z3V8 zm0k6W7@^go;+xi?^FE>R)q@`QxwvGp5QF5U5m2(|=;)5mq7f&8lEHGh0fO`bvZ+b& z#Gxs&U(9Pu7y~gAfr!-0q^bU&v5~lP08Y49PYP?A%xT^fJVdJdscQ29wTo0(m6<9R z&XgeUZ7+uwLAz)#UB#-2t%{L%nmsR1#Y^aseU2Q`*btB<6-S;%oZE6d8{8J;IYL@E0z0+g*Ke_e2X(9|LK$akJd<3s3GbRlFt+8Qe)g$@LnH&_;wMj zaQ0ZMpo__?3X+l>0x$I2faQVt@p;N|ijSd$Hr>qZe$=td+D)O!6{kx^H3vfp$x%D; zRe@aw5-O0^6>q5Qw*v^JM(Y%5Hrj!=%fnBpr!3Y! zt4~?fm(@wKxL>(M$U!gOYZNGp9bt5k)z!g;Sp!Cr>W?S5=5hPc3S~@k6!oD!3(U0p zF!w{^v*Zpn+wf6{@5XrS&B9l@~N)FKh z`+P0Ug5@|1x^*e#%uds|IltxB+q=;l8gIm(5_UN(KyQj!ub7KQzCe2x{;_ktE68J2!lJII1M*J#fcci1k#}5Bp5i}t^F(; zJ10`>HoJeOMgQCFR{T1-+(U4XCLF{UFZ)%K`El~3T5R?6mesfC35n+oNWux_X*(T} zXo>j+(0^=>dVA84+~U|-xo6Xk5KL91FlA#L$p|-*Uz6E9kbRkKUpC< z6uPhAXSfAn$}M%-Do|mN5uTz-ea4+Vu$(%F2o6HFWj)`M33_f{i#k3!vvlPq#@Cl9 zuN@teR_?63$p2R(>5)>RM1;EH)aGS`-#)s8$kG?03^^>$n=3F_OwgUQI$o~~LbN0K z{Cn%>$F!_SqQV3;@<;>m8(b?Nug%ySje0w97*dNMulXo4?1yZITAC^tXDroQ(a@}=z4S8ip~p6{kZNpA1mkai+KSPS)@aElX#DLq|_+2^e_;lMh$fQ_58k zgrSO2K{FX+NW}M&mynQ)*5JlE&}N25s9hjPQQ;>L1W2ii3M4`y(O~r)g&bm@KE!E@ zBauQ9es}`~G>V|xgrcaG3zwL80@2uSWdX%5Z%z{NP#=wgKL7eePLyN9!4W_1tCS@U z$C(=;%f^OMSpVJJWkL=s7!e5pDj1tXU~Ez?vxTY#D=I1C$X5{C=6^YWkj=rkt*z~Z z0AeAz?G^0GpTZEB9`ZpNSyqZOir&HzQ`n$Ta#Y?+k}nqOUvo#`8fXGS1K9cKM@9!S z0w`=BY(G-Md@*6lMXBWKrNJshwS}hC_|XzD^VcR#9%|Yk3(lw?O_cnTsAUQ~HXkCx zvMQDB6jr5Lj8#O6G*|7R@MPh#efRJN=Rs(A)}$>!y(nDp=1qa zgiYy1G*b=;&kUkY`h#tbusg#SIjb^j)1XMoG&_bt8iQL0H8zsTC0&W1g2-xJLtaO< zsveilotfzB20oKhw0LXz@g?Q?m{B?VoVnY7xCLwkHJ$`6;z1aV{AA_k)rQrK0s0Ug z&KU@zrsF6`X5PHTa`gO5L5YwSf*Yn(O!>Lecc(*_^9?*j4WK4Y#J|q&f?vMpx%V}W z-(2CpJ$=IF)^R6&*pjkAg;omN!DL_dRpeRHz~vfTakL%%MUSmkl`=f=0l3W}r%~mr z4gP4#R@9D%S?GKleAVGq@cLo&TIlz((9mJ$0hTg_)!pw@;)>RF+fS(!`8yj3oRyDJYSz$0b=NM96 zNCSv^p{6~(>ztd*YG5i7Nf>tA43`NhH|sj~k{uLBG?M7&x`1c_gCTXP*?SdG{*dz2 zJ@UGK9W}2PogLZO9z`exHlkEm7@J!F+q|HSmaUZ@Hj1et<*$f~c+&`S%FyWf&-Dgz z=cPqWASiaOIM4dx8YUYUZ6vy2TTuq~Ot{Q(Br4hol+&v}pMj;JC#m-VJ>x-e9PRD{ z|HEAv%uQIHGRnZPll_=Ij&Zm@9g#?4$z+sg{@lP|DD9&qFj=o{9_0~&JN(E9X3Y1q z8YkDsZr>a{>88=WZp_6t)|5TLc0A{5Q(W*iF!d=|7Q@L{O1ZTB=7ot41VAEwX-V8b zFvr$MxXJ4<_VA+nS#%Kx1A;e#w8_O z@F+g!|7ibSG2|zf#bcN7?Q0FER)t;`(Ejf5>o{FDkA1ea%<8JWE~N^;Q|gaTi*+v( z2^YveQ??%1Ap(YjJKY0$}4?_?8ZFu)q(BQiOoHQ`i zQUXqI6A6zWWzf$RN-LtI&wK2I6wz`LMsW35<${A!j+F5-m84aV{d-$dH5RhHi3Hyx za^D~;5Rv0+d3SvmxPEWGzKXDBcAyzq&{!+H6IJ`3=&ovc=WQcN^ye*iS~5F_5CKn)sr_G4-~B9Ei7orpLHPaS3Nkv#l!`Ti{LXXem;|>P z#$Mg;LulU2+;^M6-(L&KMDj*uu!UTG7E&Do6hnY_shM)gf*kawhB=D2#V#=(vZYxZ zi{BrR6yIkj$VA*0#wPeL{_}KniRYUZBju4?lS{pP-G6_@nb~bxzp%6l<9WWY4Bc1` z-S-YXq^bH{#I4y5rHJSxvX$_CS8|%7==^QALHyly>4QSnS(nUbDs95*?x*9G$K;8F zsn@N7Df|YX!70eX ze;Q6ItebV(|KN`7?JP%KI@d?qRYIbF%Kz=!9kQ}fB8C z1t?cVhQbh94JoCKt?etluS2o<82%#InTh%Nyp7qS&ANX!2F>Wa&m1s8od z_UG-;H!7}>5bE#NR}kpvW_yG7b%y`>5wsIC%$cEse`;nOm@qw!LIFgVl~u-+=x+$g zlf!KIL&c`vXM`D`R6)&wReBShg9Z4E11G_MM+0txApF{NK|y98m2^b00G2osRsj$Z zt5!jWT`==`8uyTJ6^illahCgLGnawS(PLd7WiM9#xm@tDwgddnXc_yp7B=z02%>ee zoVKSOvW}~jF_xz0MuS5oBg00zxUdWjv^+F5t+S;;NaYtV8u zn@{bcwA<$Cj*w^EZq)oBne4{IhSG-?h6ts(G=n-SzQ*cUb$68M#!K z--s0gXS8nGlDu;MQ!iY=c^xcAEluk+b>fMOqE<6v<7laF_GEHmiG4z6@x=)weCJlr zN$eUhk}qenK&8Qv;|GyL&&^HxJr*4x3u!4h&-Mm?U`mNp{HfROH8>P=o4i+a`FCV* zpJP}#s}}pr_w6xk9@*Xi@H(-2fjkscq+swoIkup>De^iiQn}vd`_>1)2k#e05eys5 z#Kcr@1F?eyEy$!3$(lbow$h53aINv|{!=3a6}Ot-Ty)+(+yd~dHh%xRtD(YDn(9vA z=J3(x3(&&6(*!B4Oep2~Pi`xvH zDdc^(S>z>PcTnr8(AQ6!7Hw%JpnDR(% z0^Io85@gGY@5PGQolN2~FA0xxV$_&E+j*t&e)T)a(~!`&|M}Xk@0lot%=hDFZ8e65 z@aJwnBGjnhN+~oT9l0s5SKX(CtS}bJTafV7d9Mw{@4xsej9j!YP$nf)N z5~uIjw~CeLIc_vealkBRr-3sSaRftIw(j*6@|0FGE<3o*6WJmON4{xn z;B{_YJe_pI!e_FrTz%5K+(f!l$IYPUl-SUst zri0eH{lyu>2N$PjTL5G~8%|^_Xd>BRnUw0< zyizD#T-Kg$3gVJ%IUa348mfWYP>HmMA>quPrTKO|OW=8``~BsrNFkQEea^MyGjVfd z)TrfDHc7W9U?Y=0@T=~sk}eh&+#K!mo6GC*t^MntDS*v^UZ=x(Fq(*7r{~o9y1;?f zH#{$MPKdmsaweZ!?)1eq!l;4oQL86T_VT=DmMsNT!daY%*ZXzKqZ;oe1Of%^ zz9=QgD=0)oMp9juZ~A6|(X0Xs@uX`vXNgb@yXV_&ZX;KukY&V!IiiHLes!NMkYsca z25Bp3xzR7r3-Fe>bq$*axynq5BF$V}A{o~tWe_%%D(%r zo^c@BbnND!8D=*qvyspUbs=PXCLwY=;o4I8LxcJ;M8>eotCY_M4w!(TD-j=pX|UU| z6Qri4|9IhlwSTrh^?q{UPlgG<>3t@UlmbK_XDm|8_~wMdrXks41%n~huJMXSW<3?h z*4oy^hQsh5H1=OFwS8|4toQ|zSQRla1v{>^IPMfz3SjTl56wn0QS^+1 zy$dNMQBt7C?c@`MZ&jtB?w&|^SW-Q^cASWzeQPV8 zt8V8vg$RUk#Ka$qnz(=!Oe7*~a+30ZB5dg&X6G$qdwC(CE?T$IkZIGZ9#ch* z!ZE~Vj2cws^e~Uh<#oyY(#IKxaUvBOddqP^pBf%%;|xXUEGjGmmO2_78l|sleEmDu z=kE{w=T+I^&5YQj%+P5)<6BmR`8K#%RO$%2;tZoLphB)p+0~RgJcRHOBZXR3R_u_I zh}-Zjr1t;no_~LQP}eM*ud5(9$J^P_vNbN8tX55B|BuNUwYk3okxxtl;{Bfwn47|+ zgDyQP22@JTwmo*ftTy?Q8sld>=zZU{y*NBSA^TN{G-vMtqod!tluv-Ng;F&{!cXMB zlj)XQsQrGN=rEjH}g0S!ed+SiWM;pm*i>Bv6m3*X1_YXi4-Bt)#cwzu)oL>i1(5 zv|JuWEM{-S%TqJ!cUs^}I~s5Oo;~Q}(-KFuAm_zarhnb#n92kSV$NHHZhmD${kYo1 zqStbhXn)~%USzLtcS6s?bfu7hQs*j&`@PGdca3je;`Lbh+kGq-VA1}U3QeV|IM3#( ziAQm2*1%pbrQxaNljB5_M38K;uWXjf*V-D!Pt*GSZ4ql@!2P`g8mM$8AMQZJo@Ecv#%19A#x z=e?cB#wzzz_r}UoNmvERmT1u^1FEf1d(GfJAZU*MBgb2cyn&69EGH{WsIZHuHPgl| zvsc@V@sJ{sgeaHi{mMJb7sKF?zYR_7Z3R(qVk|k3>RZbqh$e7QFr!%IJ9@@KIAsE) zsNW`aULo_l?}Xp*2?^W08uvE+URjz2yqT^zX1jc!(TRL)hEuYpgzc%fnIu+DBOO+mN z_1&U3lYST6KoU=7em;FQP1S$1g#UHDUs-v1UEV|28sR1P5Ao4OE2vi!w0pmr1u!*P zy}wD+TYQ~hW?8?!sbBAYoh_aEH2mYkVzr?_LC3(&C?q&?61vNyN9yV_+ppFPXaEgt ztYc^yKjx5AL)EBKL23dTN_dH0^l6xMW1Ey9vvJy)*C%Di-@bTzBO4m-;I%fCwQP1op{6A1?8DSK z9I*GlJ`s6)B17a~6wVLLF-8Yg{srd^seJMq8pl*bp~;gx?tVFV2li2_S;fNvzzauH zDqLCoG};38Kg}}0A7+|x{10Az{XO0#pn}9|xDwGfngEM$&F|MRP&9@PH5ZlL&)%h6 zvK%3l2lf3`k2`9?hy$;^Vh1dXeS7~h)PLrb6MdxaM^J^*0GLv^ukWVGBZpfWV>7W$ zEFCG3!vvtweoSBfYDPy1g_&XoVd+ZBdwj>&iD-2$+j!`W%8Y!49O3$}gP@e3)POh! zFivYS1xhiQJU{6#W`98!vNm)Q{E|0mTJ8|G3XNmaZ#2vREw>9_Ab&|wafjn@Y7p15@3^HG|(Wws8ryX>X# zWB|k2aW>wHJ(=pwQbQGfwQ-hT(PyTmy zNJlN#>0SQUiW-?FD@n6=?+OTU?q>iTqNpW@j1|<+NR+6KDZ$tAbaP}F(lfl$0$Sw{ zGWj}bC`{xx?^7EYkc3A6sB1G5AR~DPy7AqN&~J(GdrD@TY4+X$8@RWR{3V?C7vg z%SH(C&1kop9HE&mu}ur|xsH+CPgi(h&dqcRxjPb*a1+~}i5BFl($wS*| zptoOOhM_t7CECr2BR4#daBBk6^w~j-3QlQff-Zl?rrdqM{6CtmI;zd)=?2%gcyV_r zQi?l;qQzYc6e#ZQ?rz21g1ftWix!HzQ=9;S?`ePMJ12jUknFQNJG*n|&YfgjS4>cU zByUYXzkWJ4b&evPbej5CBYpq3dfH;a;gqnY_LpIK|lOsgqxix!GBmoRG9mV_4q6_@Y7 z4G?goCR<2RoqYA9J;is`QYh?o$`Y2J!03WO+F5;Xx7{t|#hr?D+OH(j)m*+{hzt;* zyE)*=KOcbe7VoArt4Jwn&%4#9-Zj$Pmyp9Y{&Cub!o61MK0PhwPy~b)S}|2rFlpgw zyezV#4*;)L;#E1L(QKe{AWC>ZWRBP7RQG9yp19q$@CLEf{R?FHILsaaX66}q8T@q`abFZ-W*&(S)g>3`1? zmbxDB}$1N2T|x&{JCCe0wR2i)xyt@a4H$7^$-q|Un-Adr%kn+V%s^bK?DQgc8aavRufTh z=)yjp z7=m3@qLCZNm#&p?SFJ8@_(ke(8Zt`Ue9yL~TOB++_R_Bp`bSNA3zXg5TxK^tnSXlV z)yg3MwL1cio7nqDt57j!;%&1Qebk2XVwdNK%Ss_oy2U#=H)%i0yUf!`OCxau99PGSa{AXT&vz zJ0?v+f916qt@!fjCAP-Yc2hV?u{TSm*3|LIgaWY|caD7Rx9&5r1s&WNvmqBNvCb5J z@hYSvy#q9O)>0r$V?0Wn1i13!yela6xZ5WXBf*=rJa|#YZds4>b{U&jd|goOdr-b` zG6hJ~5{KmPR9}iF&3Kl*Ev+e=kAM}OIjqht+w^rxQ!I17tBI3}VvKO-i^jtOCcwSc zDB1fU&63)S=j>sCW`c}^sN9gS zzjhIgyRG{mtl@%-&Jfy&y5DX8n-KH7ZI!1wgl`k=tubY&rwYu_Ydh84b0Jhv(+*mN z861C63h(Kj7ET;$w94EbjI8%CZBCWuSKG15b{7+-QFFXnY zR`3V-!HwJYpBU({(k&bbxJB}VLJfE!eGc{p69r5|%ry=#;?j`_wZ*)~ZsZAso6q4S zaZBI=O#2t8_zczf-?=anv`JnNI2uQ2cx!!6VAg7MU>xANY|ZmauLj(j>tklz|{G^KTVVgPjd_Gi4?Kx;sMK)tOWO+c(ArxX*dkkVr1HgDC~{GbmtV?C2)Ev!&nfQ{RPALd``zEtuPr z;uu4;Mm^Z2rLSKJaTrA?#!qHMpc3kl`t{pLHLts`4;!xzFOv~u z;{loM?Ilc-!N@``Bt9cGm38lfy?(_ePk&A)8)Qe{`TYfCD35>wD+W>$tH$S`3Psim zNA3@gY&k?_z`wh9I$_<6SD7=PB;^9X|G|P`WHiB^?biyY)t8l~mx%2p>xr5-h$*i2 zK)=fM*0rW*al=cqY@!Mr+1oZajyF*vPX=@fnabq!I{nTsMU>`s>}0K;*xzp$%Is&o zkYSNDf{+nI9nf*L{olWRqOqIOnI~qca3KKY&)3k@v|DMkI0+*!o;wmoTiHtk1#GI+ zbX^4vHPG|9kKs?^r5x7iuJig{vGj^hGJT0DG)f(fNL5!?%;B%%{>fprGOxNc2lNCM ziG4TMV-0~`a^m|4>rA0pX(P}KJ$zQyW1+4*y!@fxEeI^{r`#L$TRkyY#CX|OE-Q_9 zy6s=(^{dkUJT5$Tb%=;KKF?RSIRR|!r4}eeB$kY?au5}%7Q0Fh`pYSk=jGh+<)G>D zL_8qydmebmT6$2aR?`?SbTHkel*|EvtGJVpA+j3hGy>$@?cy+Wgl5zP)A+!DGQPp^Id7zpdi2%P$h_p9$Ol4|Mw&dd<6|`p16=42=c3j`S{sK4A?4YVRZ`GmYGLxZhsM`q%hb?> zf94|uw)Q(z0}RppW=%#^WZrEFQ|)C%1L*mD#LgRU=xwZ=<~(ib=bYU{Mk-AMu5kp@ zlU(=b)$)O=HZAN#aAmbYhpSDZ&u5Qq#C{f)KoAiho)l$-Cn{?CIq%+7P9p4^cv|E} z7FZNM@np=Ia1aBm7^>*%CtAuuj7RZk)5slo_~8Gn|t{!lUK@Mc1cq*Hb4e1Hk7TpSbh zMM6P{8VQtfvkMPHArBh~>t31=Gw|1`a8N0$W8yeicyucljyOA)M12P3eO>&zspBg& zUgYhiuKKv(Uo>|l0zFc!>3SJwuIXwwok*4EUpgRU*h3hW_q_Sj*VEmhA8zu5QFe{n z`t>S;ez8jLN}4If(sG6d&YlK*oCRpDLo10a{vu8ru%-vAaemJxirx)e&=Wt%ee*Q( z^gHeLZ)@Paolgs{uFd2^j$iMqXE=>$Sey1QU843s6N{A4rPzPwA$slHn<9?+On}wk z4GbQGL6#+1>-J%1jy`7XD$GyBX+N=)B2d8l!5)O+*tC#xG(CO1(jF4^*3Vj3od7nb899StbJ{m5x!I=5CdtT!s{i-c3%~eBPG7< zkiY)QablGgu6wK-(O{{ls_t;Oy(;~^F5!bLy#F2O1P_a8@B8A@W#eZme|xa>C1jI&HuVh4fNNnG3dIg zGcB}asy3{8eJHFtVABf#%eCL03h2Eumh zo!dd6ZS>LdBzku1)sK;c@wSu-o%Bg#){wbeLoaBGCZKi_BeIu50g>!?_;@^!AA~D` z_2EzQ)hV~#dZXuFy;|~^bBR?&1zIs^dme=#XKQB+kO;Q2k~wH>6FLb=`L?&B4hR@A zq{*`;m6w(4cAZDa6oQ+dZ>!>ihU8K|6YpWVe4RV0udkP<3#xA7oV^wVyl1ok-9`62 zMjba>0*hhex1W%tFt%(yMzv}N0*!BZ66%bWoeq0HhN9$_tOdPyt);kiRXIH&SEgQf z-ff;^oj`TBhRg0rjh_2c6_3DNy3E{x^=fT?alY?Hz<^PIl!`Qp0)QNCl^-X+yxiNB zG~l+^m*%(|L$9<2grSGBg@z5Dd(0va>pd?gJ zmn0GAt~~D+Hr$^+-}K+cMyS#$czYW?7wMaNGKkrqELCX(2kP`$E;l!Kwbu2+z3&50 z4>awyrK-wiAhap2N~hcWAeMQFctk#3z;`P0c~VjDa->r9-w6wkT@yLA;Wy&=^BHb! z>H}(9qHZg=MyJX0vVc~Vb~MCC)c}Pshb& z6Q>JEyI=RXe}FLO(+Q0;gTb2BdgqvDzg{kLgbZUdyAGQ)4RUzAh9}FDF!cX};YHc- zdYXccC?v?qz)SiEiY@^+wpMa`Q=n2$awz1HSpi|*w3m`X1zgR9!I{o>N3PxK4$1Pk zjw9o%b>5OsyxF3b!k^04R#t|=X5i7Y>s3&h#o3$}z+(u@xzpyccO(f!$}s-urm3PR16XG3VElQ04y!7LhCmiBwu6?AR2ojjAi?u?K@-LrD1vEdNZNc9 z9_k+ku+W#sb5{hMJ$RX7u={!h+Tcl8xfe+Z0@7=Ngob)l%3Y%xyNV;Jy2&edjLY&+ z7c@H)7={DJ;pypQk<$*H0u1dHUN2}jTeV4W50Lhg9)Lr7Q>k_LSYn=ACEaR*)Pu;> zu&$Ys0z)9=x+*8)zISyA{oUdd@36Pw;p)UuC;sE{aiVbi@W?!;XZ;~;xcNf@=#Rn1 z#^XzhQ>l278dsj4loUU!#vQw8JbR|aw{OwGRF)l%kp7#x4X>|16Mf9xhO2cUW<+eN zF>I0Xfq3$;Y-Fg85UwJRE~zYweB(B9gafy7_Y09<&9|rN*AlVe66ICYJSELK;pH{8 zbYvT!;r_+d7_88kotT&yV8&uvqQt}8S2fk`4bFee&+|SUU?=?bQUYw9yk?W%0g8sr z@8-3krp6)Z;?hDX&Bq-@KLt0lCj~!TZ_z7dEG$sKFzY4Clq)w^qg$x-0;@rHG$T3# z>W5teWVzIoXhxnng{ead z5HMyLm}V&=fL8MSQ4T2^;C^w zJ#N`z6)?Z0nhjH*p>KEL%hk|hMNCyTC8`+}x<#AvYuE;q8=WclweMuF{Huz$mz6GS z8X)S|d^Frd%wwx(XWhQ|Yc@4PTo##t=)Gexm!Z^KbWs%9`2OGkI9luohRDwQBWKU~ z?{#_N*hPLOxK!=SvpHj5%_qMrQz0*SW9bEIQc;D~7B+DZf8r_}S+%wV&Q9oKOz24r z&+L)2s+E^+D&15D1_kS!2LFFX1C4(zTkyxtv>4`xwdMRsF!=acw1#9R8&}gSK2R(@ zHg=7d8VTk>afT(d!WPp8dq?hB~CHzm|kG?3z(g~f97?auV{^mh48&HyEX zm{vaWTUG<^lY~7~`l3o+*N4$3-4-%IduM2!As=Q?fjw0+j~4gbbFVn8o)F z$IIF1Voq*rU)!UM+sMt0*E2UB-QPg-kJ_X1+Y=t42G2E|>dMHxQiUu5w@W5$6Hs{K zU|OT!!7pAe&a13HfIAAsbI8c0oIdFO)n(Neon{Cb+W^kL3t1=^6_4w4u(R&d7Bv#k z4D$N~uY^IZ*S+oKa?L&I)iDLpr)3FKLIg1@tBP-pW@{^OhUAt~9+HyU^GERT*eXWf zPezq`5I)k+Zv zplcmM)sn0yoW~%xsj{~KC_7TyyG$L@M&U;ja_;ZUxQ1f@vEB=`RHJ&W_Go@Db!A1z zEVe-zcZvI4Md^KS+jK0{tH;x5U?*YoVYwngqM*JrD}@jYPOr@6w^=N52&*ydao9Zh zXth)&R(3F*<0__PpzZSK$K|xV=Y4ljW{$UtU@pIp!33Cj0{g!&iha^u4}UNImw$d7 zeEq}%`l-wKfrywGc(@qu7hS+`Diis{v zJR)YGJFlzhSnuA@w6H8~G7Zqs)ZEK!}*_WWM0Gkw{9GnP^Z<%bR~Sz7l{NkD%m%B zn?D@MfnmB~oIh!3pipSwN&8#tIz62DJwe-D;@AwDjn6_YuYLkjn0FTfy^|3VOqn<^ z4cq&-kN4~CxT3*uN;F1?DhtQsz;1fjgjNyfYBjk-#z(1>Y(<>w`HJ8x_!C)1RqgjI z2ksQzeC_6TN)>=s?_0ywGr}kWnVMC4ZH_kj-zTz!9d-|Y7XC;m5jf0<0TEP&$7r?| zH#tqOLV(W2en@{K`-Sqz(8yY9Q&jvg0?c|OwM!6b`v(*T$h z!EktBI1>|7yZm@r>0l5Z>DgKOv^-_>kpKaMHs6}tvnzfO|H{MU{l(_tudmZ$AM!}b z?ZpHploq8UoYbl*0?l_zFQhvi;|p=6RTJqXM^O}EKq?N%+9An8d~w0poRTd0|H9^g zQutAN+RvDA$t$~w4tA6jkp+aDCAYVEdV+r4RaH#|?UvF$uFXvv>gq8xJ)cZ=`y%h| z?u3PZ6R{~PgT$4T1kWx$P`Q_0ddg^kaA8<3q4ww&t2t_k8FUoek;oEE=!%%3m{gL$ zKo}D<5aSDFK{96);dzruIP4T})-85FU~gNuaF1dA85v>%*mAG?yXD@9G&%H?gK*#S zC=VvOk@qTg5y=!HWVSGeHHUEy%1A_CEDK5GD+xv61NuEsK`4zUvZ%0%)emGa;bp6; zqC|YbZ}?OgU=;(T#MMa|`x;Ow$1`Io7;DkP|M|$?oAUne{4SQ-9wC@oKcCDefwZh+ zI!k1ertEfr)c%x^!*;Mzx!dXwhCB9$kG9JH-M?ix_$-91q!IW{xDqb}bI4Mca$h{* zi?Tl{C{TQe8jC*CIM+{ufiD0~tPdT}9b4VFM!zOrS1Nt&}28R$V z%B1(*NvXD!20#I2A>8BDE$VkcXfSG8W8$LTw;<=3=8#nM2pm-qE{XE{o(A_O%J^1| zH*k91Y_h-69LBuVvZ_aiY}Ary74i3QNO5!J{kQevibeiLA3I_tZjT?;+4S3k0|N`= za)*j!vZ%rZj+m)h1z_V651m2hEO4-(!+K6-jolzpgh=BfV>mpf;8936krh~OjzFi~ zQSz$C`8SG&vU;)a&9A?v+f{5{#p)9UWSSCJ#r5X(?^kpf-r{H2Qa3(T$Yd_x8zJ@BCD`^=MMmDMdBhQYlZpv(`9c zyURlG%7U<~^Nf}EcD8KDe5h34aWt|{1{$;3*UkK!c-(ys;F+SaWHp_|g3l8%k1xy( zWsax{Li>CdR*{U>5xZwJSG8hrM|@D#SX?&Kx{dF1#R<^%?r zKDp?pbackH6$gEaJRbyR2uM02IXt|LIhA4U)0UvrZpW9ufdK~I(2t5%2YkD+ z3x#AQ?BD3ULgW6pA9IC$)Sse4%QgR4DklTe7$hENXNPWDM3T4nZVKoXj6*AVgaj$V z;9Wy>HjQMP2LbunsDc2y^R0ahrU(#i;rP=)`*_ANkt&Q#U!-6z|te!rW`2+?l+-y-QOQ> z+g34=nmIGt8iz=Zp&e}d;>tGvmnC<+s8|ywyC77&l!eF2{Y(LLYje!3nPeZu%qahV zF2LV)*plV2e&EI?zkl!CT8KXS&d}R0q96w;@n&h^i#X3;B^Rzuq}42_=T>HRzX(H*E*0l{`BsfNfdE-u;G z*&l`eu+9e(e&4bHSpkb5Eg{?RhuXff4NzRzMZqzj&a&V~`?qjp&T1|%ol>?JS{PGb z-!C24yt(b|NQ60hXBfe}=Bk`k!|YJI;~UErEzmnp9?Btq1ZmL*uQ4pXKU$Hxe8Wmc zJcE(EqJEPCN3gsBA_;wYE}j_Lko#gL)n3rOy39`aAjZLFd840to`8K{9$Z2#O)9SV|jejLsFh=w-UBR z`q!|~O^}_V2p>~r(fNG|R3x17-%~(H!ej+c42YyWj465i&a;koDim`C9gcK%Y>@Tg z-xT??50-;4Ycoa{4`EP17a!Hn5;SNWr(tRZffI)s)r{Mr`0un6U|_CgbDZns2HAO+ z5@J%mb@Z?!3)w*!fd(^XiiC*?O!qlmMS06iA=`25MjIsl1&<1o=1wPOX=(6DJ*E6a z;TJqL_4fxwX$c7`P=+CB03GE6yaX0HzyE3dpmauS92NiSa)RAfOJV z1aeVB>MU86mEdw|4sU)OL^185T|%c4g-mXf{zHIF&L9Dw=-yU`9TA9(Zp08wg(^Bp zgdGuf?T`~j5dJ-Q6iz(jo1MjayUWQSN7Kk1IWDRoEG;6Hio+ieu0rqSmiQzt->p1Y zR#xo~ITlrY=0<{eM1fS-qqMph%cVIra*p@*rD?;+>)nR$8qp_FY=7~2DSLRw`|Cjf z1?~PEj*O`u(Q^`-h|IX)fL;KW;|Ur1Vorou`qB!`cn>+|zGN$T`MU+s3k5z-`vYlL zkG^+%U3et@)JQ$PJ5bVO@OtfKc@uNmyDSm&`|~xMY=}C&xId!l9-hc+Y3_ms&BpTp zxnj0(k2B;7-_X16c6@B(<*5f~+w3}#_RgTrc=@5uM7>}BcZ|~kt2Ok`MEGF@eGY+0 zNy)X&003F=h&)Z+EOSbwq^TDL#>O`P#TZvEJ*K6wXiF4w?@otZl1Ab}9!pAy$-8@Q zXZIeBLEn;IsWGuHm4!K^B=XAV%qusZPa7{?X5D8gX=s$*voT>ih?*fA6AX`1ztFKn ziiHYqF5?&eGkof---ZZtDW54Q!&NEWRlYiHB_x?^0^-|m;rC2s#u-2=yu1O_@Zy+> z!J!I6)R^|7>*e>XT2F(a2RT}Zx0*zDCD^vxWlX7KCsSKc zGg*P-NckL}Po~W^r>V}yeL$dcA9TQt8R2Gg$>L_9`-dI|{`@TTQ;~a@m4W3^pu!iI zw)o7JmK+N-f1;)P=Ih@L{VAbC3GEXM0-BcCppLF;*Y?k$*!ZaaX6H}E?>FR|zFfnI zB8$iKic`nnb{By6f9^38WaoM}+B|8N?oZ`DwbTd>#jh)WPxQYDy5D%&ALX&ig`BT& zRqRjYG?~wkC8wnoGj%$5LZ@{PSoIM3kOK%xUI>1cS!xC|zx>)7pe)v4$-(bA0?V$Z zyw)oQ0?r15a@VH;#ALY|nw8JsZZ1?-o1XnQo(Sb&2Mi4v1x_iH>1yut=~zgz&hxM1 zkcnI+V^NCiu-Kf|<%|V2ms}_V-b>uUe#G(dF%#27f%9DX4!vl3IQB{k$B}O(6NQ4% zpdr`}8H;8@CBi>bhv=lnup)>1PRj4;9%xVVd=({RKBNCO={8o$Q)?CzOO3F=Kk(`m z8FD4@gN&cT3H1ZI$ydx$5y2?1IWOzyKy-v-wtMA*>BCpKhKaTyF1x#YpXLiTwz-)X z-`u+0N?f&vuHR@aHSYh;NKgAGB222p^SL815k!w0hg+{g#PjA5uxD`^Jakh8wBA3v zU4ddKjwm2_%KxHd;*CYXD|XSigCkJE&j`xi);VhDDK#K%oW zCHtc7G4-WI1kayO^6i|On(BNGv%5R=3If4zuxCiQTAH2B9Jx&K;#jn2YcIa)yUdo` z|FquKxUZ1dMUJht8Zq~BJ9{|4H`HrBOh9llBZWMQO2m>3MWp0+8vvXcTnEeq)BsPg zU}UZ$Zjzzw;QWp~aFam_5B#qUM6ss@O_19kX3N$RE(UvknmU4-7g+Jp%OId7P*J3S zv{lHhxObirC}DevR{8yu*>xiSYfIJxgYU?F@8;kHgn6Cq{UjLu<%aMjFZV$kwWxy! zh`koO6ak<7KRF0X%Ah!K2V?QwMw5WP%_c={66`Vr2b0XC;u;`eC5T?BGN{-~b1;_u zb0Z$MQgDeFJ!*`8Cyek82>0*BlsZw=;~#3<>c--7YR|`Fv~yQ)_g*+lrxmi;jc(gZ z-?I6jmxDoMUjnSGEQEbeM}LjHt=xDxJuWq*i_yx20l^k{FB1mhzWpIBGfMjfCWcht zj$Y&mY0UX6XLeKJv08as@n@7TtT}&34cJT`;{AA1(%}}A8e0*`B*KQ3}y|4Ln7HP5QU{3qjgHc|6HDBfTMEQFL4j|U2l8uqKmg%*6+2$ zZ$<)^H-GR@dEs^M!ochGj3+LuQ!80#kRQ#o6gJ>}3IiMJPO0iefHk+*pUu9CrDHyB zSXK4_X_@43K?Vh*L1xv~KYOQRLqd7+Dae7FKw}ec>!X5<$okY_=HD%LBCgvkxc;8J zX~-ItT9*&wdxciujErNE^}CcL@bB)X zRhujh7ja#4q&`0gisVIl0v>hf}n?ZcJ!)mOYA#6-_}t7k+E#DnDaKlbutNGrVo43}W*hfh5ck1BiYQ!#hL zq+?)rLE*_LdL-E5E$=d&aRy z#>G(jKIMINd8a6{zW>1(xjax}E4j6x@@g}`@TprEw0*!+-tK zed+BZ=exNb3-UrPK+cnpI3+07W;cyj|L-s65EyPYnWN2B&6I|IP?_%Lc@a_0S)#63 z^M+4;Kd|9xS~^Hkvb+EX41Pa06ovnLc3Jy+e~IVZyq8a>_Df$nCp~9+^g#x|3%C&M z@K~=TRQ|r!08|6586w2ycx*u+Xl6^v^lbW_SFg~cvnH!tt=nITSMaQcyiXk>-TBGS zfarSNLE(KYxqynQIt0+WpUD)E8<@-wdJ={@hwX}HxgYIaOy&qxfLT&bP9GjC)^g8Q z_0 zKEuXVGYQ+&)b!iUcQJk1pwqq$V)oJIGhR7v+Uoqg8gK>KJ?y)4Vb+ADSm?C+2T&0V zXPb+XsBQAq*cKbKAsW?!l#EPsc`P*p%u{}maov5p#tmj|+e$Yl_xbv%xr+}u@c4bH z(c@-5b%KN}?o-@g`?8k#HVOzw5EKA6)UI2`^xcQ#m9br6Z2wt`6&^`yl8LJ5SVHs< zE(j(dorI5$%GoC}ctIE)u%9pSR|^fdcqU9x-UXe>HQo)r1NFqV&FwV&-F8KZ2;;{f zP-v}L!}97@622L$yYod_7!9gaOm!B!?~T{*xPy2J@b(j153chE=yA`h$U>62?}ZoJ zTA5vCkGt7)ql2M$H*_@LB&&>s(_*~9RPN>B_r?eP9vx-XB|>o(RyA1ObbB^GH%{r7 zTg_J$4UN053t{HtW}7ELe+5#*hU+aF;{HEeHYP&6X znC}J;>l4fx!bDB_8&_ekqkE2l-pv_#l{8tPujh@7HlLSte%pqUnIC5^TDvzFL!dGy zC*X1=+y9s^3zy~@KYK+fRrmPo<&&_osef8FwNuKYByG~3=MR_4;L}4*H9q9EqNb~h_mC$DD zKR^;Qj)BkeG#IEo@x42`yW7t!Rm|;jzr3m>%|3N_am6si|*x+@JZq9QdufJsir_bgF3To~h-bisD|n-4;mg*XX?* zE@EuV1RE(hA53TO3`av1Uynq(ULH??n!Il|HfNWKq!h9_+NTxk{{VvX)qKRFZ=A~= zAf!l{ug)%bk5TH{0J&w_*XPT7BhLvhKMycC_x2SAagTTnfD&g;96=J26V( zr2lZn97P#iF;o}CwOIqr6PmeuY!14YuisAP@SXj5ff}SUC^=H-blRVz%?RJF8@_r% zX1U@bCA5!`X6#lwc7oQt;-<2M&zjP5EWTO%_S*a|yQCI&^In(@$gE3YPDK9=P=-H_ z12CYiwtlhHc;S7a{%_3E5aB-5vcxx%^~!?jFw^7XebLUXA4H7zUjcB6W}7lPBuQG5 z^yhL{skspM)Almj)@>~d(@DZhIFg}REKldXA-Uk6{!p&}G}+Wju+3vhmDP>LKN(3$ zjQ}FQ6Z^!s;^_Zr>AyenK5`BL`Wr~~@VQ!#1!~s__Heyss1Vdz3t0$M+`#h(?3fJ z%Mc5NU90+lih0zF9Zhx=H*2^<)eeCUHNESip;?=u^do{Guos)Fd}Eu3iYCrF(G|=d zobkK*FR>qT`zWi_v@Bhx3a|JG_{wtRD%^P2nD)--&PaLrDLkH)_Egwh0KwM)B3&dB z_1iZz_N0Bw`sM|XW%Eyk$jX99TQKZ218p< z-J16p(^TL$2Dxy%(Ha0|l4zjfqGOL0Z#uKr3h^ld8vvn(w?PHlD3E9Sb1RRtl@5I>6aK-U3A#4E(_1;KlNbjk4a*|}6I zatoK$6^S_`jt<9ku4Jm!_Yt`c5Y3Y<)~MFw^Vvp=Ews{Pec6k7_jzin@Ka2Npv!Uj z^#QZp{;(oJhx@Rqq6kDnA^=<1+wTOxQ7)%D$c39c=7vjiV(QOE-kh&J?R9^h3T9L- z(G$_0m{ckQgT=K`V9&XsUl#2DMsGV?q1?jPP=hDL%Ji#unrcTJR>6S}RNM+U*f&|j zj4a+1ZREv50DjORvxQ19fB;*-kilW1V~8;hTpoU}=rDsTLg&rRb*5^R*w+xj5mI65vDN&pfM({h83n&X! zOb{XRSAZU}LkbU){|Bl9aReT$ZTG3Ds3ov}l-JeFn>?+{{}TL#_!#Iv(1^lN;)46d zOl(Tr>C^+I3N&Z(`dh7;ss2Ty48}z9Vm}4g{&AQv&>hG=qEIMa{$QJ*;*E%4xH|&n zGYc}9DImjNA?HzqgHiYJCopj#yOLQ0F>7UT@2p2WVhpCPyhx-wl8FL&ud#KD2GaIJ zqS~UL2SEbluhrrbWD<}xL|8R-G;W4}w^jlUA5&384VBe^X=8Q|_{E%C;r6#!ZX(S; zbo2MJQA80JwQu~Uq1oyeU|NS~dN@HjKdlEdz}!~(25Iq5t_xN0!9qeck*@o7C{}fW z#G9bAfzwXs7!(T&43L_r#1I~Xq@81~n?%f!9Rqn;s9Y|>MTdg$OH;1zzbAGtV;}4K zcYVfPRl)q3>*tZB_hsMRoa3F!BZKE2MWl^!&-KF}V+=SFE_X!UMF4*P`{|o~+sZ&j z2gJ6N#DdcI1vPj2*Me|QR=A7z`%45HSIb@`Ydi9?aZC$f^o5{8`{pjRaP*=V_EI1r zh`vLnBrkN6&UcugbeNx`q`r$k_ifCV@vZy`+mRB%&h{4arHJ4|OB9!>r*rM&QW;z&mP3#qAd`bc(Nt|+1)gz)S2GFoV=w{D`PO#brhT$B^>%93&s#T1Wgo^Noj{u$x8zEH|38w%p1 zB%?;{uch|CqhG~;j5avmgCG9FMNxb(zh_zEVd$3sU?9ZQjKx%`NJ%R(QK_WvQVH%VEox+z>61l=MIcbuByDaon`;Xf6+axoseU?%(Nv?) z-l5N3|KO{(H^RfpWiW=qw-F{f{Gg%a3Q;5xYllvDObRx(0LdV9Cq$x)r?!QQ{cW)> zaIsQ(XF*ZJx&F=aYj3{PYh3Dl2Wc>yMnrU@Zmy8!-klTn&=Tu_7iI0gEhMqJwf2@1 zSpC&j|TLMFG(5f1=jNt)a;9Fr1)7C2SJ%xFr=${ZHs$&si$CM&yW2h+iz zUh(w=+^9pw)(N~6c-p%3K)E?_@k)DELKgk{KR<`!qR9p5_Vk=402F-s?8V#8O1Bgn zg`)zKd49xk!xjpMI#YK^oR=Y;OzZoi=x3;#;C&G1$Ln>K3`K#5{n^D+!znyw-eAv;Ql|Z=Erbk9>5BYnvRYb) zK6NE(mgfDE?{4IIcNDWpn8S52k5fdW@)f~B@|IcT@p59uE{DrhUXsZkCWesH@^U*Q zf&xTgKjfPg8gO@<8=Q}Rfys}DM@0i-i0J1m5Oq@M!W@yojAs|5{(1?|P=^h-{o$OF zO5GeK?n!p3;DI3GK<4Z-2T&j6D24actDMI3;^;HaMf9)s!uuC7z=gKq1_p}VF6Am} zTwF8WqA{Oy!u$(vSKSbgf_x_wLg%{;|C8p=UavQusHo|F1bDH4v^Z_HLYlCYcj_SO z&>piUF;V8Hm@Ndfw0u{oe~Lh)oes<1MD?0{!@Z`n)BC#Qza;utoQRA}9bxy^9e|C!SYyaz z=4|tL+(a!ETaK2>qTg=2Q14J^DIg#K(3Ts$mJZU{XY)?WKWFaGd{@%d&S79yY;p5; z`nFN6-!Yr;;9ZF$U`w-DWyoW-ulAYebh+WA_h&=)@@m=71uiZwo=jbM6c^jEM9CA4 zcjN#!u%Q8ZnDDW~?TqorPsPU6gGxR1{M+mPk8aYA?)dN>;%^75alSV?;*G_{w|+1C zEp6-onY^9$x+I2NABU#3ybcoc?qcbXCPvi9j6b@++j8jTT=F?)_8T zlbEv04bXRQm>4x)B@4#`Qhc#ddJWWa5rlhn^4H{UiZj?52#*E_CR_iZxWi~7pV0kB zH-Z~jn{8ar!H^~qO_ncZC>3ZBN}?D)+zTcOeHpFWnakDVcNmUVUcmdWYI!_zcal&Q zPKNw-KQp?NgjSWY=4;;{Uk-HvL^aXFKs$_^jH(Hn^1X z=Jm=2m08&3_<)xSwzRafvHqK-aapzw){;rcZQN3~(iCcQFe%=3Om~)?I$W^I&H4Z~ zn=ssnXJ**&bwgJe89S-pv3mJ{Xd_Z+$nVp%^JfFY~}NT>=EShhrq^` z7CHX8*>y5TMtXcUZEl<7xiab`p!H?JyRekF8n3%@yvf5!OH-Grm8Oc3IKr;I7#joX zb9%=pyPr>WBRuD11#HmGJ*|Bl!!w?Z=M$(fJ#Z)pRwl~R(SOcJmZuAw!2bTLR?tpY zxcA!v^Kfg;uQE%(O%PD~11K-V(uXyzuv0*C@?D7vP}9-9lL!b#o_AcloSKWQHCoJd zTy$=91c+yS0;1(#e;x{skADKBcgt13uZxp1O$&QR&BH*WY)=~1g+k!=o1ggmYD5Ik zT5|2b;S+)Ww2EW--f(?ABM=l(fWjjp` zl+s;wE^|I!_&w6+`W2$ahkv^0xjt`GEfqrGlZB|Iysrzs=z{LQn3NXeF_jVA{{kV zwuE|@=Pmf&V`@`WHeTiR%D`a1+Smd;IR5uLQPYX$6~e!Ag$rMfsfx^{C=!KfCDkGvH4vN z;b&%KwAgI`6n+k;ex=#k;>+O(CrxsFUc2>v+OXJct$O=rr!Al%pDlNbB!I;K1j^7= zWKjK{lX{df_WTz*uCD6GQ5yrU4amrdf^#x-fkx|g-vxY5Peh;{Xy%G%YYq34>BH_G zwe*pr3$cW{x_!cP-dBe_29W3CP%9pt;aEj^5|ORZrin`XV?t~T`}}UoFpn`cZA^Gv z7ld%IEVic4mT{ysGO7g0d<>DQ?pTHez?1wAw2&y@vQ(Y;?hp}yGkm+|r|m(7<*uSZ zi5HkC&JfOkrRnH!oQ;}Ho&U3NBw15^|NT4qSTf~eGyrq5l~>ID;M*U3nO3c5jC-oo zVREguL8Xtc2Qp#^S8KY#pl{An7VHe6|8M8b5wwMqrp_3+CLG7zoP&F&O$?8J%{_7x zyx9Pi*AJ#tEK!pWO_;%$5%qEhMMz((QaL>Tl*)AP0jsod2d%SGc3&Apgh$%o7TY1E zkr9&{>a1}4B`}`Hgri=h)+>=8GnBgz!Y!EH+W*`iqmA*=%j4KnhksBxw>xMkQXh zn^PP&)bC{?bzve6NImvdE<2ekWJ3GAa!L4oJ4dYYc@JU2uT<{`QT=65#8DB#3{sHg zFI6#d7zt*7F6NBMNSZFsA8{&V@#6J(BMNdv9f-4QAND**%oo1u;f5JVpsLrL_P&aE z*}agCIP7`(Uc^+28zCXHg6<`Y7$T{LeDEY?Q>{JHzK&`9UL$ks;W2*+eL-4JhhVIq z+%-zsbUd}uj@P?JVADF_52+WmvQbEb^lIUn%vJ?NBsB6-8T>2m9!N+967W1Ysotk! zIqa~~z4p>>)W{~dhTtK7v8Va0j#9Eqt28bW_}&w9248B6@%hZoZ?5hY!FMl;Jnn#a z{LoLJrR=3OzC zBTh90ZxD|)dVi9CAyALv;4UtJ<8j@w;=n)~u9qg%@aueDQe%Q+mzKc$8jAKG+tSqk z@$}VkQN3@lOGBRXGf58m3YQ$pfQ^4i_m$rdyrW(~_eUSEnnW5o_L zX(P^RmzS|%m=00&_x%uQ|Ki)`b)Y8t&^!Bva8uM_Dm-ymYp{$2`LVtI;agEjiSJ8+ z^t#vGbk+*DxkU7c(eV5ym1LC&u%;v*wi$h#Jlz{sy*8b@o{l%ERk9exn0~&z$&>Rx z)fKa5z4z5ul^1UU@@u>XJw%q!pU31*g1&}tHjf7f&< zh=pm*gWoW0poq;-wK}R^aSn4H)0;{c6#Z!bPSyH`p3oW`;RaJl0YQ-ksZfcJG8g{z zmkE=-A_-O{Ls4hpI{3^H#e`5RE{h_c_Y~agpBJwheoZ?^f`E$6#3kX*=CqJ{U#g+^ z?N`N^NVe`MzFOrG4$1vgl<}wC3lHEeTPmO&#bWC+hwKDem|Zs-R&cm0u%%HTJv=@( zcixFJCXY1}aECUslrQPs{>hiT(s?JZPJ@L(Cg4bq+lPbOw`;UHL!Ap$5YCe;H{+>n z)7pu$G`Hs)7rim-CxT(oJ^aso10QjHulhDW3MPM~J0>Nqv{CXSUyD#TmVNW)4dz?n z+mRAQQa%Um&Mz#Q3+%tv4gso45Zjh2gq|UdV&*zSE1V|-gbW9}a&r-54Rj3Jmg#rz z)w${Bz+?Z^sqj18^sK9To(uZG>#$Pmbd@r1yC`&Td36~e`~GJ=V^S3kg)xzVwQcUw zcaNQYmI@8lH-8-bJP&OICln5|i+4cEuJ6y|ucd+3A0O*}b zT;1;!(|>+9-m$QC8-Cfj`>WzLTBqSwmyTSowr>4KCc0|FU) zsB26{j@fcVz}CNh_4tRC?n-1#>mrbSG5C&XKq^76GvIZaYRIAl)Q z$U`r!;IKf?K0!6lr|7BheD&_ip`5nqwRrH+@@KQ@bBp$_+7X7W8}1J#nIM05X6|OM z;VJix|h; zSEdCwn}2uM>~i1qIk#~a9mYBAeSv%Y(tY{h#egi^i^M`KA4(alj9oSa4--$kA^x+r z`{2Rv<$hX7RA6m6djhDkjg^$90c*n9!=kv8&Gz$k_l@oqk&!wnS~gLx6mKF*m1Ht< zAJ>zm3s($$h})rh6V{w7?X#a(ENq*Ic2UF^`*X!!D`xBdSIWVXnAE@Wy7u z#xLYLz4LwgVm~O}4FJL(wx_>zqM#nUiCj@jRW(T`4l115G^jb#;)G2)j#Vf{h?h9S zLn%uNBuXxCS*#(%Zv}ypo~%i};uD1=ZJMaV*+2;*Yq_O$cDHP>jOPXavml5PN3G6o zFJqZ2Bf2R4B=qw~*CFRf>S&6Cu_ZNIKGxTdh#ovv?0|9l%JkCuBkE9o$dR z;r2MFl8Q{(H3XZk_^+jShT;dZr((RGZoWiwN7-UfjM~19s$cpulHdOU!`Ehnr15K_ zrfL86`Rl=Jlx+r<4m^ck_sYd~Bpad$a()1qXr>3}_4xHcguYT)gQJz`v_9oR-uE`ca!sI(~0)`_>sEe>C|>X(rtRBGfUs?)1!Z@98^v6h9M;S0m0 z>3v=|X`8xXsBK*N=f@AR)jd?NFQ;d}JB)t{H0;*6eZnumIz8-~S>?-9kN`1ALtPL! zsF*NmIiYeSBwhAZ9mbv;JMy%Gq2S^YS(dS0}?iV-Rzn}{aeut7IN&Omo^zn~a zH&`>tgg>**)2saZSMbDI9RkP@vpuhO`c3dpyxH}Ca?pm^X1ZAr+P`%j!+?(%p&xD; z3WQL;t*ETd*x)oj$Pv+?kUGI*rI}KigW^Pm{oDS~(6F$L?2_c=wO~+SWaA6++uEFu zzVB0$lc|ES+9QZn#UOZQ>W$6KUPiT>gCg@~O{@w$d00f)ho`4!Cg}<7%T0t z;X)6<(3C5U=^|zj76L8@6vRyIFE*iLCLzhpexLTI4+VmP?cc*FF@b`$iZdUIFhGSE z3Ih*i|J0gS{4)9p7E~rCa4A7cafzu3LQ^%-<@Tc1vZBcaGTzhU(4@qT66n)d_#H?q zx5pA&Hn4D7l*<^WP&Z_9e%j>~5cv1Bgg=C_-Vz`ga+`K;7~b~{?c9f|E=91NTadQi6)LnH>VtwDhyz@hN*I2E-FFp)bInI6r{BUJ5=M zxvuX}+XRbr?ky#aKR41xl$~}696~Rwgjt?lchU>9ZP1~w@nBZE8f-n`YITTuIY<&- z86LKuLKQ|T4Mfx!9H)*~DFsX0sL+=I2a3>+&_*el%)&Q4Z1I38Q?KozM^aX`jOt6N z#P7J<@B&GpBf-Mq6jB5;=i>yJz<%M$5;3h8_qcHO!ayE(3Lrr!C(`nlfL{PCq&MDx z(37-!K9NYG4SWUbIGMvE;SD9g7}1dC)wbe{Ox~q>w>bT`BAf+d`<6g;I_{JgmprPd zMJ{nDL@#|=hBhzif5rSMBJ(5rEm2S_!@687sxoSz?Q0_;RU{!*z&wBkGy3P6bv7Ao zGii(0Lrf|jrXghlaecJIn~u@YU&h8!)^6scBkL`sFYR)4awsVzOAd~Pm%Q@(Ru*&R z+4MFB!}pm}>*yplX^Vbfak^40H8LuO4^i1>7g<_%5(v*gOOBWJcvY`codX^fPVUbO6Zhe{&AlwI<5znnHs#Vl@m%b#eL z(?QscjQ;ejX=|>s-J+KS<{gpyK1W^=PMs~aYKGYyXBzySzY^N?x_W9=izH;#YWVUT z{BY#z{Cm!)9rNvr0)Lb9e8xAz$--#R51o{Et=rL8QW2l}q|5j7a7?VO zE*q}wQqNYvSFUP9fz8ES_~8Y6Ex*@iR1zB~@vO_v(J=XWhwGtD&(oD9wYBm_t&@Jf z!eaCkna~$Ko71R<`{lXm+q*ma)mutDOyNIU07_uPWV-ybr@Zjn?a68;vb<@J=fA_r z&V9Ws4Gy3nP$gw0m2 zq~i;7z?#)nj#s*-%t$}X64i&LVk&tW0Y+7OUSAh^7KlJ*FcPOCDPw|*LmCOxYe9#T z>TL)Q`arJTZn9Ue^ySR1dtsjYv}?2?71QqnOj%r8PX+veI2<4QV3_p{U=PAu{Rp_c z#N1Z65Q<#ySo+@VzxrOBgrI7xb=sV+oY&}Hj-F)`3Oy3McYK|jetIfSH4yaaO62|_ zk@@?$07^h5F72bZ7nOqC{(d7LMP9}jA_*d>l}d@~l%`<|KQVaeWE!9j+bKiWT%khxt z`?I^u)L&Paoc2{`d^e!$?DV`77rr-PkE!eKr;@?T&MO~#nw+Y#rW%)z++206C@e9L zh0gduKvy+pbLTK^lk0u`aPGq)pn@BAwHaAc|2A&80jAZZs~2Xdiy%ed->m-}7#1-<4{SR32Y|8$ zO%o-PUFh!usRJek#*hh&tXSXF4K_2p1{OoOyt`1xk_LS8JVc(5P563rN~GO=pWFAr z`3??s-#B=lvRt=-kb7MMD+0{Oq{(LJyxX&}F615@mfbO{zsYYrd9+8!np05*)$6j` zvwZXAWVKb~<4x7))Jzf_Ce51nUKehxyFbj)(9xgAyPr`1BuBgV!*x5yeBG}UkSTuN z{z?&cf&;UNDS>8=2;Sx}odCkSs|`*;X;_Sd2$j%h5x#1@wzv{0Sq^cK(OIQM!XF9U z%wn+~`0tGde}Qc6k7|CuW|NWKl$dp%ejGNC5#^+?V`KMqb)7XeHDP(1(m(dG<>V}I z?)^-^LUHCfX>nEIQ3eNI-?$mJxpGMT$ewt*T(;E7pfeQo8$uus=X9iWHFvgY3!UwL z7;I3}E%}~v3~ns)=S*q?KJA_XdB--7fy=n1il5(q**-~qV*cL@jhUO7iI*=$jhWu7 zBu=ycn(wpK6964M5^&k4$oK>5xxYF9oM4sdUHC{xguYG5fKMs|viClg=^O^`cC zmOt;jXcmW}KuOR2{`AuMT{G`B83=pz?KbSjB&lVT4N6oY+Z-KL?0DZ?BB&5PCh0@h zy|2r>pW-?bZEA#6=OyFQ@geI}}%YK%CDWM8zS-q`cCjw@jn-}`e(Suq=duCQRISlmK!C@01 z%{t{K?-3N&Wh`q1IDsP&1V;@-3_`uPbH3v-J5)V`c~PF7Cc7l zU}Fu#wA3n3UH9wDCKv*8xgm4eMtYP3DXhSeT1?Y>V^w|SvrGJU28@Y#2X$<$38HZ- zZwwK}D9S6UvQ~ue*OoN`0^~6(rDz=mrnEvJ$i_tb27WtLsV@(FxHVtTG}(mf?LWsU zoorK3A!rW(Ra%LXwT+0E1SDDE0)1U7I6cx;SJf;ZSKEcSSK{8ofHkQS2A>gEWTAkQkwz-?yYi6gk0tJlVI>isA1=@HnoV5~V++;QtbBKl zq-f1JeG~s4{N1$v?CR?3j~~k37C*P-L7twT_5_;W=>zgcfbNJN^YaLK6&1FRx2Jb< z&jDWLrOIENKI-!dTAs0=b?huO(!hqHiuf_%3`M1gR)3*Rs1Y73N{@UY&E*)w8-3eg zZ+>EG=hk zQ6E%T=Pn;?8Xtg|aeRJEHnEhVeHA{^GZZzI!O>V+N|!u(dwo|rcSy`-L7&)1gqz9t zdCW-DQZQS?f)Wuy*}^=OT|!mY&iQYMgnAJU6>M5kl+nwCMdN5>>MDzge?+ikdTp06 zXHb+5EK(mzAbdvW+QNgOL&w$3MnC5Jzha711}6qI1F#qZX&hef`t&{=O9#rv2NVW6 zkI}x%75RKA=5EQG8B>2EQ^HYg0ltj}H5_$lC(f6!8Q5bAz@R7#juXqPY5Y{BuS1b0 zNQFu;n;nso>|}|w`7O>*-A>sqzDdxQ7q;7@G*`W*4}b8%_+Pz=edxLZ3u$Z|r27zc zB9-*9gmW2>DmCn~Vjo*G#Q5*an^i%<)lHx}VMqHgjeeHKMvF1xjwf1v37k-HB3Kyc z$0#B_AnQMgT^dTD_4a*mm?FaEF@6!mQ0Tq9Xh;=3+tUsm?O#uVQH!2 zP`HyZN~hE2IPaGLB+UXu3^26^n|ib z$y64Xx!m}c*d{8Qu=|b9@UcV34#A4)Qq-D!0sR*1jn3{VE|DbQLD{nM#hyN09VJ-m zv&X)~F*PY~rFQen*j|j3q}oI+1X^ANpb_>Trv3K@iasVf3ouc^LsGLv9FE0@0Gy(` zvxhzQoNcqc6Tc=r&MXVo%~&9N5c&t zQ=l!}Z5aL!e7*wD%+1X$E%CqL$wEB6ydcT;T)Mi9$)0!X?&HWE74a|cY{epW^rDs! z8RJ9^4Gh@@`{ij|)F-qk8P<+dj`pKWhyOP6OP+rIAVI9CauSPu*NU5NDt86E+Zvvb z$|Deng9PLDZedFQdVKSRk4e^i_B6Go<1$s$fZtuEWxFT8Fgzm^;7>$!AnBh<+pFEL z(Qey?X`%2vI_m&GnPQO~0{Kqun*#l;(S!KX5Bp!?}6dlZ&hU1d{Q$j4-->F$=XvUmh_ z|9AW}6Yl(YO0`xqYtAS0b_dtnVpVa)=UM!UW(7M{Ccf&%?yHw?lEk(qChGMA1eG93 zlH&?GC7TuZEo%Hw<4~f^)|HKo4xM!5BA67`Ks!|b>hrj}~i-TY?lb<*$ zh~ZF~7jA{iit*9(Le~G_bt+f+b(-^>A zeh%?xN3)#FP&xZxZTp#w=ywgYs+A>uG~gv8gXCVdkkM1P`Ar2H28PD`!us+aN|DP5>d@Vt{&G{bsEw%IMynC8PO&dc3<#KkQ3>!37Hn_z!U=NX_q(jEv>Br z2I(LWRt4-7UgE&yuRKs=v*D=uoH=+Ri}x9@Wec%ht*YQfL5UB!I%i68WJ+p=X*F$> ztMe5HEXKc!$a0vYC4b0fZk(cbQQ{9;ebPty(g$Kr3X^lG;Xk&Lw*KXv_)Whu@-%vU z?Tc1#uQW+C)vrB#f&%DRuu5B6y+Cwfdn^u5o1^yg)bBA|6{n(}?W<}wy&KuACs{+a zCS3ah6~;qyDSh=%OzMo@EshtDxXG@c$#+7VOMU^Xydg0>4@nyw2!M{H*T`_!phWO) zqA-ryvX_Qhtg_1RC@u0MWC1*KQ-RopM(>prTtDO&WbjX;Mjvf$7ZLd4n=DXp8dB0) zfil^MrSO^7j>j8Wjqt%(=KGs`6;(GTyvg8d|HrtB7Vj2#Z1%)saz?vRdX64BEcSkh zd?e;NY3Yd{1+83yI!fr~^?d<>$O+SmsFbx=>u$ykdT-*erLwX202ldkxw0Y>^+|); z+Qw5m!E8hbEUZcCqazsEq~I&;x2O&*%0Ng#M+JL1R{BHudvE8IBfzFPQlh5crOPK`1kSgDDtBnT3wQo#lcuXFjwj6#M8-Ab z^EI$ z(5v_p&KvPj@kU6r6)P}Yp=dKdfkRfN>b1B?6!n&;QVy3^A#Lra$L}5k)Z{?0wAnX) z@6(pe)xJV(dRhgmH1;~DJG!c-yaL7*F8Xpf<6vlCbg4aDIHg_Sxyb!M(a?(N2lreN z7mP5NJ~I>;18-*5{|bsO?@KQ+)u-+v>M6YBZuk7TGv;$$Vz#t;nBCAu98(Xoq|c*P zDyH0N*pt2kWexj|DP;CTgE-RmR3) zL#mWM;vQ^7)9#7w*MS#n;9>)Iv5hO*`bf$Ys_YxG5zVv?a7@EYOT{!cHp3EZC5aUQ zbj;$r;C@QD%y=N7QW>J1>^FCN?`_@xJ)E{e?)!gp*4%VylM0n!?SN*j%>-Nz!+^YK zI7L`^GI++=Y+;HYhlO2KOJ9u5~52wkZ5nbv1pCjEszz(;BfS zMS+>9VKFLV{PpR-#za{|C&J!%0YbACMWo}@#wj^R)&rZ-myli32T zw@=GlpR@V&7Nb2x#@@wJ`?&^O_!ZYrQ`>NYj7v4t(HsDO2)HO}0E+kM`c($y7+R4H zKl!3Bk(Nm!x`D!~rc`&f^QG!lMjgL)DoP4lD&?K~RBe2v{8#ybAHR&Pn2%Nd`~L_V z`ki^)a`w6HXmB(18?f&nv0y|PW4Bu3QcwYX8z5|0$-J@nQzoXUGMM8Le-fHNF&=^e zQ_<)9yHXydmD?jyAR!otf3e9>eidKbtG;y&Wn>}p2iJ!w=5XJqm*z%TSMQvpuf;*& zjwK!lrSGsT(G<~wS>4Dbm+zf}PLVn#t?@ADJJzKQ2>t6bD9k~sYgJ=Vqc|+obbo7S zABeM~5j`0rdb86|QuwNY*L`1WWd27jc3Ji0MjG`@C{FrFo>vqKh=Oy*moOgI22FmX z7DSO^3_>AJ@KY!xXH|Ls**G7U-Xsu)ICD{*EbGeVWauqrz zg4_ytAyU+Anb^EMXk}%1YlUHy#sMl0i z*Hv2jAsdJ0A(SaDmI(5s-}!7xrOMaRcGXw1wWzKBK3Dg%a5~uBKL6?TEgZe~HH$hj zV#vj|!F7=6?eNPI`J<`dNo;r74i%c-$ke41-SD?C5)u;5bcJ8jy4*CxyfH+_f${O2 zWxI3`99*xHx=H!7ToqmI-nW3>Am@xU7hnkU;l4>`qVr%$Y&3(F2{g>Z5TF$mx+RJg zpUpeIzhHZt714#b}fgNhtaIWM7` zr;LHeoCj)(i_9MwD#cRdjlia_jynGKn-$^JTca#mxTR1n9=_Bm!F#E1`zSzbgudq! zrfLg31vCt7(*5Z0s%(hwx6}Osq!$VxFt0ahCINJwx8&jP-&WS?oNhfrnKG6}*WS z$1ti$rf^`iCn(6y2Dy3r~#FCeu77ldKy`KH2(uEEi}3JvoI} zo=8(p6jGpuP&9M=kUD>`Dp3eYJ(lgrVX1-vf|^Hw)TJ?^eg!%+Xg;l2OkCn^gEspI zkxGGgF=dulox4-~s8+TV%1!We;>hr*_@Q&jM3F6hAS`EnO0>IL6Ij{-P??%UIJy>+ zxGYBv{6D||hCg!>!2a-GTz9q{Jr=zPb?ofc*i~*5b%kZPvi3=OmUNl#$~aBu|Ghc@ z#BdK-%|uMXjH?3-&Vkl(z%;%jAs+J495M(PgRgFGzOHO;&|?Uc48Rh`s=XKT-a5s+ z4Z+iGa5_ziwmr|}a#lCWv&)ToI-TWS`2Fs@iUj;=k3auDE?QA-40qlW- zjU7?j3UI7NPLuocj$Vj ziY7*3!r)pl zr|Ta;N|%~|Rv-?Ww^46fGG?=aO%c1>(L~7N=0-ThZS3naUn8V$(a`h$q;&ys=6-+k z^X_qV`uSq|H69*BWSa9j*g*Ma-RDsB=@(mh<$A5Lq?(%IrOv^@@F)7rK}?bF_Iv$_ zB*%%Pz0a4H!5gK%kDwuY{$9^imj!t8otu!*>+oVm0sUE50fH~nE zw;TG;yY4l3HTWL8?v;dF%_W|L0XOcGX}<^gZqBe&R_zYJXgnO3L=_GW&y6(L}^@w6lwgcduQ4OJCHd9u8{&9tT6r>@vP+ z-23`0|DkZ{s!shY$JB7j@NDc_^aC|H$n8n!{}Mj?7sYlPQ8eeZVo)3GLD3yROB%nO zZkbbFS&1hB3{9?S;X>$n=i|%ou;czw4;T=BxRx8FFAf8PwgC`MBGY5p{zT!27k80i z!SB;bV3!DG;WqG(!_DONJCElXC&8szw$s-S;wbqYbB~1sl8ZK7&mm$$xy1LrUOG%H zonHJ20Sx1pz!! zqAwS*7TikQyozGX8rV9pgL-+|L2o1RYyHuk11z*4NKiq3{w24<_R@Bx%1BHqe?iTd z`3lYVH~n-#6)(%G4_U0WSsyC(dy)#20CQQM1@4!uq}2r-DgfH+^)~$DMP^IbzScJ8!dcueH!7^aSfglb|#G~&!cZjyh_kHy*6mI@y!Fe z((CVVltdBtd>%UPs@C#qLDsE9F3Z7D1KW1)Bvi|ZOwrcz81BTe<&to7*ACks#K8bv0(v_mJrbP*M9 zB0l#}%OR;X&S!r1;l=N7bvz9j0)T_ivFKJyH@CXq)ZVAFdmwg|AMgH$9K5 zN^?9>dKOY&GvtavSTkjR{}e$1c-)|Ce- zW3rqArh=RP^D`=8J=x03hERKqUYc;#UNf8UV1IytwG@9sL2E3~2;q94FenhgaItq~ z`TCTq$vyEI@H3DZpFY$5+jG5q1>O0(wH|BldILVPVAXAu0pA=gyb3yXxeP^FcYl3( zgG$Kyz9~P|}2MTo7H-vzAl zEKCPenL@UUInx6Y>1I)QeV~zbGLU0&15-Nkagdg&si}$4?_bFk>a#UnPpOH7)%&{5 zMhDfnl2f(8*MP{1-|1vvt@zmYZx9{9RIXR8`F0Pw(|M3Tf_Op6mp!O?Wvl*b^Z9bK z-`JY;^!w+z$Nk9+u7b!#z z|F6=|X}1g}oTw$CN*}WW_AX$FxxcCL1BPFx+wp=rqnh4#e)5P%m)e?F&sTIF2+$6r z35NRq?I#Da2z^`oQ9vw7EFUj*oE;bwv<(e&%c8zXE>33)*z%;{;=*!p_G5D-q4sWl zD2u6IwoChDAyBeqCI}~+*P0*Db@zVd^*T1UvLdM=mt5fCeiREo8kZIb&msNYFfB?a z3+fvgP>wGvkA%=3RS>0jO=FPZji&@D$dc?`MALqucB!P7{aMCyB%wbLvJ`i9oFAv? zGZ)-u)0Dn3H>{+!7nxe)?s~)fDMm4!^U%ZcO(LRAbcF1W1z&wv18He#VT=0w;GIDa z(uE*AQ4(w*rlhjEnl}zGY1yF4r-@Ps8M1*wXMN!8?(ef zMA*_KcyVdD=QX)FeM)0bE8OBxian z4krnpye7ovqHn$XEp8RHv;FQ5wf}$RPL8fwhaP*m*?x_#;n(Y#56ynp#*E2~IiHgn zH3hz2?yj!h1(%hj|F~}7e1f*-8=rM7pO)JDqmMWm$%6d}F7q)A)7`CAD8)iW^w1I+ zFk(@_WfCxPwIF|l=(NWyey8nqK0|ksI5A0S5Nr~iQr5tPGB*zoU|#OZ2p^uX^5dO| zH=w44r3eNRI?USg+elc~9aQ9(vtfcv%uJ*Pide&=kOkdk<--a5D;qyxkja{7tt%{q zawo~frX@tkZlnh5KXW^nf3{k}YKxms%Su#96LNaGN=7Bmd~f+u;A#$0V9T~XCRecT zfbqAD#Q9brWk$r9rG}4{hQe370NgTtq5|?TQvRsLn?&n80vdSAkU$rWM1s}8GRDA>pt`EO+$mj4eO^Z>P2m8i z#)No8;+Mh^o_&OhS_)q||MOWQ8!) zm}=@R5P|^mxd`C4sURU_F{pRH`&4M$K@~4&X~d9>Mrs4qE%_wm_VCs=WxfqU z-WZ?nQ$}zk83ztFC4v+((!j0z%{AsnMg9IKY1H<+cWFGIx08khkskr;l6MGMj8cAvJ5o%g>rHdwcM5XI6Vp2oZ?g$J#dlD!u3?*dlB2C7_UGWv6V9uu zN#>Vt_1jmlC6_k7++RjtCoxP`q%01Z{khtqW7+=nQT5WK42!|3sDl!zALGUiv&4mG z{=$y8-F><`34Qw_FM%ToC7%9D6^5eGBz}?s%OQjRa68?F14nv_s$|U6JRQ(rvGfmY4zkI;Zxp>@yO=~E8%%gyDsOeE=vGuAdAyS{;%YI{JkZHj_tKG~;ItocZ zI!}7kJp`as&W>`qt!E#iGbK34-|{~$&-5oQvTAqQ&lT*&BH2kR^fNOL+GNr`8+JdM z3s2QN?lm%a+OG0A@gpZjRZ9K z@*mD$V-J|NpH}=jyGtWcX<^~h>=@8hxApgsE|YIzNbQ-g+MvyQdCyD* zQmbMTmLL%>jb4t0XGEX46!5Y^{XtZ>+4m-+@fZ+4$}s{PK8D_{GW5br|IgKBC>@a>)WC@|5l7Og7+5@|68AH=$bj? z8FJB^WuH}K2_%b&{}zzOp~&0UHe0gKD`%ILN4pS(!mVc$PUUWPJ2)8ng@(UhZ)Tcm z9gIAysj6Pk+n3Qi4{2h8ZLDP_jsgeg#Wwlj5pntH^!FIeE}d*-lNUd<19!_=UXDzH zaqZUF?0$2gtWB(mqK#eh4GA`kvmwH2ih`SBFMF<3+1tA+kH5X6 zxJO|PK|y?BEWN(-OJ`buUhw+CWMI4*5_DKq0TCZiY={Dn_a9 zlFq%C{ay^V5zeQx^*YIed1%Yd^Y>6RDG-qG9*74MOvH^K&RlNa={I}5*~4VZ7S4Zf zq2O+Kx1|jIlLTq3si|q=G;Rp8Wc}`Y{qm+fF6}f1vl((Rv^-aefsRhxn0vOIs#v@T z+>E+m=DQP7;AYOU+h@y^c>%-OfkWV|G_QF$x-%TgnO`hd*7BE(fJ0KpDpmdg4joTM zyJ|&W=nJ6utMT->7$^y)$m{CL$llu~3{tam5!hyK? ztS@d~cyk+4{&l4ICkd0cHjA#i6@>Q}EdJBkR!8u}K5kdFMzb+YXcV2$o!b2J=ih>K zAT-_5kW~MpF!t4B4R$av;y@SS3SEN;B;yhs7bY80K%9nMOI3@)&OqbOV}ne|U5nQ+ zAO|}MDJye~!G-Dspp8_e!yyDDfrpi6l^I5q*H6;te6K3#PLyBCeP4%r^Yv1q!L*d- zIQ0IY6!onrbZ+%+W=J4IiafST3XU97oJAd!qc{z$3QIXmf~XAYcYlMFf=$wh*GP}V zfao8`#fS#9N`=)chk#X8C5Q-7Rg@_P`k=K=b;^l5!Tv}BGDKx=oo3jWy;3@q)}Rm) zD2442Am(|&?DnQgJ@LCReC6j)8V?9T?~%Yl;D`d?O9XPt5FBk2n1*`J@cKv(8!QP) z<+wSWm_gsGO~W+Ke9uQF;C+8#9+trlt50x3r=zDv8-PtJMgg-{H^@@hB}F4-lukdi zANXpdpFGR~H082GgC3=sY;g*00@Qk@TptqNDo;@u9-T)?VJiIj_5ZGs8{L>%G9goAXemf*tUcx6{3mTA}pGzaSwRby?0-jEe;w{LCUL8a&MWKq+Kd5f5 zY=jRAJwMz3GLVof&mdNN*3pKF>@S`K4%MeuEAiRMp*H#@AOr`lPJ|KDwvSY#0#)tG#_+9P>Cyzlqd3$3~sLHymy$DqTDTW-i zolvQSTh!cBLN!qp3~?dGj_@#c-O$jqlR@=<*owgl9JN8wOjJv%Qq9>vAW?|{%!}nd z+DT%W%#b+hYG6HW(Z4z-;;x21GReT~xgK0duPU1#8+CTmR>n0B!GfPKm1XivRk1zqQ=(qn?Z>&) zSNUY|!WiBbqVF|*8W(V}D%6A`#-CfEl{~LV3?W`9i>nWD0-~#FA!6ZH!Al*|m`E}( zRDiE|%F*D_t>l_Et**-mUJZw7GeKMB%RWDxO#hsCvDcv6vbG)EKQDaUKe!>=zkoLoL z@Efb(xgB4NvaiR8a!7H*lgjh+^IKc* zXN%>V8yP#w2`8+wT#22Up6n+?q>+^2R138c^-z{(2C2Bhe#aJcu`+NE-PhnYFKw(6 zpu#x|go8^Yq%_Jack6N)P{s4mct|+$C@g6U_nH*C74b+2LVr!9&xL>IhJ%HhUGOwE z%Ik)f5cE_%_ILqoGuMHL8B3YPDut{_mCGy5 zbqS_3M!txD0;7h8IUQRusHd*byTW1^vC(Qc_D@o=P7Hcpe3CtekX3cFKTm)0y=Ky!v+!)#hM;aI37R!eIg+mZ8g5pVo zt%1x?*^8LBhTH^oC|lvQC@)F{zUjKz;n`($=QiLQUxf5-`>bp1!7XLd{nA>2_QC%}nF6z1A?A)zu)IQ^rAg1Bgzx#AV;J<=2S`3=ftH zm$Da6uQq+r>MH&kApCYL+p4l1Z)&qm6Bu3pFnG@)V)S#1ypDD? zZZoTw2xek`Ji5a9`6Tfe#8Jq@3iiM-;N>hS2;}S9*2W+&YGIYAj4pT7JHx^V{NX15 z@tP?)n*p41Dx%gp_9H`ggFlnQJcR2g{p5+O{yffP(ki<%eSHRaQg&FX@1HbUwe|;R zBTg6nou3Ph{6Vz-SP~(~=E)QUD6r8sYQF;LwZgOtNeISOV7VXM{<7}zxr+H{x8u_cv zsu-bVdjygD4ql57s1GZrP@lyXIQ;B(v!%e!zc!5buXT5OU0|$W3W;TIMb~j|D#pq@ zBx8bkfw8r8pQhofs-ir2C#m!%JQNP;os1-3?J(JV*5g)}2hbQ>)HzOc-#1Q4G-u_h z^nnLA()bSF-JTb{_q#t9=P0RWrQrJNQuf1LGDu^|L=v#Uf7aC(PQl-~mF$70%HMTr zsArkBCq0LW!;UGZZoQiH_LY^ZwXw9#A4#sc7 z@%{Htl|wB#LyXM(;sVomPc?e}je(qVSo7frT|@mCe9Px4UeYxQyX$uo{O}jrqODb4 zQN^Rgn~nDiWVu&h`GT)_Bh%HPex3M}vsG)))p`x*fvca-MFPT-D5!*w^cl{kg&G2! z+`7C340X}8XYD-8=y=T;dk5p@t!!0{A1bR_ zZC~$Lh}yCQs$1g%10#pYzKG9&sEZkQR&jA5VtQSPCKo9qs@}_EJm^GEPd^MOs_9z} z*Khf>dSP+dHEO)mg8l~VSf2SgI#nNUha`T8aK)`;zvvOZx}Ns*YTsPHrBs%Aj`yQr zW>+oy)$0?Cg2f~b(MFcCVYHac&OMnx$iU?+WaN1W$0R4@J}*SP*w&EO7Q(YUpfZXZ zP)DNdIZJN!D%ATAY3PzpYKbS9Vr;SE(vnFf7uQxC=0A$9^QBjqUc)D{Y%QkdDtapOZl^Pjgt)tA&FLX+bjnzf z2{*hrf*d@(diG2NmgJ~8JyXkItE;{Y0*&)|mICUJ*Xa!}zeOj#`>sv>c^_C#Bt#HR zFNH-IGCUW))yI^ew|Ew2(IPqN?PurbRW}=$4jZ`_3vCYT+Af<*rd~t0OZ67aOsaTb z95jFqw~oW4F}-N(abv6_stbt3$Ubn*#k zZl`BtEUT{W^7t#sE5Rs~%K(ENXr~*B$C-kGryw?!=8{`u{-eAB7$D5B9raODMrBnB9@A%$+ zwO_nNKVFogif`A?9>yR)`u+P)64$5EODQ1_9cF^aK~5Lv#h`J`%8}=#_4RCp0rs7* zT)GF?2g};}LblkSZLStR2Bp-k(_tri@t%`$awcTGGDvpaa6L|W<|saq1OvZo3-a$w(PBC9?em^Ec*E06!ON{afr0sW~f@O3@**hQRtk zPGrhGl@Vx3wsL5T4-|x&s(yQ6Z z(%KC|k4H%bH>r%J6ofAfii@KpTT1SkGeOD;DBE2jmT8q@b!a|EU_y5bTc*uXq(8HSgv=@=z}mVkyQE?g*25viaHln?D?J8 z8$nQU6tiL54@#HPs@1|Wvx4Y{1#X%lY=S@s-FyU^$`Il)fnLJ| zWNm##N`1=~#T$b2@sBV%L;v5uJFzfGZ{tiTvEH2avm@;~(tbXz2cy2_bYBq?iSXPd zQPm+b0_Px=Q^>yIpm6*@Y`q0c9bLFKiaQk77Iz91Em~~j?(SaP-QC^Yi$igDx8m+r z+}-Z<`_IY!Z*r3zl1+fk%&fIudjwYl96uXfHc%ki_w_0Pmp&g8q=r@->d!1ExPtmw zA6mr#41$v~_G_Xj!0xQK8XMGh$_OVDArtN;zINPe>6n^hD1UtI_INO8?X(`F+k9K_ zc|A~l2%c@p=D1f~H>g$nn*3EA7H0F6)R2vaV&Aw!8=kn1E{SAmKGi6J8#|Pqs*L}B z_|{@7q!B5#PP**KKP5EQy{3H1av*}=%P*|w@TUGNUf$2a)*>e5@s!|}FMm?b@4sS? zC5W9@GYEwtSZ%ggm>UmpcfB|%Dz@dtiauZOk7@8ASkiUxEnF-$6qfDqu{a?)8X6X0 z!?4zuRc|(Wyfj|#^ z5}Dj?r)GV%foMhRra4}X_eXd813~1)ZFRe5LcqitLntizb+0mq{CvavBH+`IHNUw? z=-QvoXJE!}O+!!9vbcWrR6?|Fes<}&eZJmy^1ZSYkTR3>a&O+2GQ82W>0tZg^%C%~ zA8FWNi+x1NRMMoe>(4ktq^?e&H0 z-^};u0S|yq){nKBWl2`b!wT&-=ey$u0G#7)qFM=6%q77YcU&Y{t_3~ zNSAPf_HRA!&dox7L8x4<$-7(Fk3OI*jwOK(1%sfFlS1{!K(PT9&i=u_?-P@Ie^!=< zVZc0FQ~o&Zx=!SmB8l*xp=ay%b@(3tKnSZx9Xod@ri8_kKzY)k^* zE%U-UXSZM_jSy#GEa-AG8r+xTd|%Xbv2@=@(oiVdUJZaf=w{;|`&@L#v9>&fh=BVx7g8%IcWj-O=+>C|p))Z|r_Gw4M;iu3%@2bsMhzCa(!pFxxHJ+B5)`PTl?vNJ`M%|WU z8dH7c`M=%kQ=d97IE#QC_;%l<^ZMZPA}z~PU)lK{q}^~hmCZS~qN}C#IOCuxIfgI| z&-X^}_eEU0#j*2hrzjMV!Lu?b0u0wY{*`@YMPo{Tk$7}!a(i(dKHiD#Ngv;fNQ2b> zMY3ySvSIM_E8JlTcL@;a+9>E|!d&?GWc#PW&dyHfzS)GA4$r6SquCO0zU!qvkBRX# z7$ZTE$WmG(KACTjZjW#{>&dNPvLTMYHDhjR%gdb znZe;kF^3|}pH)Lh;UcdS{WNi-tT$Vadebn5@S|}Nv`_vr(&sceh^}WPQ&L0B4eRN? zWF5S?b7-l(26PZqX>qJ_?>IX;FIH8RRaIHHFT=~nI{N5to~<_V+Eq2(zwb>fx|}U7 ztgPS%_)}9<1P<-E!UrKne7D+g-&uM|nam;-e%ku;=H&Cym!6(JdEx~I^Wv_VVM8vL zs(=RhHuZ6mBbZ4Z`r1;Tk7O)1|5Y<@$+3YIQYKa*goA)rDOU^?zD46CIyyQ#n}HvV zM{YP4JudI1<4fg)UEnM&dA@*JOGW%r+w6w<7BeQe75zk13gh6fJsA&EVe?cZm-+^TOgRDl_2L!DLz!$2??mKK z>LNjO+Odw(_fEf_&Nhg7ZyU4yYb;4x9IZC{v@kFvir!EGPc|b;Nq1f>H%JMNxD`_Y&-5Y5!lY4{+&+(NFikVo} zj}6jx#uh9cwJ5n8UmQ%N*Wc3hiRbJ?nB~jlsW(yF+ERVY$^zNlyODk+-`8ZkxGYK& zbUDt17xtwS~x^cMl~)0#aQTac21f`XdbW3S|<`%{rkF&YGF zo~yh*jXd9Lp7ALk7A`4-sJ$as44FUBP>=xS&V?(>IyJaNMo0TgP`QZeMLBM+XP>yV zc0IijRd#wm?X=W&tbZZR%-|6f1CRFGMX%IsDQwzZ7&cogh?Qe6u0VCm&ReMRI9aSh z3=-7gN|JB*22TwqJLZcYUZeoCU4w{As!@AfT6JF)w=FMpGbV9}735@NS{KUCnf6U9 zjY?#+yHdUO=f!;#;(;TpFIL$a`Kg=(K5ZJB>G3*$a5N!N z|BF4$J3~e^40sVesp(jBuNq*KZdh=>j-<=$^GrY!i;x#TN~FBJTo{U6TU#4hG)4?{ z@?AcSi~toKfpmrwV~~v`6MopN5KiQ&x6(-3EGe&ssL?9e>p2V^e&o(V1@o8KZW6)Q z(J}G7Lzgo-_xbqWW~}Z<2kZ4lQ!=IgGvDy-z+mJ9M&O^`4aWVj&M|&@)u?^d#RHLJ zM^Ge;1SKg4&pnz+R*?U}?>DFgk=EAy4o~XR-h!RGlx8XZfWA%eMMc9i+!CFKhzQk3 zD?$PpJJNZZV4$X5ZAT^Hm&{_KQG6Tp^t{}SKFBZ+1UDmxHKZY<7*5+Q#$VnI&E?FS z_#N**pd2Cdg`>y&QkB$?5(^?1}Tnep*F0g$S=o|kJAMsduT z9rAraoWc|Xl5~eY-hX{VZDl@3S!S{nU2AAT=*2P;(&#hihl=&k(TAz1GVr{OFa`Jd zRsUJ)z(ors^}D8*2t1%yyR5wMZrUAuG7RC96?-gtn}WEcBfjLtL{@;nhA`i^Jy?&1 zMpoLMpjUnb`jn95e^SiP<;>Zk$wmCJ*gE!UwF z(1!oA*lJEi4EfFHg(kp&;raO)g^-6MBpIP@ zqw`)`Cup#;7(ScMPIU(%yGtaeX5sz>(J$8Cu8ZFtp{PKNE8^dm@ItcDhu+VNc;Kp` zL7t`GFW{78?E-l!cO%LpXvfyRQ3bSv?S_=cX*JdB4G5}|i8HUiF|!0}lXt$1w_g0H zV*ritWz?|ry?AMP$T_Us23`yz$#~x3R`X>XSE_$aW_k_$e57=JFp)8)QeCN`K(=#O zKmn$ax<#|S#+4M20wJfzR~JTi73bGhYt;JN1WUjIS9R=n(fr#RBdMQ;!8K#se(y0|&(;zJ%c zxuN>!YaHwZ2uhYDNxC5ea%%0}!Gs!)6M@bGyY_qfm6{*bh8(lDMx&e`ad52k;zWAF znU#z8M@bBdj~xI&brOz49h}NQG9FfM^$aRh)TVbq9hVkxfw&%?&iivcfEe=Js!*CJ zrGTq1MYoruK<-*rtRQtN<55DQ3Lk@n99`@>y^{ojVn>KvYZ3u`TkejOy*qz2>y`;187wQ5pO)A^)QMWkrH6nGI-idUt_kCBSe zv96R}fKL|SULOGa>6u&8VPI{^mwLNbu*q8SY?%eeu+%j?AN6~a;6jKBIG|yq_A*B* zst}Ev!pmPootgZ{$IG?q!CPy~FBJra9*E0D0GqgFeuW8@??r=*DkmYY5U~(X37&|m zfZ&aYYL`@~6>d&UP9}vfh%^+74$-5icebh#L05>2us0z34C4zB$T8#7$1TYYM={{XHz#KRs!gc$(K1-LPLipI1$7+EF&XfRw z$y9<7M!`0Q{k5!p{k#3sYs(w=F;7CXQ;Dc~Qw_0W-W7f~F+1@jhax6+((9wB58I#h z+a3>}(F1=s&rmpdF`5cp<=I(h4@XC(Vel}f182ptPC$0{bY*>c`U{OFs)9r=f`o)B zz3zgC4|yd5u!CpJ`^KS2AQDDlNGU2+D>deq7x0Os`GexZ9<;Q6RJWQ0;m`u&GnJWq zfD&4a`zab{d;v7&3@7{np08hd=4ES1@?AZ9)F3)i$QH$J0SJ2)a}^=3 zbi>_1!U*Ut-uQ+V8wCj|HWk3!vaz#AO9+so@1zp72qnTLpzpO}l)HX)-aT&ka>uP; zZhaKCIQNwXPE>3wmgx^^zu~pYV2P+FEGgj^!q0@bzcw4F_S1R_>#Wz)^Yil?qh&s_ z5d1<<>&<~<3tLjh6Eb!Lg}H?&Dc{cz+~{7cSi#?jaimNTXp{Lt!a_hsw6F|%iXjj0 zH2_{YAi3&)EGxLqb3+%-AN=^fV1Ii^b*=G;=cFJ7e z(oL>e2jFUB3*kv*B@{64*idp4chqnTqFuK_MdJc7Yw!s^L1*{V;E9#VkaV$^V0Dfe$afmc$D3puQ*HDMgL&I6YMclS0H&K#AgDJ{HpmGYE6X z6bF8gHHriMlO%dRZ(<=D8(dn57hT#RXHqPiYVRWE*?6h>V2bXi+rN>z3u@C#25HfH z%#fH%aeKph*QQ;MXXoqO7T;|)-=s*EPTE#I-q*akn8WO1KYx*We^u>TZnJm@mR@x} zqq{!Ih&d^ z5!jQQ83&$1TYkG`&Bcb_!$yneO>$l={!NzLF3?)sn}YLE!hN>9Gd%uw8n@p0Nh z=z)W&&k{%)`+o={9u=|RvOZpW6i6RYetYw}&ZS-935&h2%kTWd#*WjUKI z<|olEe7x_=B_a2go2}bN^x4E{1g$Hqss4~mQ0#lS_Qai55iq%&b~~<+&DMR5yiU** z4<1~^!OSzF=fUWK;$r;os)CS5dw*dsK8~Ke+H~IE3G|0LHa7u}!Mw(MYm%WZ2-1Z= zQ~z2@*n9Fq-$Dzf;bJPEyd*WSmRP{>F$wUbDH5QlQNae)C@0p4^IA9+Q0ImVGvTkt zqIdHGTxS3gplcF_j5#kZ@~a)&*9;E};l1u@g1(U&QF%ySl0!HFOa(c+Uvcyn$>$zF zpI|S1W|A1f$^}U{5;f`!Hj3|Lld!Omk)w4w*BtBabo}#!Q5E{ilj+MeyUOanaq;lj zch(-MjOl6M*}>>~nwUB&5t(za?P1r(-;Is3+o&mPDd9@)?6n-+CtEOHRBC9c9qCwgZXm@=b=JD!fVa8uiH|yOK)OS zSQvT$g|jc9&y;b!eVVhElnzI<$y$fIx{x)kkZ#dqihV;uJTwU_KHC#+P?-70N>`Z) ziL`yYJ?cq^L)Gp*4pvZhfVmK93>7<%l9Ezl23nuN6M$0{6%j!Qmn7U-vDEqsj%SbR z+KCIj`Fjd3v0GnZN7jDl5L@4$+Lc^NkCG~?CYIEdfglJIOoY0UHK8C?2pu*ypW3&k zKdD9(4Tc66%>co?a`QRk`}4(v_hpZ0coe!UJW>$POt2vN=XiOD8BFLcvG{HvM43=O z`sMe}?%>(S(;^X%+xhDUBAW}Lz=x2_=!t~p9jX8&>0*`k>deN04{ISWsAA@%=sJ zLzk5BVfbqJsxN{X7da+S;51{#Xk3H``q3CVCX8e>mdq9ybe*j!wAqY}>VCX1UQlhx zi@~8LnJ2K#L>J=ikUg zZmYX)riX}hx@HG3C(&XOeIYoZ#%)8jupiA+#rv^j#V^Yz_iB_q|K$z?%=F62%6Ybl zG@)Or0_{gHF%Fy^>phOhQTuNU@MnGJzUtWT4n~!#@Y9M$re}&TF8Wqzb{qqUqrp$_ z>@0*cKvzLY#&4H|D=`)lIcn;n+{64LeF}f_uILaCQ#$q?sviy>H}N(XT4M{pIasVT zE@=$shi=pRLBmPO1+ua#y}R|HgeVMS;}F*6?Qj3}J6SfJ3!Yi8rk~=|+3P1*x)wjJ zB+V1tpn~WAlwnEcRIj?eJVz<=_I+j`z!hUkjKz;~)75ItX8rl5xbN{$(em-W|4zcf z5zAqBt-;Ln`xkjVa6k9{w(l;&!eJ)sQ%H!1im5<(^tFJ$7t0zeK z7_JuI-*f9`;_m(NG}QX!9t=1O0O^%QFi{}sE;ab{&J(mR=bsCUy**iZ(zJ3<8-$`< z^3bqRq8Nm|oFP7E7(3Md{a-A=5lbbHk=e~D01Zx9%X)bQx(P_v1#@x*__W(-gZb?uOP)6M!ui?rz%MH6V?wSGWa zNJ6kAl0Is3ASHD$POcdqSUqDi5e}mbX?L1vOKK|eWZ#PZ5D3$?YEtZXdV$tAK^TU(3^9}(Ua#w-H||F3W!p=o!$1wc?;K*e zV4}GcVp@m3{dNi&3ydetv+moLyRnKZ(+)23tc~Vx@#R_HLDL5fvQABFJT9GV$W0B^ zoC>(h?N!C)cge7f&2G4=Mc*rR7navN&&QJlw9}Q$<){Tw1U^OeU(|0tgo*^)bSbpt z{?tGEqo?Zb+Dx|~X|rU8IL=OP+))?&!Svz~9%`r2DkcK7e?* zes&z)%=~AO*VcwZ9G*;GAariA=c5mZ8YW|)4sXgL$r^9=VU|N^d4TjIXH3QC&q0hD zRt*LIJ5MrhI`47uU=>;KPqQ{ma6o1qegzjWG@e`+X)VdQ1= zY&l&G_Saw3y!Ox00=aX;kye(Kle>^@2CpmG%;|`J=y^{!2lT0`-sj`+BKIOty`&JN zQgeuV9nwkWDroK3ZpJqUs-JnCs6XJ~!6F}Retxa42^~3*q0FzviJPx`d_Z#)1u3bk zw-h-$$?J1uv1@;!zvz6J79)+S5IVjxnEbfaR-#w9snxm&P5c*i_gM#-Y^Aye<_;Q^ zeZ0Rac2Pb=Nv$b4#Q-4!k|5Ms7-M=XjQ%Hg4mO_jWHYr8BiDBb)7gWDn^EjQ4)wt8 zNe^*^8HVTT1;$u^!!7qcq>?$a^n7bqN7q|-i^b%1Zn_jlt+wUml`oJ=_$y>q4aEOi zt^6SJ@`z#W6Nt-{g7MFPZ*R1@6BsCUBn5YBLww%$yNo9fm|DhzuScCkvi=wj20K)w zz^CjfBS_(nOIhnbbCdePd`x{jyUYbQ3q-T3c^BSvu*W_ z@BNUEWOp8-+lSB>HOjbq;)h?m%kg24zJUS$Un@XB_v*vU6{L@6v&$9L%(9HZv8lWq z7jdvxYM7Jr>3 z#GKMf3{N(rqGMiS`jhLyAWWbht3nWoo5?02_7BI~9b4K{k0+C&+S*Z2YNkpwWf)G{ zctc|OOk7x-ENABq&r=$dLN2vDf-TbLn0oLqL3wbUwYEDQUDdvA$#{vgW&Qq-|cf~=mJ*;#-z`$;NNN1lq!>N z90R{NN`6yuiZEcTq}A&6ET6x^FkdcM)*;`8@U%(vy@-)f{$-*gNUD5bbbDG4?h2}~ijp=SDB^D}#O1_ifwYX`bK zWOpPk5nVMPNDmB8+Wu>{Ry*~|3eR7DJTN&8T)eSpJ+Sb=cA&Lv6mN9}=nuI$qL$;WHWa_PF6A(rh&T@3mho zsm@n{MYuP-yTuIZp&SL3k$6N$+jzSO! zAC2alNZfzQoEaC0BRRctrzv7{=ayZPzfUOGX)st&1A&VV5z`xv82}}LYG8-_nF>?A z(LjCk4{tGO`87zKZ153H=HS@wAI1P0jPP-mE2#uYIxfZg_mD&tmE?w^;vKhg5T}$n zpWpfU`9KojY8X*+EQa~?uiMf=TvjJ}U%z1?kHkswEho0-G&wZ9>Q9#UM}pr2Le;#V1??%i#2#t?&c z%e?Hyy!1(qMrYo)tOXnbZrJli4{B_V{9dF11--&y+iktIr@h+=`i#ZS*By7}?3B<= z>my_5T_J7An6#D%ZZP-d_CBmuof6zb)D?5<1O>qc7j?|fLy zK05nhboyOUfCFQWbr_QQ36@P3x6Bi*#UIO~#uqyl3S3cru-b%bIT9{543f>t^Q1Hg zY_gxFM=6={QwFzGk%Eqch31BgLpiV3$&-eMtzygl*C$I&SC}Z<&Q90YA3td;eA>#aYtQ)L9Tu3?`vsM$Q_mVdX`(&(7k0wjc-rGQNr?Qy z2sBu`6yDGT5%KW+;Ljw!3eo+2%zL4JfxMXHrGLm&6Ya?Hw)Mf_m3!qF3{Z9$s7O|X z!uZh!pk?C2Qxo$!;JF9EiMdMATsTXiAr^&Lt0bUonIMRxNcR#0Ec96|=tX901QqI- zLn-BVxzM`@+h@Ud%nRL9viD!{fo(!yL zEYpaO9ec?)4?pHzf6w%u4t~QZc0bsSmI7oBcGLe02x=z%DF#3Or7LwJp-ePn(Xqsq zo>+Z>avwW%99nw*%^-*~0a!Kxh{bL;Ibtc+Ml?7rC|V>i{CN79J7d&ILOeP|eiF28 z{@Z5&Kkw z%vsF>y@Rv=ayNq~^lymE#C<8fsKzQ%C_lb?Nv({)qfv=W<_L(bS{D1B>IUyMS^|Jk zpk91{lP=-Pd9T=?eq_O&q;|8t=l-8-;WV0T*E2ys_3``yD67vdq@jMp9xky+wKa-o zLijdvPS#XSbM}KWCS=IS3OxzF45vP)3GLtSf_)-DG<<$;F(u+$7!0W3D{i;``u8ZZ zz~qNfW;7lMERB9=RYQg=`^&0S!OY~6vxz6;Y!pun_^6b~&I1lo2-R)xlQERVgZ{tv z%-Gr$2WgLNkI?v*OrGP^8$Tr(XRp4m4HvixN(C(_G?ZMnOkdZBNysMwK0T>+fxM(N z*Tj}~^_s&J@53Iijuelt=2anaKTOmQOM(0i&5rWWe_E`Sxs1YU-yBE=l@H-lw=Lmt z|IbCYaFiyy5Lv(`nwAHF^Anm((SDzN-QBWlvT;KvR_V=7kqEJ%%%DMQkIU9%5UCG5 zt;!!@hIU9mw92FyzLqe*}AYEOh_qMUyA6Fexgv!N3;+wTcIcJ5= z#qEgy=N|;*p-ZI#Kzq|BOX`wIz#ZY02>C>wQ`App#t+8BfE)zTy*0GZxi2oj?@I+( zCS7M8po(aV`>%PL(qoV1B=KlGis~J(I4*SCo^yR7j+g3tDwz(6mSpp4zvM4r$$+_$ zE#_31W|hd4ns#AEcvq0ip4+n&VMDI7G+kS=fBz45XD}j^ot4xDq7EC%MpmWN^pq-p zN!@5Na$oOP!v4&1qgvbgFG+Q?+^g~^3FT#SFo|!X$bfPg%*(#6X9^%|;r@hS=)bqZ zv4?ji6&-m&hlxO6rGXg+0+G8gPaJ=x)KnhNvoe&~#UOK=fNwd&hzkX00!hD~+{nF9 zKe;Y>jBqz{5+ngj4LI0gW2Gop+qhA<^J)}PE zoP+g-sru@m#6b>CwR8=pxOq0CkXzKZ1KO{I`B_Ip>^M{X$yUq%y`JD17%SC1-=3X9 z-9zuV|D{1dBc`FT@tMeUsdsnL_wj*~zGX({{KtHSK~#9C|NUwPt{7VhPp7nxC67M& z0y;25RqM<&^3ExFOq;AdO|5vjv_cnGYRp=G&N@(;lB%aQ?UncIBr`?G7P<>T2MKLQ zpEH<1-K6-m6$+EHa6t1Dg9?o*P!8vYSdB+ki<70YK6_0r%{}?DJB)(JzbP(|FdIU` z5nd$51k9y^PeNW;SauTmc&?45$pa24iqTfdXG_27SO0tYM2!w72f^q2!&%hfq&>sH z9MY#=q)-`gq`|PQT6fl?$}eoC9jve}{(A%tLGg?M`DsMcVPvZF48-7xHQwV#ilNjs z@54gh^622_7^FU_T)~liZxp>R9<(6`Znl&Gr6d-HMru{{DIC@32S12J^v45Q3>W6> zhwHuJZ~vX#o>^{Yu+bGp@lLSbTNE*yFyxCn6sRh$f^M|(9E%&SwB=E!>Xa13ly(DT zXhNvJ)?l~BbeLuKvZzH(M3`%F{%x3XEVtzSS&{uz%W5Wq&E#6V5PM7Xp+g5ezW7|U z>=6h}_}WeUS~8>>bkmBfwU>19y*Su8c}^h(5XchjTmPa5m_#2Lf7s|tC^MRx{lT9r zZJMG%3ujC)QfTlnQD0xTS3@^B+KjCyL`)3$2q9v>QMWJ&8f*hIXv%Ma%HR5Hqpx}6 zM;vaK+Om1ji$_dL(od*}niTy0Te>V1F0-Vve^@prS*d3Odn<-Jejacob}hY+ENx&u z2~*nUGJu=CZX^*C{E5Cnk^VIn}K#x5(2y}&5Ga5+nEHsowL0_08Qaw2%6JrhhA*bKjl?K~|UVTU* zm`6)>ez)h>4{z6x@)D%Re(UygL*k$1tq^U9ZuD%p>~c!0R=-RUc<<@D{rfeU3IUu6 zQwaX`w5(#pOag<4V)*1ZwjAwY41*nkvL2oZDu(=wj?Bn^qW3lf$xsuE69z7EzS2kv zIw`<*TGQv@6W{B(Btty0>A>iG#ApOt$!~mAG@ty0+GHYJZzuaKyPuJ3En#nfcgd_C zX@Q_-lZ|d?p+Z`7wLt3Hl2q^(cj34Cg~@xX42xuyBJs+~_1~W24Ty}Q-}B(2PXaey z%NKRM&H3WMkg(BFkF+$gy2e5Iy_zR347v(gU-a*>c~LWmev7|*f z==>TjKACVSF+?A`oe}`MAApJA8>;rnaaP&KrLk1_tl)KO|S5G$A;8PsOv!gtAq9TKv4W*T2{9&~C{j9~_Adc312 zkIk12rK(e(=UtyCi|m$(BfVPBGgNd1K}`GgFLzFW#B~KVopHkX*%6o1&f`46_b8rW zaq`uf)KUnr34UtKvQ3>Pr`^Hu-wN2(CFM8cI8;?yUHZ9T=x~0JHtF4DGmF89U=-tl znH=~%mJ@{_sKVFS+)jn;P+}uu!h$ZFZlqg`z`Rgac*UEHT*K3 z7E079i9(>T!p{_Evd}BoiMby`%dIpZi1~_87hjQKwJ&S~))xm^>DiakbAyMpRUaY$g|#9MU2(_te=+t^V7JEr(}Q$?oI! z6c-m)yC*HnfB8f9H_CCle@<}!1DI#dA=GIcp>)3bc+N6eG>B=Y_5dM^EtDtoPSMfO zxSib*y>EZ=2_1?Dg9=u1z8Q^4W<+A9MMvhgdA%&6P`+W*ZuWc?qe|3NZWy-fLsFJg zD*ocKd2V&qL=d9T3nB!te+bU!Yv=Ja;m10z{xyv4w73+6P?6un4x`7r(PFV7nM8ts z5h(3w(4&V5LVvK$j+$qufkro{@ioRj_L^D?6~61~xhX|SS&PG^ zHm`#qx~;HGA*5Gg$roH%US3{B#s1JaA$0E;`e1^Nj*)SSm!ThN0#3AGyGiF`a%a@- za7x?K>;n7jtBzLD8Y-9{dtrScmRb)a7(Ct#60EO($p9)3v~eytDpJp^H@-GJ9s>3t zesH%2PpE-C7;=K=g{;rl-cUdE(cm<=^ErOBurbQnQo~_m>iBv{=Q#FURxL1;c&OmO zYdQgwLSTGQ<9Q?)GM?3V&I)jQL@cgBhy9CSFI+gdo1UQDz~LriTg=c|Fn(+NDv}a+ z^QiLXwtW=Tt4cw&TrqM7dNI5~V5q)z!O!oItB49=AG#K}WuBSpMjs5hbRV zuq7DNub~F9&(cVWrhvZ+tq4Av`TZ9ZeqrEea7;DC*H98DxZGSM8r_0DIJz42T{jBeuxU4zXt;}<#n2CmaHsV&cxvgUvEc124YNtd2+w3Z)V$?tQf+6 zi-ZTJ)t8V>ovmxOd2W~zebh3ZTdp>qwQasHT)gk9dI{0QCde14vHUk9@SIO$a6f2v zH+&Ck}Hx&T`psyPH?l^*Cz)wA(&i zxn^Fc6~OfbRIPRzn(x1UxW6<>PkhN2gqZ}?zS($4l$DfFBciY|Qz~@2w#Nr75J&@& zGuVGtE=yuz2R68Z6g1n8f)4MKF@rI#;v{hx*jYatvYUSsU`gZ`?mhpc>$RS=$9?g- zD2G4lbkbzK(duSsZa31gQPmgn!`#Bc5TVcd)aPvEo$7U$=mQSw=O+YKlIsptou@J6 z34^DfEF?_aLQE~IF6SB=>Q|E&-s^L7*3?G11OxRH<2AO*Yo<_GVZ?X$$q=Ig{JlHvs2xbW=57hxNpyw#PvdHl;y9Nf(Fn z!7GK(ki-&cs|}VHp=kh{V$h*yN`^i z7}eZmtT>AMxG@o17`^X-?}y`_#*5s2F_KZ=Y{S&Ws-hNZ_>d?8qwed@BRrF3;zoz* zX7M~5#Xu0myi4`N-|YZ;t1}o4F`JcUeGhGIH+4LZupW=BkWWO`N2kW5LanXECNIZj z*$bMrk#MGgku&p>DK@%wze<3K6gC>6gA2NRz4O@$aBW|>9RH?iH~7NUq9$T!Xem0J zjp9f**+iE;0jhNiUC&YBGc%oeZJ(~5|AlUYr%|T6>on3hv5D~gWgaSZ`I>#1S+aY! zkZGX<5F|QzRYVl!`(+4U`~8ZAIt3j03G?1d8JI5tN~LaBQD$E-Vif{Ltp9Ud~x2> zxp9+HjUEXO3Tim%yxrNz4{d*Idk|I3RGD7qZoF7;2bAplvff6l?wt4@-<#ggQo5SI z@4*LEJnvdWi&4r&ULZmmAh0F^c2!jh3lhTyZesR_v=n;6G92xljg4pb|HEx1OUKB zkqLC&&O|Rh?s2|nWXw{R>a;rk>N+WS_+P%mXiDeq^SKC=Xf zR^`@a1#KloidHqab5>(LdHzpo^b#W3a8}_kv3#q^$)nPKhn2o zhH8Ae2aOvclvDB!WoI26MtOV!82DQFg>7%+696lhGs)^H?{JxOGMFG{i0^SLfav3~ z63{<;2b@fjGnd}|s6OT8<`OHQ0$c}wTvGZL!j`Rki0?^}IW4YMUS|u+!;?4gwrhYWl#})$YCxcm|$r z$2I4P{mRvwO>R{j`qA#WEfqN$91OeozL#VyiSwXsK2WGFF}8?I+KajEAVy58Xl&7_ z_O(tBgn)opFl{Dj8bVowu=c-LfE`vP5c9`_=Lhs?me^3fceHRd5`v!Z7hw@NetmuY zdSZaIRJvruh^>J1T>*8y(`oi;b|{AJ^VOEl|A(VtkQe9IdCxuZaA(_G^9bv)sNLb( zXwh$V@wZI7T0MbKDzNhWq_tvhD?cc^@T-A5z(|Wo#7B^8)2#T~jaqqexud)CfW=~#= z8JUx9Oof_Vn|{JKAqg84$6^)$vZ5@S+Vn{4kNes6|A#T>-RH#!to^J|_SLmI`%A9Vz#;q#EStv)O&w(I!>) zw(%$2m@NWlqRV;UksZIU&+7}lbf40XZ>-lYHtllC4Zpg`8tulC>2PGTEjf53_Jf3l zvzz4M1pOog#fbx{Lm@o!jiuTuUw4Cf&qR_=sAcGg)1>^j!-5O)292=35%GHN{iAJY z^1X{YO#Qhwmi*-XI{9a~My^MwOW?m`yAxeXgAF#CHY+RZ2sBg8JW%xWVB~r-lT%Sn zuEt;hp-Z$VF%%G+%i?q}sn})u!0+BP)j6wO9FZFal5`R?rV(G7(EWHesHD3tnzC@Bz}k~-;4#UxqF`;@n~j5)%Xeb3#)9u@0nrkAM6hc8kP0J;5m zr?Lqasv_FECi=J{0{_hC-F`LNK=g5YA($8I^ZN1L^&Zy^aMs@UW&`}?8X3DLYAcxX zK|zudW8q{ns$TSgM&M<2JQ2M>7Y|Nh>)@yb-Py(gh$>SXJp^>O# zDM{GD5J{4flNX}3%MXS7@<{Gk0#L`AD@=_kgoA>DUI3Bc0UX_1YAKL@KLTPz*fWSc z0Tr1vzsDEuv)iNCFNFNu`TvVVax+J|Aoj3+*0M13c1>qf)w%3?wYI!*j$*>1UyRLx zv`A9%k3n*FVEgs_Ndp}h0&*ud#i;)FcD_m-2O+545fD8gCPnbJHoO;$iY)#=6q3rj z>k^e`GFG5~rej1n$&RK(Lz@Bhb1GKGG6RpN!oApg*Z=8e*S1fiHAJH>fE=5m!tZg| zY~?1?tho?`1zH2xBYf}?a7;A8>8dSz4c|Z5F%dX3qeq87?z=vSb^M!pX^;QUDN@8N|Sf!1i;D6;ger#}}3$ zB{6lhg(Jmb)LKz^=`+3W!bLtRkHSGhz%ZwQ4+2kA9e+80&^c~Tc}pPUj{rx zG8GgQ)Vf}q^1a~EG+3VYfx^gpN({|~Rlo~gXnbx8`00KdBLK8lR_ zMmUynVF(xUad}P+goJ3XK}O0~fP{!dsi7^yPHuGn<9+Y{=Cs5h(hTKCnwgl}5c)#c zsf!CZ?f1^+Wa{yy(xD;7Ilpo%^!7_#&K%AC|8iQmaD#}{lT*cpX=rIQe*fAT&XWc8 z$Hr&g_921yU)AmH_2oE_jKblOkSbhCNajnH;bgz>1)@zPXdzA%7LU>$biHO>IIcJU zxqLDe$MVwuVCha3CWMFF$#B9&6-!`Wws3%5SaiAg-t~m+dnZd@tzCQhNA!Zp>{~XW z)Uocz^CGDJy>y=-nep*-_aRD;db=pAm!Qcy*a&m5-++l<78e&Sh%^o$o9edC**@># zFey8qjm~#@S$!NgSWIfS@ZKz%{I}Fp3FUoJr@OoP-Q1pQz4$gt_S9R2ih`?3i*UGlId%1{E_ZGx!{NrgO;nQy2Atn$rx zAr0@3__1#q@mon3A9~>$;pSfjfF5TWN#`>p81u;X><6F6#T(&}-M)v<)KNB`kuPVc zKw-=_`qB@Gwu7Y-L*AA#$9`(7K`7b|Ux_Z=s>nbZn?X$C0ysion3BAm~T>e+^H(cf*a4k+&+faA4dOZQa z6J8Ax&6s5)1l}kN?DqDdNTGCD8OxM=RgoIaK!huoT9}9a+aIM^vK-V$e5pi46H*Sx z8&{(vPp5#p33?R3PaPg6UO}BLMF2vSA9pHLoL8LfI~%JOQ1x1kRo#^XyzZaG!_`!& zm|(Db^xm#`{!)^LZ?>8$DzV)$p!_lH$uFTzMYHC6ojbOle`_Q12C_NY8~FctzT|;N zcamQGBCJ-_1*i@d+z@JbKdm?^==&Gw7h@!`(DKB{5_03m48gvQc7j?%UB}rJmrQ(g zNt5DQ91a(BTcj$6)AKn-$#=i6%wTGA*yvF5F3m4z&zbR~1^Lt!@JiPLZkW(%AX8Nj ziF|P3&;IY(go3d*?W63cHuaf7dN3(68R zF3`gS&}bOwvebm)@TCf8!aQ-~n2w>EiDxUzU<59&Hh5$B-XGrqIuP;j_b*%nYhkZ0 z(vUkZ-hh{HHm?VISpVNsw&BBT)zc00fNtDrdz3B44e(_-V-P=51~_4UloZYG=iLytv&=eDBNs zQJ9yLZn8-bY-fu3iWhdM07%%X)$P6bJk;qD@jAB`Rcg~PyPTgI>AvobB!IP5kogI! z_n@@kuH+?*)$Ho`GF@ps5BCUJq)a|-%lbH-Mqtoaei_|ZYjMEv>L`)P;&R?uGmN>A zpbSVpv5Ki!ZFb#W`}A=Ei+81zL@Zd*uPzH~yAkUqa2p2Q0)Uy^cyPq^Aq_x}*})nQR? z?bk!&P(w+>kWxxZcMTyuAkr-$T|;+@G)gxjEg%9?0@B^m4bt89J)Ylp&dW6yf6X<| z-ut=t-gm5fE%)xBsgD=0HwG|EgOpfHv_+cAL>Eb}I(dNT)I1}?aR{A(COJdoa8#bg z&e}#9n?aJx-0p>%wxG*Og2jtB4VL?}o-URxr&Duu8W5;-gflrUU8tEh2p9h%uk<5_ zwZjh~&J?06R0RffmCRdb`F(f%mc`$YNW=olcGpmwGPj`rXkKcK<>mQV!e8}=7y9)h z9G#t2Ul?9TAX&h3M`#$uLtRWAmrpw1*Xr5wsO;!rd}KG!($`vN!9=1pmXSa{2@z4p z>x0ID6FE`WSwt{Y!}OB`jcQ9mDmZd4qM-M_qN1V;dLDa<7!|_X<+?Q`-@Y|fd1_cT zhXFS2^vJD!rrh9px`~MrH~I;Z+t?`VZ{Bosn!LWmkk|`UP#jIksu!Cyo%ZW($>V_Y zyl&Bpjbo;M114I4jxp8>XhL{Q*z45*9cwY%5w31-nPN^uW*7NYu0o)nBLJ^NtQcZR zQ~e5?q#)6+K(QItRBAgc@16bIu-RSH4VtEioz*zM+hY|`L=ZrA_x&P0n?~SnjIefk zXP1M|jau-`lWC@=*0TRIl|_Pz(e+MyA;+-bN%wVi^=?BWpFkIXMwxGTNr;mxYFXS0 z*N5j{TTPV7aHRj3`zwiROlkS60+-%eY@>(0QKM6TOn2Xe#cC#1k%~JZK9P*;&aZ9F zXDVTyh9~~cYgAU4wk564PB~g{HCt3*Ba=>RXT_QUlp;Rt8*jJ1&lY8|@9w?zeXM-E z9INfZCgEb~uvivEpl%flnReqBYJEY1A4_53BfU4C8#3T2wLI$*7W3Mw+C>fYBYi85MIfpNP-Rr-vnHw-HlA$&yE7m2HGBn2HcTpj z&}^b06^oy1)D$@&;Sf?<#%jdtJ(ZOhPOMTLrFS@x>eAU+IGJ8|FiT7Y$khnXd7-1{ z7sAuO@1^kBeuRy9=5GYYij%|bNR-Hx<$ZC)L^AfNY|nbeNUiD#Wtp;uOaBaQby)|u@@-x6Gc!%$ZJnV zh+gld-Is+Q&lC6N{EyU>yf1q>Vtg)B_jgNdm*zZN0rLIEI_*k>RzWoe3~!DJH(?n} zDTsUvJq<(+git$SaJw2oa*yPj6t3GV+56#?`Aag1h?)qNA?|WAww+&~UZ~bci!H-@ zys=IANreVC7Vm@q!*o{C6O)1zHTtfS1Ztnh1^(>C=G{;of70_j{*>eu`uLsm^HPaEnN- z*Ru+H0#A5I93FEQ1N#f!`5yAL1j3+p3QG?~_H}&yfkIG$Ad^g4A0{j`w?HkAw^M>V znJ54|DhV>lAW2DwbPGnFnEt8R&8p}a5l!f~#0y<=XPWYWKv3Jp3g34NvAn{;@%3#6 zsBScEoW54pm6h$e=hKpRx&CCmzrSbPAWm#HF7t#DDdK~o0~BBeDfwD$;79id)FfWp z*GORyO6xGK6s;bQh$kudh#j9-r6p@A^uzJ0tm>~I>R?$=UK%ta2PW|=PqBn+_|)&> zl%~sJ5pmdz_nOicBmr?-M>`@VpF~9>V?|%Kje;UC#XmEKpDT#qyyk9?=k3(uD|y0% zV@sNVL`QlO^rYMN3=c3kz{5E9a=99%O;}l--N8|{e{&N4cvW{mghqsxv&;#_?C`{t z!isTxM+~?|*f5xG1W!aD11rlHEcwq~N`v1ACm+vm{S_D!$q-1_4YR*A<&&^{!A8~T z%t+zadq{EtWwXjK#<1_IQUIsjZ?x?7cpNdjx{%9v; zk0WEwqKPUop7#miPXT2K>t81h$jR-0$lTAPglTc9oSqrR$2ELAz2nhk{6 z-oTHHE6>F11C<~K(|{TAIXfhCf8~DfF?-YK_y{ZbDJ!MRB{Deq1YJH7lNE}ph-xYjheg6^LPEse zgI=g@ItYk4!o_p`02!(%Gv42GRFO~ZN_Df9ne_B#PijP1K)stCgR%0sgJF|s+YvR$ zrrOM;AtSHC_*%7{f<|(~B4E`k!;~O$op}Dqdtx?+jq;}Cx)be(bZx)iwSX8LD9lzo zB∓LJ4b1oLgaYD5!x^LV6mt_*hX4uj#sweYm7ZupLEbiQRZu%+ZbU9E7V$OxmO7nrF;Olt?_5+JgWN`LW7Z zYVoT9hMP>%DUXyp15sbrCa={ z_~^S~a1qpIfb$8;077V~eAFkTchm74?Eh8ux@$nqj5OtJ!Pbzot1gknyN!szi_rWW zS6cpL_GT_yxa%vTsTSSDwU>)Y2S?f)y#zKdYUjl6Km?HCADU)paPO2U_mY~s4tn?ouVf+GqH~PV z z!arH&7r9)r8uqF7&-LV6Lgqi+J+&{j!mz|}&h34|W}>6-n?&i16(7t3XTl)pn!UO_ ztbmTwb_PlS=Z|OAhe!eDzr6=LzeCY@NzD4yRkBv%d*lr%x)4QA`jj2*-WqBc_X`eG z)O__>fql?q^7i>1N1TKtE&mbEO|7isR!n2_?s&d_YxV@Zo0QukLFP%5F{1DmW=Rk| zF7NGdxZm7dcX{#9;xFFjljbdsx|F8^0#yZt$d^Sk)#c|X0}#GqEt{*tx|h8fxHvt& zZ9Pu>z?-Hxtn=d&n1G)OK#{-n55{{eR zV$G3jCOk1u!LHlO2mEIfQfMGIOzZ7N)MTAbI7FjZ%T{UkYt*xqwHP#Bhusd)f&7Xr9GBFpg z;hvNalmFw^TZOos$?%*Lj2S#9g?_^o8fnp>HQPgIOBGzKW_H?NJV{jo4QkBrFzZdh5jgtT28|<#)y3S^M0k z%!-f>jjq{kDW|f8a;KnE#C`RaZsZTurZAg%AH#0c^SVkZY*F*YvT*DfMvEJpkVzvj zu&zul-QDQogv?hGf>s|*UWJ6q4M%DAI&kZz%1?OCv1o;<8y9kyajY212a>41YdH(8;I5&Ht?S-18?AmO9w80jFUcBr(^0oXY%F?jh#xF;EM*@ZUYwgiCK9sp# z$(|W4vG&uz7T}ET&EJWT;N@PtBVtQ^BlI1^uCWX%CcI=J|Mu@nqz}|tV=T76r1-mN z$Ahcx{3xGG+DT{jj$Hlf%*o$L$i0C_uM~2#_s#y+CN*Kd$tdjl&Dgz=O}_T4$M#fm z^ewBC!at93vfT}yOTTfxy^E?UBmHa~BRFegtd%h_>a6n4mU9tKNJA6LWk;x$43JLB z@l?Fa_u`i`{I0-m@!7_Tt7$~$oTM)?R#m9vA-A?IVxpz4&HM#x(w54B9x%)fgc<~d z=6U>?Xn~$&<8H{uL?* zMd-`L5BPY8{+eXR)%W(bCYp3E9S#ph>K%sE7WtX4r6naLBkyE9a?6UP9+vD}#r%6w zr@>C3r>1y;MM$0x7gWM^6Yoy=20JJQ`&oWTDMO?rAhho5q4FZA9;Sx5dmDqi;6`!F z75o#<;&AB1A?Ak#iPw_W}+-9(PaAid{CE6+nH`IzuH=>+ddkOKO{3ADmaP?miE z4(T8BaX=YwO64%gNXW>Px-w?L?(RYC2$W*gqlKy#gpx#mO$LF!veIHF(6D&kZ?*bt zbZ+^)D&FGoT_5%2>GvHRiHq+XdCTk_lYdIWt}o#5xsoi58J5Q={&3A<tQ9#Pw z7JQ7Vdmr*Uu%jgr%xf$#^{ogu?zlqZA}MU0)@_adRm?r0B=7>r3q+JtT(HKKy%S)4?FXrxji0P-j=sr#Oiq4(YsWC9Lh??omOSGKHxGs9fb<) zSuer_leR)hjO-hH?j-aG16e5kgkb%Wy9#CX!Ld{3&38mgCW?-wb8sV|*16E)OCn?S zBw@rrW|4-b)zFwIkIwC+5BFEL5VVjjRMsu|NT1Iong6{A|M{#*nhWtlLt~SW!2)k6 zKipkm40F8UyxrS&2^Vuv=cqPrYl1Q4g#_qImK|w{2*8P>gieU5LA1wOq`1bP`Z1-| z@Q^L!_acZ}h9O{vZ}UI(DELbA@J=)!auev1_Z+NCfC{IU%WZKD ziILaC_Ewf>b$MYiSC(O;p9`}pD;MJ8EEVNeyuY`@({u59d7QqeMZeBu-$hka`whXX zXM|QN*&;{RlK3QXu5T}i|HT3jj3zKDJ=}^v#E$8nuJL=v00RPYp~~tQhJYNRrm84h z_e}=*o}mBE)fNUWTDzLV!Vh@t;jCbO345k8rP$ZU-;s$isiLjlEnNmTKM-4N7*N?| z@6V0@{>{!iIHY;`v5907sB-Y#?3GqFahH$%-#P#H|7Do*#gT;HA2b8r~d<{cl+sAb8}aXB3S zee%rQr2EeNCzVhc;GN<3jq{K~)NOaqFBiQz;$Z`NP3w^h!qUBPWRTd}?|8HQ`)bpT z9TLO4n0|&A`u3)>dba(yhppMLf{*MOw|fUW73uu3nYL+yE}KfjdqN%;S93;_B*dQz z(9tP9t)uoATYN7+%U}9mSnGLwwp2R5H1gcOY`wii_%Hv2s>6f$Me9afb8h-dD-|V^ z5fb9TqMI`tVC5EkYpVZNkXufiXRK{X7o;IEx|-5zar)!*SuFK@ zt5MTx|MVI$kLARa7XK7K2onErkUmYZ(|G^zTH%^acsfhmx0)W`&feOf(e$A`{Aot3 z4+_Y7#GA)&e>NEu-~XXp{C?DSu5N;t|0ND`-menWZZt(Q#$Rj4SqR}Iin+CZ8p9um zwx&}n)gmY;A+in8#M8N<@B&}_GiumZo3eUy&GXHgXR(wEkGk!ZZ{N0v)$DW*>eU(8 z-i@%l4$8_(KR-W@xf^PEFuGqYdcQf`Fy_KzTQk#J_x39-jySnoSBH~4&+xdIx5k5xhTYf#8;^3m#P-9q zm3H&D+#j2(le?<|f5ot!;+AG+zum0NV9}(hZlwVQNRcH68yg#2+h79eK$yByZ`WXY zES1mWT`FO){>*}l%Xy`tH=?)wDzxh>mJIL4w;xN!C%VqgxrQArPa;e8CK2k_IVJik~lti8Sne6XwP{3}*Ygq+CD z-k5X0TQLdH#AzQXkjWUx$t||)&aa8eGmND;@p_c?pO-rVj=Z%~QRncw@x_ij_jW-M@)He@`*$5#(e)X08lApAuzGq zweH*Wd2-syOe!f?pId7N{Oco7a~mTEizk#C+%2ebaK= zVN_9%co>K@dp3f!Ds=OpvMnAR-<7LQOFIMucL|K z!w+Oo4my>P6X*KRFL}vXP7Z-Ho)82F_nJEo&3pD(k|yu5U`ke7YceZq(uI-zdh{^Pn?!od4s zG28$+cj_61LrblB@9OOHbpMM}qbw00$HRx z=`pnBdNaL4Qw9r^yO)bmHvNy2{@2EqSUs8yS(my;r7-|~jIq)^ih*Ad$vMkVdO(a6bCLz^1`EEw zO+~5v`6-9zmxn&az(dc)4y>(~Jm$@IkU*GuykS25+Wl}0Xm@99N- zeN(k;rlLes(s{KP=2{~xX`NeIr|99Ya{uPgqnd)eK0KA6NF>E;WZLM}bR;s+evI2zUtgUTHM&<=i6YqVW4QGh?-RuGMl!^Bhh2{J4Zo8je zq};X{i48HMKnHDJU@FSwshLV3p$QaGn^^c=t!z;61H6W39C{6wGtyX>Eytd6J|I+l zIyA{3W!9>wZt>YzSra5xBYs&3HQK*JxSj**9SSRp3>Du! z!nv7UoE?`ZiZz#JTI@LEOB{BAtNVO46l?Kjjv^4#GegM-OXJIrJ@`>P{4=IjKBL9^ zk1tfc3k!KRE9SoXwPlnv(=_|$pITPw6qe}K9SV3~ylZ_*(lv5dI|}o>EOsW(yM;@2K2A-InknrER^uIZc znYFW>0xC*Xn((zuDl3&tlg{}3qKvD#I(zS?Zw00K`yF);UWErDQz<>+a?lfa?_!9!xjuY^X$0TmQ90NgVq2(vtxR2$ z$91;+>b_e+Wl?xM6jQh4^>|Wi2}PHPKr8$83!o-<1~0`I$pj$M6bcF8r`iukTdlYm z7w6}I6zab3<%fzG75Pqn;WcmZ+0)+_&u#hbz?6)HzFU>h$?EzH}3JosA1q5Vdp=m(9xRO&z7eb&Yu*Fn$IS6 zNQP_!IQ@EqO+d&XNGuEvVFlRkX-)WF-TyBrFp^dkyFEFp+mQgK>WED%pdFV$?;gP% zxGaJrB0vY8@K|TN9f7j4fUH@O|EYqRf&-8PWX%Q;_s=TbFI|0)9;h`Cjvz|97$4f$ z<-7za^i`)?6cp`km?^kVF>e(gS|Wq1E-#}dH|v3vGNx4)@}&TiV5%4zcay+GtPwCn zoAyDYM9nm?azHJzfd~o7GnXOYY84X+<@!r$yB~nBOQeERe>A%?grboh(Z@&>W7jZ*XcMLOyhJooZ@uYMkDSc>hZ~s^? z7n>0WpkgTGDFw&sHl=(UAI@cY;A1b{@LAq&cczN01EbPwy>~} z+O#+GaWP4Lcz8%SiK^6t{Qsc4p6atWfz8!bIkIehqly{=8L29WjQrpyTa{**R2#z! zibMkHpQvau%e2Zl$m`e=0ux0cHxch}0)hl7@Zcf&C1A`5suZF!wd9e{NCD%yu53hx zuss2A0Gxp>m}H9Omzw%jj?c){SK6n{EIcx(0RkssO2{BZcIiRMWKz0cX@fQGijH9O zjuLPL627AZK1i%hQdc%vqD4X?AShQ+iXcY99nt<+&5fewo@+^8rW-$Q4?2>Pk9+qi zG8fqbsT~uFmvD@+QN8!oG&-q{0fhNE`G4-|;q`3w3N8zBl4Ra5HEp=1#b8$!I}E0z zCV$jk_HCt>e21M^=ZOIJ^E4%c{!qU4N@pmY+!xedVrU-ch&96cMH#C_DEpUi;?Ptl zm*yT~Xm7uCM}M>%u^fzaRg#a!4XzvBzm_9|%>)m%`3-c9w)viaM?*L;t%+yHEzc*+_(mM}-@5=DLGGZ<~x0SHzmU=<#Bd zhiS1RdT1WbNF-b{DniLaSG@0~xrzD&lk4%&W;28>T^mDB5YV^3gYnU^=y{f(8VXZ~ zf>WL-s_6FygDl_%Ms<|`drtGJZ4FWb{oB2w`-~UC#{qBH{yp6O2Grj3E;z8aBAg%j zWCJ&3U%u)74f}Jgf#|u|`LNw~!2ADA1t>G1dH>}3@3O*PYun123&7s6AK@0-yWDck->6dWPp|Ls>Gz1nO#OP3pyf2D>FFWT4WzY$39Bq+RoAH7&b zGkjy#RMS-#-Y(oGDGi4`@ZDK?MEtuzT_``#W&%QDPp_WlQZ3j@Mt^Bn63PqmRT!I(KozBo>-&Z)zyHCfIQhoCq6jS zLIO;HEbTjZl|~x@!HOpc4q5RTyxI|ggl+_pQ3I!HnVuL{VaQ$zb^jI&3UQE;i6z^HC-~Pi1X1YM@%;E zPsPQLY3BgOiM5|i$l)9H?Bn)ishb4RK&Go|NXv4nye!HVB{)7LLUJu#pE-$)M}KI@_+@CmZ8vTpyF@y|bA{aBtpGU;&&Z~50KM8jl+9@ zV>NM0LSfoj*xS|;&`i|%ecImBWa}nB9B-=byy8vm&g5b1-KNo_dqYKH1oaz%#zV{Q z?n7TmY8{8UoA&<7yY0HisikR*D1niMK|Zs2qv)>(h>c{OYD;-A$f5aM`z?l1VL8R!?;aj-Ht{l3OriUNw7_BvbIi&`A%FI`qt^wD5AbPG&t#R6^ zGHi#&#L()WBPiZVhV?g#!ER2k`H7sZ9t72%RfTHY9wiT)0Q@PlI$ca`yX|U=Hizeq9>xtc#|dkr5{sDjX?@jDpiIG5*& z!3qzJXVb)RVASbW|Cl{#TS+oo>kl7%FX3Fl5H&cQ4=6EiZd$IJk}F#_a}AP1Xip-( zFh6zbMK~-CjvGw@ZTq{D`ex+do>S6SJ+_+H6=tbHhe{UyV|Slos^Yz@>9%VW zE%Zruz5_I|52oNWFpdow#AwY}bmDAD#e|}6a+>4F9bnH{el}Srvm58)nn_rj)yE$6 zAx9s^?N@7y7Kka7L%4o}f~8(|j$cKWxRl(dsldL~*Vk86&=Lyt;f`dYp8l8){24Ya z?xy#{;l$&gqpXXH9xOq&JePjZ*yz1Am_^uvM>EcJsxQ9q5()yQPSAUXTe3r9^`fPv z_#*Y0`$w7cz0gynCRaJLsyxT!GL?(HIZdPKyrYmL;>f@@*Er9$at{&?-Rk_$nD5I$i_6<8-dUEI$geR30SOXM)Az-S-V9^ua43#OzVq4<5j zMtPp{NK@g@EX`wzs6kLMgKezsyVt32PayFT^Ih73)*t@w-DU4dZK}8n+nf115A#96 z@-nQ_4KGs!wTZU-3y^+!-&7csFID+<9(RO$VPGUT10!bSQe7}U95iPIxN=nPt^}Fc z;sc1WqY&i0{JalURp*hJbJRwyH@`^a*F^7wdnfWbv34sAHLkL@p6hYearn_^DiGs- zDIxedpa?}={y6ONMP)JcJ+H;bu-mXF%_lRn@@snL_#FE-_j(#1uao}si@uM>gVxl~ zvgX1v+2YJa(t}4&NZ!P+$Hd})TBne5mq-ftavXN0nGs+Yu~zDxzuMw?A5PCgPF}Rh z+6!hUjQwifN_floe;9(O_9L3Q7y3sJ6{KN{fJ5-RuBqlpF@%BOIOyk-6r%1}Dz-k? zOc76w2r6bWREyEKA&guj{x^G}RJD2x!Cg1#7tODodS|e7`uqAma(Mrdvzm7r&6sf+ zmt4+JVk9h5&AJ)7m<1MRdbS3T@#@4PX*%xu8#QJHpaBH}2>jXd&-wlj$n7$micl&q zoa1-bcBGf2vM-l8OJRXq?>$-kn%7&Siv93q7{EhhBY|>k*6cU+f6dHC$z&6s>F0RE zdb$`#iq9vy0@;>Nu=nk!6P?1dOZQ(Rk!WconAYgxQ{)*&g2P*21wnR`6hMPtu<~lD z(sWA`qWnw2H%r_73|$_WQ-1lz0^a3W;j*UVU5^O|Prcpl&}_F7fW>rZJwK=|9hAUU z3jll#%df1`lujURz1I{V&=6x|@(2slWz@Vo*UXCo6C%#B*G?i7A-yw9%&?!>#}oJ= z$!JNnYnc9p@-r#XB)4YV&tt6axtfw4^3LTk@fX|<6S%rGpA@V(xOUg*=Zyd@KA8ISBgthwJg{&O$&;s#t{+VAYmfWze;2AF_hIF=a_-w8*dSp&(&YYcVhW` zGDmH2q-|&w8N-)nc1TUuL1o6l66R(glZ+$|EeXUwC(=F|jGb4I8Sp$LLaw4;ipL9> zz%Ps(3kiUhahrIAFc&F|#+wx#iE?OYHgD&Ko<14at3;SzXUy;j+?I32>%47EyU)}L zN+nG z4c}jvWKBgaj+wVnjh@Qi#b#iuIFiI5Q8(d3#=Q$$SL8Lhb6d#naxbaT`kn*ra?lw4 z#ru<|4Gs5n8$pzT9qejhbwius?|{%kr)P6fL!0Ab^>lcDzUk#^gKO9M#UGMLD^Bq-e!Q*Bzr2+OK@aQ}L~om*1X2#N4c&>8MhHR$Ro%167aSQ%Fu9bp3M{r`Mz_?wUDiEp&M+%e&%-m!|bZIMxk1 zMg{*QmFSVDo~?JKp-)*!T{GucEn{;f71Tc z@i2nTZdgWQ%k>_ncOT~Y=*S-9iYngt-SoX#9n&k z4{l(sqcLKE{$OI2HXu)pL5O1ZF+-yuwu$!W>;A`}t74C8satYhs3zlps}kmPvGlrK zX?KJD`teBE%Pm=U%>7s3xzsN&fj4GGhY9jVx|Lm#d^$pb~AY*i1oY5g_PN8$B@%nT7!HB~>)X z+({HrXeALoXCj1>x1*rUaXmOck4Q+JdkmDLE}un@zFd5&g$X5QH2DfyD$H5)pwWg> zh=*Dxk$8A4UWwEANbJFf4C`yYZx)H)w6#877|mh8dSany5)gG{03ysNEQVByESZ8x zX+zdFMG7aU!yD2(=G9U{0!5&zNi3I^$nRKx1{kvGz$OuImudH{)dU>amLupHEr8#{ z`#8)Y?!9vrs5*BKdv~xG1yiqL4lHOA7@J)Dv6UYi@wRI~kb=MB_FTN|{^%Z0C0nSn z4^8xg)=(DB#hFLz{imTo2}tIHL)+izsAc1e&6FlQ;uj`%d$!g;Ugi)Nhh1Z+lwgEnYtbuvsC$zB?LwJZ^o+u1@`1a>b>&S|4aQ4SCH#em_|0 z?|gltlkMlstX7`FcT3}cf0^BK|LHvm2dA0KtMef{J*%NF;Gna?7>aD4Ge2pw8bti7 z$#joN`*cA7l&|3C;U3bx(rHs?xAdmDsw8h+PH(rR*?X6wLO$aA;cTt-Y1j<$26_Fk z`1RR*1CRLq^)Ar0wd8plEO8j?!&~f{*IdY#?^-LkxHNqqTPe!tA_uoxepqR}m)_Os z7QZ$(Lf02G>w3I9UuwR6@ZPHUCVKr#bF!4Cn3)6JlV507?Di_y_8bS7E2b-JURO`E zxq(-EYH?8ujxS4U$5z^H43b*E)^Bv1`?&Lpr%%b@o6%C8hpKOfo9fK_u?mA&hw?>I zHr5uu{6?Vq@%$NWAhFE4UFAYG&>7o$f4l^EOHY_TFK6tqA|t~zdLMQ_-~^5&|Jc;D z8uB@892UKw_n57yb8~kmU`STn@R}7InY&M?5fN-GX>x)38Tq{Wn^!mvplwJwmq9HA z^n!`mwAYz7^rZyy`aiX_+gZQ(oW$>u;A00k=13{To(@T)$4oVUV7syUI1-X2F+tpd zmroOpW=A7MoTq3)-DxcEO+ch%jnkabEb)BWTD^kL6yxHh!(;3WuXj3Zrwj$JHV!g{ zJ#@sYA8VSQ8>HIR>KWcXTmlRSk36Xm5dIi_INrjaBH~hQ|3mMkesk`vmUpxkK@duq zjIaD&4}B=!LbdO1#WNalw?B@)9p}HJbvZU@={p5+?c(0#L#H^xTs0h!LNyySi>?*C1c`Wvk1@ zItfb3=(xAbxxcqvZ-VS|Be!@(>4nMhA{AVX$;q(Kx;&=ow%>g`s|E?d@tT95u%yw3j7(zA+XXlTVMor(AbFcaJGy#Gi z0XNsfQIu?KQ#oI=)0q2(|E9No=n{X^S(OHj?sr$mQe5_pSmU3O$cQb>n8oe24P4oL zdwR&j&@O1SxpkPPWgggL-*xPXMXRTrhnpzl<7_>nTla(WmDcn;bZufws{wRg8rnB| zSRA5{cc)05AR=|34x7M?3;iU@23wRJ(x1rgFJO>jl&hl)TVN3i&ySF7Tx-) z63BL^#BNsEDh2AtNp^HQomc-NcDtJCc^iAkjIhzRVYt{rTiRu>x!P*WUFG=} zBS1f?S=I0MP^?H8?ep^Iw|G7kDbTqAqN`Z|qw4+nZ|ri(LeG1wxWeI+Ql-So%GR%> zxMn~F${5EJe>n15*uXXgi4=IDkE^h?V_5)zd0Y~CH99p^lduW>w3M&1@$uGbqOl6y z?oV-KE_|;{qz!nS(eZm*WYjo6m9YK{hzfvhi+ScT!Gu!({Q1-Kg~auu*1+M-%dJL0 z@lsUX1DJjPwqh7gWXPy!3aSbv(K=E^&QR`S=rR%*ckLIl@{rV@}kI6 z2aZ|A|1cW@jg&Fv+_oInlX9I6FOh=L#{)2pBn5?U%kRyQky3OG_0=h@V?08$#I{|l z^YiH;;4hWuUTb55OL~p&``cLO9%j)0%PoAu&s9F9paA4dT&xDOuHMiB&4}9t^aJmE zT6}K}eU3YIYb@Q|+<>&`;fpS$)bAUa(VX>h`ZRLCE!K$?xy?^SRB(8GP$OZf0Ahxo zHy&*$gR|#{K zc6@p+(~a-{Fg0;wG0YnrV`5`bWe$w1T3)*o9;D5;;G8d4Spn5 z_Yw4duEdcJ42bL*H~w10MpfUJTC)nQaHcLmX5UoT${~yOv7A8zFNQWCEbAu-*16^? zlqup-YjK=U@pRaTG#Q%~3WQg>;wIp3LtV_6yNGVp{oiT@!A1rxA0@TIg@!;Wf&}Li z3My_Ee{Hx+)Est?6bhD11e&a$v$0uGUFc^UfkJ>*Q^H6|<+mtY-z=fXfE+N*;llXr zR5g%AhGMpJ6NWyH^oo7=^?eETfIM-ZuMxVAiKQJ*8as{F=|v5de|DbAQ0Kd{UklWq zo#sMbl8+|zg?|*`?q*}l+X;2CIsiI5Gc%7vindg!{C~JBEhp$u*ncHqc^;W1 z<{t@1VyR54xmqE9p;7B}`Z;`a5Fp9Crzr}gZ$WI#@~Db8cwep!4i0w0UCh`~w<-RJ zUv6xq?Z*}+jLlC~9mVrm$f~Obw><>bkhB2Y83>`mSLOWeYthCzt(Ozqr?rjl^!Ye$ z=X;t83Y{}GMuMz&3RcXYJ|;f^bQbR%9DKL-@hIPZOYP+c2NE>pt)4t)Hw*j6ln$Jr zKZ!kv0=L4*8X#^RQ3_fSySjFRD;6x}@D8_VHkz8Vra zSLc`e^ntUB)a)Sa;^Lw=@fn^0b*m9m-uLg{tJ80>*o~AK{HFI>d(64BeNK7<&fiBq zv$){g;aVESjL6RK7aBWqy;7kJP6LrFH632ig*rslUWS!!auyK5j_a#5$qu=LS9s_n zdg{cP9*HUhc)(Rd;=z6wDDIHkH>RWEx$zNKbtwMJ_lMv7zik*&eZ)n?U6t?7S1MW! zlgw@#i8;13b1k_9=iNG$C>m#n#t)voN~vBZad>>;?A6>@v~m|J?ti7So6Htv&TKz@ zVW)m`6ScvnEyDLAyFWwN?JnIxMW)Bl{zv`ZD2D&>rb9;F29O0mZ1R54Po&G;0n}*q z4FF)^Mv0Z*QSiH$bBnXC&9`r~J^R0BlD#T>U1M(`+qrR3$|{)L1UuWZYaD|;H)?h_ zu-GX4u9D2oi_!B*_`=n~ZL+$sbh_+=vuKfO_QU3Hg)boNbK?HOqf`yBP;B{jTIjbV zDFPN2Ev<7KkDE4mrq=t5IY)J&O#2p}W4y=X&Y4pZf;XP)KU>o!dU%3_pH5C{BK#rC;l8P}}b2;g%)r%^gKYyQ8p0oqF?A&!M?oy`^z? zHWX`1?A~!H`QuC3FUpw<+YPbGez*H|3i^PLe7f;jyy-OCYREv1;knT;sabTxX~dW6 z>Mdiu>|M|AJzkti8KFzcYWrAXWPXMJ%pdw=c z9SE5z1R41ko+0`Wf5Yx;i-)`GXDZoboF6_4NJ&J2u2;8WATszuVB$c{lW~o^8F!*h zW;AHAH3*l55jT$UJRLRWj%DX@8}`mqV;ogReuf9*G_!6>i$_v|fTf=nA7jk1MTSpHXCeCsA^=W?*ks{fhhu{;_;iWa{RyQ$G;7i!NzObV$jzd; zmWrz{U-4KCm3=8AU|n~@zG?M1UhRCxv|mT{Eip89K-g_(A_wmn{~#zlCU=io&RyO` zVs(ZFC7k-bDR%uF<(MSXcTbT6PUy{Y5ysM(00xb%1ZHp~ks6c4#HCMk?1;_-J*~=H z)GtA%TwF}Zv{EZ26^{_+TBW1IXjGYzhu6J7C7o8jMMIYB1AljOZe4prq{)uFNxfU+ z!3%6~$cF|BaH{>WY(YVL0-D5#R8Qj-M(t7W`ZUYOry3#nw<_%Tgtz6dk-jd~LI`K5ECy^rwl%eXct4770^j`s9U;6zOwx zik!S^;;~8a#7^n|N7PrxMYVok4)rZf7u_pIbUD4C## zC@^zEY_VRecqE=AIhr;}fLyj8PVx_=ZOtY={j1-hUm>x+OKjlM2uXIH| zhk{}Q&`jRfX_Gaa@F?x=yyBr4t%TID+=~Q>LqOMzgx6dC!4AHX`ECY9%Swi}mo||+ zeaXji^QAcj1uWz>S|Lc-rdb^E?fIDrV5wjK^`FNqyl`qHiI+|3ri)i%aq> zu+f&Lw>%~mnpM$3Pwzi8LPui$fQQk=1LbXdvA6ZpP~63|gH-gwxqRNfYIA_~W)e^* z$>6|}*2D4R_O=eQr)5(gS5#wYvbBBhVn1RZEd^keq(SPVhtI0K)HENWW5}F{<6zQd z5(vDs500c3EmDstzP<`n76|_EQ$*bfEK}qlXNY&!mwSy|eI%WYVFCqX^A)I>RBwje z4936QRi+$kzddSM_7TP`-kmB05^nFto$|r$4zsbj1lN4Z`G`pG%DV_FNO&eT5r0+? z7n`=vxuSIYcBv$872h~2E+9sQz%ZNYFr4dvccxE=hg^2eJfVd#g z!bjZjLUh;mYuYsC6R;>zTK zU1mQC2%S`ChyFPmno>002lSwR?I1x#8Xe?mlKrMAr|cmD2F45$ThD|E5uGMdvT^?w|tYTP?U+c103Y2?^s#t zJzOTi#T8o$LI>(Mr51#qW!g)>fJL9j?PV+Lk(*E1qH!E@bjRY=@TJC3<@*YU68 z8ZexCNEg-cYS+NKd%u#fVHBHom7cTnUF44wdfyVyMahS^$C=6qmK@0RL$)RATiXYc zCM@l(=fuLs9-e_1VUZWZm9%^2!Pggf?wju~qmB@7md){O?vzR;{qlc(U;4X^rnX1m z2a_}hEInNHzF>v+s>4FoaXlZwN1yDnc{1CQ%4rjuEOF@1ts$-Sn%yB>3P%P_ zTAvy8MGZb4nvJPpq2Pbx9V{$-aez$DIH* zP$e$n`L^+0)F-YBq0_pjpP6TnljTo9_9&-CJ<~(gBoxHUpjiGlBW3M|p?oliu zVvAgH)jzh#K*5i7KQd_fL3s*t>6xuwyH|;Q$zF_Z-TnzX3Kr)POp4%H5*n(YP1Uei zBL+tC_EGcUm4e&vwW;)Lrf7R34DdB-l%j~%lP>7cn6#qm@_~IAsy&I zjF_YgyLs-{FGn|ix(jT8-EY?uOA0xk_N>dFWg0YmH1{!jvD{#_6D^J%4N~8BRV(-S zIHIY6{2e!TMJHT5Jd`9>W!+nSv+(Wm*E7H|4)BU^Wq2hwCU$$?6?E1=`-{7FS94>4 ztnjgq%S7!5Ekg-62b8~&sGG=uut}53o#BT8%Nh<24w3+TT@f!&5!l&6%|`!B-HvmI z$E_eBBeT+ON87ov*MEOBTj_JR%i2F$k6cgc82n_obT?3#e^r4Q5*s&ASDM7-vpz7W z4fT36b$uh(?Po#(=!wT&j?zI#GX#`bbzcaiwImWd#iIwsaCqJYS!EU%t6JQ6c1Sq6 z^k3RJeAx)6DAg?o@)CJBN^*#?4xj=I#glU~!1}P$Q#zpMg?29ZI)cdq=8NRJSHI*5)Sl(+XQ~c6mlgoPG<$E}uqD!!}BsecD7n z9l}{XA|=t=jL5G|rwS%#wF$0T>&;}%vV$Zeo4ReSyTK&@v_vkSZ?AAeCVoTa?xt3E zpB43YZ;e~e-J*q93mdNQs4V3!`{RTX&_J0I48NQH=yhiOMmzcp`U)3iI&3bSl|YQQFPpqD-c5Z07+fUa<5fQ7GS-)esDDSVP{*Yp2(@;0KHDBi_2S#b zwjA}j+^qwAQx)1J61Ntbh$WH__uOG;^| zL+=B>{W-l50N^e!r$w@b_rGajH;VoqnZwHul2%Io|te*S4DMZ0SxjyP2E9*+=3| zc@3=2+k+P#F~gaF^$0@CBu=NNGHYiv{q_omf<3EpU}!q8mu#C_p552HP@)4(! zc$}JV-Tp>y$Ys9AUERJeW$XRTfj-~Ds0+77edU#bf|p3qsAg~8;2t*tU<>fZz<{m} zFDTITUeSjY4V|If_Wv~m|G<8C_v4G&24~;r`oQZV>PYg z?o@3x4Yr>88T-<>$wgR`rKhBe{n=Y@9~D=N-risqO(sVT^7IautuXnRLec?4PO>8I zK<)UZ7FM!Aiem1Gh}WCrQ%TC&d02$(8XIhda)2eacOylA`7R00LH-(`b|zb?|DDfD#aH+COrBy(c5`wMB(f7v{`N5*zJ8v zbFEFrAf?ix%F>Dw#YgYo2L`wA`&SCR(KXQmoU)K1|3Ll1O8&K^^WXLy$4i+=6!Yfj zs@t{(=WBLMJT@r*mIhbDj`N2&NZ8XWqdYs9PuE_U6=_+33B%RCL~ffD@LY)^5>toG zT_-13>&rd1e#9((PjY$0`j&+3oCWDbR5 zD`qhvmin$x{60s5Hi=vwXXOpVNAt+}dLx8+8KZBZlH6y{q`h&y2eEEUy{_5=VJHz_ z3ogtNi>brILshcLUsFqSLN=#E`1D@qoe1%DKL%q($%m)vJwwy|*gAT8H##52OW{NM zP-tIc7@4t;o$Oh0w`jdj#>>5Y%3s9Z^h#yw!K8d-Uw-(rr1x5h zw{NM4_o_<7dpL)#zWg0Um)-hO>swvGq#P z&s(dz7uGU&H@uKACWf5THStMsrP<2Q$&rDkBxD)Qws=jYP&-?uiFzr4v81`(ZsIda z?dH8llU_VROvk6_Rmd4M%Ch8WGac+XGJRDo?2n-*W?6MhWP~%|kVxwX-BB9N-rn%e z&pTTJRjuJr5DQ1|M|3DQ{jnb@ak$NHBkn~Q*vswk;NV~w^TMdD3D2Bh;E{%muA9jV zdU0-_Q$!8xJ%=2KN#0)9ULrMJ&*aS2>J8CMG{i3g;;;RCEMpaenba?khL+37+E%aO z(^mCUplsNyh%4QZJh5Y%1Uz za}Et}8X7a|3!|&3<~157x0g+Bg(X2LNpMw8(2)aX_PAUdEHIL>I9Aue3f0{WK5!&eO zrLe#5fxjcU5c_F5eX1s@;<1GLAB9rg%lsNjgXV{vhC((s7^y2>lBC&w6lDgA6PA!G zlVv<<^1S>S=@O;zEMTeZi=-YNC|Pifk*qfb4lqEene^spy7=uLafV^rQA$lx^WL6y zH#aW$)|$cBD=f$R)6H;79IKkJSKZaP|7WfWqczOJu?Fmctdi=l@u7-X!%2+nGivgG z`sZEmOG?TB0ySWNBCZJ4(NPisV`8G=lH3Zs`m%c{?0a{;kYH0S+%!HZZZ&!^7)~xb z)h2F2FMPLjY5_pdoW|J2oUIc!g8l zde;W77==Bi3REB~K?oFnyfi4_R^&lna#fBbj``rT+tAOUAB!~*$(1PUzv1%jaFnH8 zyz7)#qgIV+I0Bu3P+)N}?!FM_D^CVN!2> zIHhJ)y&GK*-jwM59~WTTW_j5Rx;&XsYLd)Pl5jO9=ehL?Dq1g08(S!yC5Ji^T@88-gd!z&z8AJ2KuOq8EY;?aE{XODdryfq?n9? z|HEe{?C`py;m^cs_ZOA0T+XC{cLtn-s2zGqF`pwDMG6Jt9xk@+7jj9oxg$z~)#{H|YqYADOLd;VsrC$L_SRK!ezQNbzEr4r z{{ojE4=fx=@<0{sx3$e}B1fipi)Zg!HO?qGKA*P!(NDOnibgmi@4*@j3`K!o7caKz z$ICsvd#6gR_ZAIk7=_3sIM;Fh4xH z?8U2MH>6XMCgMwHQ_)tpTN5LpWK|1}} zIty*tgVi@F&O*a#tj3}RjT7V4P-c!QeLi494vTO8>Zb2LaMPakKR>{)^kog9_fA9O zV+quQfI7bs^P3lS=sDrgX)3*T@PA2wEd_?iNJ)_vG2^h|f03O;c(D9@+JhDf*#bkV zn3Nd$+4KAZ9t905eZyujy~uy5P09pvN&|i9vPhn0%r^B4%EAHR3XC~Oh{}@S=#hkT zT`ADKCoei{v-L@z$jih1sqr+P^d~wy4qcFp+#eH@AJOQC_8e?Pc>utw$%h#RS%W;^ zBM+Ododl=Us+Oec@c#5xhEDUFkSG*6qi6zAxv&Gfeo`TtwGoIpQ zg>&Zw=K5ttLZ(~Mg)6kxix5(eY%m#e0-4T(3ekn#mrDXFQcLG6~8ABVDqt|DSSZjZjVj$V<$+QTf5F{AZ%R$;MAx$n2@+T*Tx+PF~O|rHB_Z zrbq2p^!=e_IT<$F7%Y_%PR)dT&W0$-JL%WliI>rUekY~t3VG~VBoelAv=5Y@&s4MW zrWh(ax6ntDk4k*j`XWNkVpK|BD2om%22%}2GcbUZ2*^FN92{jDfY^%1GBR}aEGf!| zzL6D&$L`o{Hfsu4xy=a*)GmlKWVsWww3 z?|gMsemo00oA>#TDMnqfpr^h1V?hD}D=zu92*2crN(u$JaHQ>8mlm_5+!F@qxvqwW z1y1c9_npSwic)1Yp1;y+NwYZuD)~<1f-)qCLTScK`7F!Z6jCosP{c&(m0E@#3sDkK zmP&qMw=md?Z)3xL%GvA-c*IkaI}2aetZ$kicTTYaWgQMTb!q9El8VbFLs^pZ{&*oh@@8)6JDgW zi_b(eo#8P4*n2+-O<7x+Hd^fZ3(uEOmbt^_cDvQfvHL(~O>WGzuV~H}J}(r+D%Rfe z^A?eCQMNbER8a%xv^Rs~>r^KrhheeTWAVr|CG^^BVQpVM3xjEl+`SCZmtN<8N743* zqhJKl()n^-i~v{zj@8~s|5fG(iC?f<1c%pB^;(Sim;uelk?XIi?0RzmlH$XFyEbVk1-47TNF zV|Dbq`wnKkO|TE3deZ@ue4>72A7l9G5zJi@o`0>Xj2$pm#ke2+K9c9{&vDPp?kzd* z*#zm0{znEK2$sb&BE&HKK@-@E8DpjR0xs}(6^f#5iDJDuUJDuce*J4WV#4HMR9d^E zY%$nEw|(L~HhcH+=bdSShHg!~VMw3hC)NguXZ^s>(hHvip^h#UKjuz|tNu-KMpBoH z#$$9@RP%Z3{QRBYoA!V)phK36k0J=uuTqC+>Cp2-zswd_wzu7M7c#hGhkmZg;=StN z?@P!Ni?6V-pE3G3UM=zSZ+44~s?2S8C@g-9vJ<*@e(g#PPv4o-KV4TlTz?Bm&(SPg zoG@~ZjxYS%UzapDYnp>$>ZChwqS*LATeRWtR{zxYYv`;u_Rhy!#7vsVc{h48{e3;2 zZrT=M2H9n~>?o3o$)lRr*ll0ZD1raKB77Pa07)GbJDEG5-V8TPCQF>O!P5kna_4mM zk~hax4XVghQK)13X8OOxD77KxHur&F6TPbC&p;cJ7JJ{Mh}s;debfZR!f%JE!u`~F zu+4tT6B3uWm&oL@A<+C>^XL{Pj3k$bALIyT(UkJW<8OpPZ8`W#7_EM@iH*|{A(W`6rI`_slJ%xnflJKC!cZj zKT83_IFsbd*UfH7Qh2RFjTt8%Q|3L-eHHaZO}|2zA~sfKLa5N($&K2o=NXq4qZE^A zw&_wJ8)_DXfQeoH{0vgkfFs|_FgjymvSA8^OJnPp&A4!So&T8tq(_1wdiWE*VaI6N zb9SJMAqqLVYkT}|3|twauVWIo2dLAQj^bBNRn_iU@rv3yfWJfs@^H~UUxwqkZqQd! zXK)4VG6i+h(4q2Fw%UpDpRrgcs*U}QKe)w@?eYs-g^qW$%QFE{d|IO7cnx>gPWPKq zqb0@6r0d7Q?GCpXYTYq=1&^bBu$lS$sGOf^3*hCxmcLwMrGKUUG=^SeyE*;4>Kn0g z&rj(akS~Osbd9k#BY|JF$wfSiG5cdO?}l`LsnAGP6qnQXcqonfo|p}szYsKJAPhtY z0b*{&?75R9_?^HW4g&V_1Fs4`C$JNu2T;#1H53F3Dd7l>hrXiv!j49v^n8$kaGHcE z03stTBaa~zEG4Vp?_j-MHorexnJJtvF+0(@_tPqf;fL9J5T$Xln1xKmg>{jK49Kq& zCt$Z`yTSx@UdHBXq$`>nPp)(y1&nT>HC}QDGcgEmUOxEaMzA$gY%-Ge;ognnaW8$=lR)_PA-uHqDRnVcJ|lZNpI4&xR~x)O0EJC;1ekxhw)$_ zv-R>6I8A7$D{bF26CbYLZbdH(T#)%*l$snawr~B({!B!K1Dc)&0H9BIsommZ9I2E= zAb;4&_misrzNpMbx7nk7VwF7eHd3I<+sJ6S>TSO&-I0Eqx$v!r@SsI_EbZS0B5)*@ zlftqT=Cse$^?jkM^f1cZOl+tGWtL z2V*kT#a#yjM@-TN8N;+^N-OPfJVpKzwI%DOcG^z|gQF8ihbYG+#Jtv5j7;c6J=4f$ z;tEIfYOmoi)aeVZm!kTX3*`2h*l??Jii1+@%}rd1RS1@Ir?gU`mExep~S<93x5 zUjQ+`)17;F88sj5Chj8XTPp3hN&yq=gXWdok{&tr!~=M)f^)n<^Pewsh2N)RN*|KNi1wzf`s}*0#Lu2S9-$;JAiD#Q4-FrBaI^|P-IGOM( z^y>zzN;<#ALegUUpJv~SgZVSS!*+W#Ll&%9-%?srDCl!x;Pzq=JIrCSht1A}RwY^JO@gU$qqum@FQUY3%Ywhk_nDfKqsz4QA_9nb=j{ zQY+mOk4VeUobAtYeUofpNfGinJzOT?N|LnO|%Ey2|ABB zy>j$w4mqJC5UujJc|y?^FoJ&UpW#+9J|k`GOAxl&8!m-WDww{SZ3?;{%Xd_-(X9dYDLcZ@%F0X4tr_Ml2a=N$27#q0DjKnmV3dN300LowMEC3tCiE*p*+q7g zTsv&sB5g{6gF;hFGLBNVvXi}w?{NxtlEkz-(!5Y^i?z`<;9d4phJo+&L&4z!3CyG(+5Z zaQYSO$_5V9drx+(+0ivAEE$=&yQ_p^r9R_?ts7IDawC2} zptwO$WOzPc&ld4xeiHgwpZIl~`_Xh587umXlsrzVBtjcpqRkW<6^GG*jg6~~E(9XN zeNdBQd%aO%0ZU%&msHh8*WkV`EX&K2IU$?ixHF5S#@9<0aA|Y$-4O=iCT01*vL?dl zsv~%%RIzqYJUlg&=>gy3Q|Q>+KVmE~%{o-_Qn;Pj$$%k;$}<7gn_n|g_AQglLHaqS zLFg29t^xvevmy%>Zl4;x;{hfRzqHKAr`q}#wVxy}x@>$hkxjf5cMc=rTC7A@mQ!X_ z1b7ZOLQv5o1uikzP3LIe*-i~P=~E?mgbYluG0>9Or*8j5cadD0rRWV}^R1es$d(Ce zD^s0s18^P|^D*7&J+rD-44|q)aODLtnf*O-@G!SHC#c_FM3G8KyQ zqlV^j)kZv(0>SY>0JfwLxhg74_ukvB-1H_9QdWYWB#b{$lK|Ll=2(|UD>ZhDzW`s_ zsL_2RdixdQG|p~NsWmuwNcZ`h%buK@=Dp}=9B&#v+BJ^j;04)Y4frVINc7&^M*&Lp z#V4uKw8lPLoKZtmZwHw{gV`J72`F$(h-4^EfZyqSdq!mk@+PIuhKNr12z-sxK2(UR z-EB+CFu6_bK-w`;*BbBMb$4`R4xF>ey#FKVP^g+xZPp#^D3E|?uh=j-7Ioc5XYJ#m z^GkC+M(xI3NZw5x``$LtbaAsLu4|xvjDs|M$y~1?<{?cB7~apS2lLr!<+_uYgTziTB5Q z;%&83yEb=oq-+feZh~|WC@BVLP>X`FPn#zdy%F2=V&3Pa_p3sAeJaU9AG>-0y0q6# z{7!~(Gr*kqtiY)bzB>gRPTN)xaH|#*-`#v+vXT{#9pZHH&Cp+EZ>j*Z&-6D9ZZ0m* z^{bo!xDo}wg&pngdT!`S?hoZP*e??X`n1hMl90V7?Vg9$vzZb|UYzTZ4$q6hgl)C@ z1B!7i8{IO!`gi?FqKk`*Nfm|yHht5m>wegu<`puHE)5~41VZN+C-tW*Ckr+JCU?FU zObEiIur9D&sD)q58ZVwBz48$|zQSl4_u~CD*DRw4rp0f zIDUTF{x{SGCv1*iNa;vH=&qPT<`1Nw`jN{0P|7a(`m%>TpCObzG~!KIOc;)|w8R{V z^<%L79~1$RVRR@d+)g>>>GQ+xzq3Jb^JwWKkO17O*#)%OSlMm1C91)lx%k^PLz&2% zeU}#|8b;4>KzpqVjz`10+saAA2dk?~(?VzdA3&I8IN-psa+v4k`9SIltg=v)d5{7+ zqnzRGeLTt)^H^efdis2atDbZMsQX42dYCkF4%Ci)`edkV#RS(fF@LHZhAyqHq2FRV zK;{LAk$PVL&R0%C-u3PbuN|cD1z#3a&4aJVnAq>7u)*k9tU3}9 zrlP@;6RzC_h_*FXK9=ykgm0;(f4nh8#No%#iOem=jc%P54;skJfJc4aZ)=t6s-*I# zJk_PJR6D=q{@od^h5wMVB=1j~0c z{L1jjzO-u55a!5Yn*F`r*AQya8v)1NxErW}_=vUGZw__H{u&>86obx98U8u3>nS?}lLlNlRFUU5 zpRKys1KHfW`B&jWOuS4VWYDM^a&A810$={!P-X*vT1S~F<9E$^!F_KxDc6&# z7{;Vmq!D4EKNH6q(nME_=&m30&#ydIcy1bQRH0Aw)2BOWmqY%$#9ZU6p%K}txey*p zf(f%@L=MaBwuZ zPtc;B{K5y{=}Hs^md-YFrL}#xgtr201=ouUHteQWmeP_(4>njLD{+?m(@kn!3#5Ns zthfLyN4=-9wmUO6^amm*Lmk%iL6GWOy?#+|b-c>XRZ-CIqK9e)gLKw;eJOX(}^z}H5z z73i(z7vH)SMum3R*-LAhK6cI)yOepBm)cKdzCdk9CH*S%*{dnPo%C5&i1DtmoWYCw zvmS&+^m#D+TV;oEIVKuDUf2(!QC33AVCJu*FZl|_GF~9uy}WMMa{G14jrEr>fHV&5 ze&iL&`#T+v_YW(YtP*_VX$3!?w}mUBv4kc*t~RjK1U~ie_S{6giRNxjABAkbt z6V9!;BPT@gTV$cq`}_l0FA1&QOS$1@u>$d{Z7?4Z}=k291GGG-tOT32#VQb4g zKdGS)SoR`DU2i4na2mA&CTK=208L}!G+$Y#%%EK322f>WRm&JGR(y__&F)YNgb`V; z#d=X=K=`HADf}pyOoPZ_?HMD!ynT_l@UjrSzCeQO+n%ji`kOs&EK3-{C^fZ1nftH4 zhNG)D%MPmj4Ldb|*WbSQrQ(l`@-olK!G?nt((AF~{*;_DT_Cn+b0lnx*R4^tB_1#+V+1 zI|B8+LQ^0>pAB^k@lTLmH`n;=?pAA7tsES|MSfK&o9I1MWdAI}=f771TB*wOcyp(h zpw*gsx{Wx9<&i<8$9Lu|a-Z=Zx?=flRS&mYg(&=rTwtQ_ z5pw?29X_u*3HfJ?0ZDGFBXvx0u4v7xLJYui%u?cU$>cw4-@c`%PA*&d~o z#N$IfPg#FzaA7B^V6JX5#+j(3a%53N)*I zTXXQ2&eylZ-%`hiB}*3c@4WllCLa}u6jOFKdv6C8;V0gvThl+@vHxPcx01_k01${G>FzIizND~Ut}v0#o#Ay zkY;RR00Y5y*efZS*X)sK(y?9j5}v%K_V*_JNf1M}E+$ND3kpDCg(O$i)d4F^J$pQ? zX$L$JYkv!)4MCRu-7)@YFz5DCx>)Ikc=(pgfp@LH_qdezYqM*_^Rkj&2rkJ3p!7wr z*%r6=qcRkV!>K6N^lrDy<4o_&4@WvQ=x&piF;wj_V`$8Kf+ThsIXIlM!xDC0XQNZD zcljiaXyIMFRbQku(>)p&h{)iE=Cb=Q=nphh@@>~)1OQe~gTMjTh6#FiT_I1viFTReECjmv<>cP2|blEPml@wk$a z2}&r{O)0|j_M4MBPT~cM-XJpTM*zcr{o#DvVwa!}2>r}fnxFthHw)er=^890x0n)k zytN+Zeb{$vcx#C^M9Ig;Cn|~%sM55yJ2d_NRmt&8^IpSsdVYR>d6|%7^Yi!cl7Qb* za^#g7G(~Tt?q-Jp`_Rjqw=rT=@rhEnOic$cRLE+B@!`Yo#0})9(vXwStW4PGfsAfW4c6DV;Loo2g9H`YN(t60qeI za>P-Le~=9(;Rj<6z?uQcrBzLg@d2COnP`KZcr}bz_Mv#y z`_W;26YtXx5=mrVY?j-iWgg(NHR-qij|=c2OT$kLR0zV0xd-edzJ_M!<=~t-Ajgp( z^+NAOq~si4qkjiI#B8Gcz%w8Vg;FTVKLdeptb)i`DEV; zR2J;Ic#h*Co0>^WzoIw2zEK+5;)e?IH+;JIguJ`RjG~w1Ng4oTChqpm=lM)*m-8bR z;AF^~a=`|l+TEpMG8jCm1K`q~O(?v1>-Dw}hpg3dL~AHhFTSj(ap{wfp^=ILJURgy z*w|KHS3@Dg>vLK5Vd^xBFskV_3)80_ocHsS`K3u5rIn>0?Cn;XHil}fMnwadKvSnr z)yfR2%G&D-a?4$Jk`_nP#K*q*X|9cvvj@n>aOgFBya;r3O>tRWNf*657R5>BaI9pT zxt**15QmaM(>&$*GW(*{GpX-y4Z}aH#;4|8oKQ9AP$(%>^~wSpTe|l33rrHU%}}3; zp1|h4t%!&3nnguLTIjx(5J84{9Le#7)Svvu`NG8hi_fu@*dj&*|TPc zw#Kl@daHzT_W6zVSOq?Z5g^atehAo70GVe{ zWrLTOB|(x$A984oVJgxxCr^!l9Ibmp|GWQv*=6G{jv|7Seb?nCH%7v9zuctUqyw2N z=_7WxE#h{x)PcI~7<0-eZ7Dk5h>_Uc-z6k#_bp+U^TB>&e)@_TETug&=7+|B!(LjF z>1~UgxNiYKi1fnvmTBbAZej#RJj*h;*t#kuqMB=F4e}bwqRZ*A{wKTg)8X7&tcXWb zO@;4LiG?!Pi!?Q;j#!h+z=UkeTRRAI8c2DDYK*sIs7gzovibvVMN1LJ1<@^92ue zrb6lY!)EKX&82fHe7)`D$GzA1{X*IH|CnnHH9ydLMjEE=}0*-9~2H#m@nIqT|&?rzZV`z<@^k=L@|$Sc-~*!^Y8aG=`Pt~_N)pLrEO~JCwOi|34+M?{T$$|mxJbv^0SN^#TewoDeq--fQ4M^uc{{}0sl?8-8#$aT;kA&r0T@Bv5C27hlT=OK@9H-?tQ}BK2_oU*Uw)EYZRLF6+j*TKCY&@h0$6V=_1A?yyKB&F7YV1H z0(8(-j_sdMqlMj1UO!=`Z>wSZCn*o~pR-yE97nO~k`vTiab>%3osDV+UyuK|5b}m4 zN*TGixgCCvxjQXi6_FN6wQ2I+{p_|X47*Irokq&`18Bk!08!nZPo=yr$h|ndSXp_F z9RN0!qWgYZ`dQK!k-s&RaeciJ({ZzB%EAQ@TJ`I#6L2crY8&c}qJ*-^VJ=)C#UBi4 z&u}|259YnNoocNSr}Wd>ZnbsZt-CRWV}iI((W5R8S9%e{B@{LQ3h}A7?&HGsM2a(I zkfzLB+tmr+p-^kvd&(R_#AVWCzTOj7vam9FSEyF+Vg=oOk)*@p1}X)ADKRyTGI=b7 zyRS+NK$-va^zfnVSo_JIn-2l>Ftf9({dI8QIt~elb2!*c!9Q2sCxv(<$RX&6odyIx zrKSUlQ_(eFDF*w~6_U$25a4Quw;9~D9D1&|^n?6Yp8}hvH^txJC>Hy7tkdS1N)|(G z@YlBxws^(P=&n$7&PWq)^{48(MY>%F@zZ;LOR4J}MV}MjnzB)&=Rm&kS?M$_E54PL zb#v2&+KmuCYO8E1Ehu;yTq$EdY}Eh%k-C4Ew57S+>dB6-|ACQmzsQ-8V3WDCfoB#e-4+oHTF~`Ir$ln$^yDRdtynWeUHD~ zFI?#ydk-NRmYlhwefUZBv*=}YcnOigC?4F|+8oN*R-&oeot#YAcav&SQy z*OBb{#s7EuvMj4+qy68zM>e6B6Y0Rz+xr@~i3c%jmrI!xPRhlUy);il$0FFS_8hRr zGXGuLfS(Lgy%lCyeIX0bTK-!huG$y%ab2n4H;)?J?(Izq_2}ONBzIq281eZ4A}4TUK28+bz*384^~Rv*ha{xQKPRfn_;mZSvFSn zo~o~V&<8nQb{J5NL9B!R?@rsC%@`!HqAEi!-~T(M121>Api;b$WdS& z%BO_nsrCqJ&#ZrgV}hgaXFF6KWf?V5D73c_!LCWX{B)4I>-{d{pzK%CcMhkDDX#xp z_^9}TmyoxYnh?gvT8&NOAcmavhM;UxZU~4663&Q;eX&aPsJjy|2o1|1KrZ~>dtLdb zM*9&Ae1_1FOHk41Qwg(^j6H-8h2fZ@;qpkMJ7I!*P=YQY1Z(Sa$?@hJ+R*K6`zCJ* ze6~1?PyhcWLB}UVtJo}9TZRCb8b}2y0{2pp7ed(NUk7HTx!X=`M9>?7lB_r~F*Lbtd5ZmOfuQH-<3@RGgl`k@zRBM#O(vuhg#lt zc%??*GHm1u6J&u&{<%g(_=>nt>Eq%IW*)9y< z$;!$C+X|o%@ydSeDJ5T#tO_}IP?#(xRu5?(r-(05ht73qm?}p40NjYStv=g{e``*T zC}D~o$K1o>fxpXeaCA?`v_^L%--{6f6@}UGQien>Cx!ldS^87I=~K|q*41rr-TyUM za%z+Lfkp>9V|cjhyS3_zTwHCh-uYeVil2l$w?Zw|dRjE)Q&{E@IZgmEMdu7-4oQ#P z@>_Y%6Fsw6mmv(K1Jc6++$59`pyoGNsOJF-OK5M+02IqH6c?nmsX#VHuZ#VhNSu-Z zn=0!uCYFCP6YvkukEiwNPq<~}{=FKqsfYeX9*gyMe}%q&mA`4QTYSMt_GIm-0 zIY8?O&V@SVnmMDq01`6}sEk|_I!X%YeWm2>{4O7>TZdaGx$6bK?5 zqV(5`3L9fS*he4Eq?U!*tCf~x7nh7UP?QagU~O4CUm<6B&a<|*0mr=Y>FSE&wW_AN znIuonR-nSw&ThG)y#mMtXPd30(a238uU5p*Cs_F|+W+l&k_irO=QqNJQ_LxLmF-wr z$+*1P%0t)nbwoU!6ak7vz9$-(OAV)A%HkVpKYCX;t;Jy+qxQD!k}3*JJmH8Y*R235 z2f~xJ5c>AJ!F{`QO5f|dniw%BE&?Vz9%)QRz6g0EbXStO=0rdmUEJ}eN4LWVks#S# zQ|O7z@wjl7i7|LXlQDm{S^U-ndAX&#vV`!3UvHG#yqOj7bw3%eH1W-M<38)Agozc_ zvjqKF^Gb>&xN191UH!Ot*+N#hs1m@oqBLfo%*gc5m9scPKv|<$IAsrb4*@7=hVRXX z*CzbhJzvP@6vGR@5fBg%MiY#sNLIEs7i2pgW)ojN7Qb`U)t#*{YC%u%O5}R*`{qs_6H>*dmW zqTsBOR@>!o$3BkN8{)^uACLdAKdvk*GhDo$3&Fm8Bz*S)cuY%++pAcEDsPY9SYt*< z`*t=qqEo*nyet^lba-p*AN|)W%VK=>_$(UwP~_d&*wxk5@k*Ph@K^q1{c+9hoTGN}96wO_=E(&smC48Z7^~*7d5)^W2M~I9zBDmu~)XR{UZ=W2R6W9hR|2rf3MMv~7-#xzrN(y&fSi&4bBZrcy zl(9&7NJ5%J@xeDYgdA*NrdU;p-j(vspD*3)G5aLF)UYT)TUXS?p{QVSkdEYjt}OoD zWzBRIVaxd_f++H(^A0B-L$spVcRg`YA$M2EX^^rmoun5JH-;g&RG}N=b9a;bk4Z0w z?Eog7o&k9FkJ8+EL%aJsWc%3K9=a?@?E&wa(YHAmrMUx#m_U^R!GB%ovv2tvoiuo@ zv$J!wVO6DYYT%3c0rNag9C2tP8rIcmNpt?QqnO%e&!et>ok|1m*;cnzq>ue^f{P!s zLbR>*$MBqk`3hxD11+e>=1|-#uHr`=HB#PjB{ zEAr@_mmwiU^oSCVd(WOf?`>wei^__GZ?*=!ZHz5p|Jxo6_#UEB(^0Jed|*(x2l!QA zh~jUe&Ippg&W(wQFHQLLH~X1)TI9FCf13i%(iMV$HML=QzSzm1ailo>=S*V3c0vDs zj_vk&*hYW!qp@C%tESPqXUDhj)w{NqG(bFYX7%vBO^YdM*i6kc0aZ1Mld`pT%P)~M@4NlAAjEg;>U(h}0$Ee(e*rKJu^hajDz z(s7Uw>E=jxcQ<_dUbyd9{9-T$JD&Z-T64|0=3Ejl?;MdIj?Jt)Xny9#>@{JTcR^Nj=tmKp; zE`*lPFYWA{K8_;HZLrHi8V^E{`0x>*uFE)qG)m3H+L?#VqSLPl>5S|!Sq2e zpQJl zubNIWe4f6Rp5t%uui0WLOS8re8|oY>x=0Fa9sUsd0^K}szC*PBPjTYhzpO zx2H)UUHFIQsNDi5je{1f^XqaK*A;28|Gs{#8inoOtBhoAC$p_H7MUvh+`fL(`Q_>Ha&xr)3^SN*B%~ zf}JCVlDDupVPu2ee>vaCiJ12cC)InuFx+C3A&qU6Q)M&uDSRx;poDv=vM(?7f+77A zdswRJu1zD(^N^@^3rRW-y!t}z3Mbpk`HRl)V-uR%ym$%h!(rPkAo5EwT$4y1V#^aYI_a^scJ8Vx* zKGLINAVmgHSayR(+rUl1FhM$^4=g$rt>CS8u0Eu(N+tB8$Rm0YNZaquwcLlYFD*Svh3I%#Qf;4es;joUlj2hm1=X|v&ofMX zR--V7_?{zY!1zTp33bVP78e(%YR9O^1w7Wa6QX{-2$qIRZF`aT3oGa0WIu`O!f4Z@ zxD5mF5G(eERH+k-G$_KoO|6GRQ>DSPhi@BCnoX)cVeDrvpzI3twEec?IK!CZKRV;y z)3ulz)hIDb&47s$1t*U!3GwTxPx~NY*7q@ffZ0VUn>U0oyw`*#xGd_8r~mR5X4Yl2 z|13)$ndWT4*{Blx70-(}{@R230{5pyo3W@ZDUHl>(v9c@g1tkp`t=f31{Bet@X|ygm8?Lm<_zP@; zp_j_D7@&3mg;zL8d2JS+O8XkmNjh;gHMLFlw&q(-Dje0cgo8#b%5?5dZ7}>6rjM>R za-?~ZsYxVRHR_3F5b9%WKMu>9RWo-^$U;7Zw$Vm5wPB}6N=(qYTFR1hNG=x2Q_`GM zCl9~i6k(+V$G+g~oPB{ZD-{nXqd{r0OMq&AriSQDfU&3|pjCi-`7iZ8Eb%AH8akNv)5;cm6nkE-5pJ`fpT|NOxg+ z^ruq?^#10+&ll*q@|_ zPbL=vw7U5iBFx=6_iGP7#K=E6qGzkUpRtBYN-)h@QrPzATbf!b#zHKi(zs|n1rd$H zGh-TATg{hSBNpr;bz5mF3((=Ysri&dYR7kWhGXA+-xwJ!be;HlLEQ^anj>TWh+fPB z4dDA8v_*r|AJ)o{FAfB+yIfJ)sm}n`&eqge_dh8ML5=&y-HG_^-3@dhaCy~L-NEpO zSHMZSkNdWJ!|dn#){FUorms7hQT&y9-1nEl6j>Vy?TyKkg%f2vJEhmYgHWK9u(!7% z8(N4_fq_xB4udIV`aQH5Q3aghU~}9ac@ltS_x>ghow?9O$%9r4Ua>O>mlEGo|2x0Z zC!9PH^EJO~gXI~By27cMoShGJXAS|&%bNE@+%%1jJ##H7gGFO@VDZ~6z*BoV=X38p z5Bb`@S3%uF$NL-P{K|%gD>-(thZvLl?|yW;u*Ri^ZZa{EZ*_a^ZL};@*CX+D&8>T2 z#-Z~_#6H3Ss>acqL4%;@p{9F{$s2Xer^iDNo-QsNQl%c+Xb6Vn$sND2#Pm8cKw$m#2R=dR`}wcw7VFK0`T)EPcD1=jr)brkIaClc4~Iz%lfG4hyi`8D9|n zZat7rOGPE}6FelZ2AlVEG7jiUPhKC}b53ibZgnewR_rOWTBU@Kir@4su<7~3N<#V) zpB21yk4L}r_FUW^mhkl+y!9dAt-)|0#oBvV8R?*k|C1T=t1SMuL+OGQur-&-5t<&% zzP~*zq7~Xu?ryj`W9O+fa^KYi@P_vSofOLoS6H^5vTKdJ@sBxL3y~6Yv5I)w`xnK` z>}23(r^_Dl=Nd-p4YfzrK(7{2x65xz!~7zrst=b~eJSkXM-%Et{S&PrwYA*Aif%-3 zA|^ATJ=Fa26h}N-j#@Hk7#UIjofkDY*gA9pE-(7DMGhD4ZypjS+}yWz_I&)h9YjTm zM20PlMEfX>W8-HwhAPTlVhz<~y;g{Q#(xB&f#>u@pUD;nM8`aHT_?7WHTC%TNxd~F34))0TzySjR?bY+6O{sY*e)7vC8szsIjOC!>Fj%c zWs~j00C5AYK}t%pY25@HdD3swC?1BR*-_6QUMFAp;*CZ)ot~1>XiZM z`m59?|Eczo+z^(4yV4UycKirhw!1M_yRT)O3=CrUr8@R@F-AnXtm~2>oI1J{+J<2W zz`6-W-QISvFD+%ixL{e_cLsWGHq<5oz4#FSV8+Spf41molvdVC#OEZ4QppVg!4r9p zO6h3HcJ>>ZV!qN$k&kO?QW|}*+6_ycs>}+q7oORJ$1CpJ9Jbv>lbD0_WNb7Yah)Fx z&{>JqIxp;Nd{2vzwrUg#igaP^+V9URl~R!9DE)RrtWF#N zPJG6|YhwoWL1t!V(8rpLv!fPaO=S*_RLVE;Y)~7!nG!q(Qp6I!6jt&^|CdR9Qc|CO zipf@^_2~NaRUe!ibs~|P--Rc)-Srac@R0m#?)di&!|R*R0nqEl(uYIzg~Gmh`4Q;TSfksl=@LgQqj``oW+N~7@)v$pGQ>Pw^b{9Jsc zuj7c(iC4?rYt?^p=&w0cQ{%IiuKSwZvbJgldOoXVD`;=!Co+-x4*aOmHp=x|e7EMn z?`CQTSfgd$WbM!X+$xDB^P3SCX4zn6!ZI@Jprw_I&oUmnmf@o{niGxL_xsk!MLC zZy;nTJfW^3-uHC`qapC;zvv%NuDjMS@`SdpAY9sy^z?b=VfoJ`$a7_jS_Q5(PwrA8 zsyvK$x$}AAu=Z#CU|2c6``rfbJUk9U;)U7Iqq{lV<)w)bo`rHL^cRHi3%1>{B(FS=N`z|+u9@mBIEvl z8T(HlV-HZjwpV)lcIA4SDMQe2dllflx+jY+cvc>b+}s~K-_IMeWkU~pYxitsnpAq> z{I91HOSjY4Yj!AumZL^SMp~6NDlsu+hA++oQ>CY-fyQSdS^;Kn8nx)bf5gL_%7v-6i`xcwVgD-oyByla20Hq3fk? z6H*OMcp`~qY*%6%BOaE|Y)n5s3edddmkfR$OdO)LiCteHQsm+)5UMMM177W1$_pQ+ z!>;2Riu7ZZWX)9o>DpFPE3zq{i+AD*;HxPKRs0JDlx5KVQEZ4oa|_w1a)YA|=OFY% z1S#eq(GkYx&m!fnA*$bn`aE1F^< zC7er8h?B`S9}kbFmchk4)qk_i&5Zg9LF(;iJ#ZX|61axJsD|S>th(O-V%T}L95cMA z*J+c z4HUc_w*3uapO3~1a&qBqqXT_$kuHye%)kXBY$qEUUzaz5egV$QYTDY#tJRRbdzpL3bgDFy6_if4nM8+uS{WHj@{>!2$jv_Syiz!}hD;pi1LeVW z3GVlBkJ)J6)B&Na1Vo>pXN7K7P5F)qLj4!Oaa;O`E<9@9EgVIh+J5jaeE-6PN$bT` zXvsE5O8%=#3&+^fxb{HoR~T^hO%k%5vmm;Fw~8AagosDCVaXrfBrL$aSng_lMNg!E z?+8Q1LB@GW@xw*aXl=jgK!0pl*Tl%K!2=B2f|&#b*z!1Nv;wx~zkjb@%7bBPKo5N^ z8@FnETvdg6X-VdAmSa8Ca!&=TDx|iG7(ey#NxOGSH7MeogGU;noPtumUl=Ov@fPZih8cNy4bkaVVL?V8Aeu=#WvNb_$gT)yt5I)!KWmSj3lXSlesmK$QX+-o~f(8z%Ix z7QV(Syo3rYs)fWlcqwr+qb7X5e_Fc29j~H;@wurD$LS)I-myAG3rMg@uw^o!nCB7Z z6TUL8^c=B*xXNl>kB_mTa`N55g}Ti(nitV#Uv6Sk4RS&fS1eN1`r6 z#xd|>YaB6P}giG0}_!ftA!-zgKDZG)}Tac1Yxm0WP(#*Jh{yOapo8vR0O-xyA zd&-8t%GsN#z}1nMoZLy+@45{+g3W}2`I#i7g_5BsFy%W{>*2;6qu3tw@0wbq61I>O zzG4WSPE@2Mh!~%PAj#iRa=?g=1I~0??TBt)ZB~65er!A*;D1%R-3$cX3Q6_Kz@bw! zkszB*{a|nQUh3=)AQGZrR{L0yppqy)mK*%rB?NhS*%jBDQ~k{UnVgw&WHFSkeC?3aal+^;Wi~TnSKH{|I0veX8?)8gp+ z8n>?P1l)rxX=tARJlB-&P(VRb9hnZz4BOkW(#$Z@sn{>F9F+J;xDyFpHuG(WfVt$y zHeT$ZD1%MG_p3j9UofbpQfo^X!nxXJcCyNZ$ct+8%Oy;FU%hR6bxKSr198JL$Faqo zX3BiIjf=u6LEaWD@R~XKo2ByGPV6WV4_gScIfSQ%lVs+L<>ec%1Xplmy@bpx1$_v1 zimN(KxcKALx2`hCK!68r=Yia|mi7m0aViG9jCQwot|Fi}GdKGIi~BzAl-{O2?15L# z(t;pcBiQWh_2tmRyNoh`kakoOJ5ar|3ce0{n-~K2f)0sCgF+#=g^!TW+(~F>K&&?lrq=owF zuyQ+ZM*yw~8Vb7r?>X-5#)U?ER*+5diikif;nkgBH&nYfyJg)u-sXF~H|vwI&V-n` z4Y9v<7rRq$kYzmX!aX0M7V7Bp){FZHssM7tQJ{cqceZ`qSlXI>aDasGW)z z4XOS_D-=ltI{}UhtrUMEMCqtdk6y92Uhb*Z3eVuTZVWUKA4Kh@RM z@mKh`+&5q6qJRAty%8lED)@iU8|t-Slf9k0!^L~J5(g)zz%BN`%m0nuOktFXUz?iD zgoQ*<1zr|V0Y-H*f#7XNXcL=Bv+1Vd!Jv36N%#_gKm4EQ4O&>$MHie-()TW&)uD6Fd4y#~(kzGvH}tY$mcfOPja!ok78&W?j2>~>a3{Ca@>o|an#TX+43s`%~U z;)aBP$fr2<+SZzqf)ZDtS0ungchH{{g|4xi+^<0~F|fou4j}ER#EouUGV1c6m(5q& z*Qz6sR_=wD`^NrcwT@E|)d^S7-)f5p<{iKT?Vq1HRHv@(%AbcHlt#x8D@pU)zkmNu z4`%m2FDcS&E-Txay+ZH!+5l89-4ivLYb<|-qEpz+VYJx_1vFGa4~{>NC=&*!UH~YN zpyze})}T=nAj|k>xwU-UTgvM&Nq)EbRod?YP>rwefx(_x(Bbh$V=bg-@s-9~@0`W` zmX&X2nl2Pe-FlXr;NuzUv|Wpvu#C2LMEthLx6Z-f7OL=Hqh07p>%3{FE!8GHM6 z*JfvN}HYDu#y3}m@Av1r5 z4{h;3YiW@@OB%yZ#>XzHDk~{1^|e27&)|2CQ^Txw>lxP^Qg6xpnDQ~L*(MPyFn?N(Kg8uv$%FAt?tmiWfJ zRm|&%XO-t?(i^d#zbXs;_+Ef0feoD%$_CQ%ui{6=jj z`(+SuLHa^TtH$di!cFu0Lhj5EE)d*zgsnd-XB9 zo|>H0vRw@AcVP=9yn#j1>Lq1N%`|~)v-X%RPyYaj(sN0L)USsomvQ^<4bLv5M0G)J zWHaE){PQ{nwQ#YgL#Y9HFZBsX0OD0mK_gI@qCiP%Yhy%J&wb+oklOpr{6T#SW@X~UIvb*3f9ZqlD8M2H%aSsXDDrh}F!^{Hb* z%P>LHy2a+Gd5J}|z}-C_Xh-dBxp*wk_f{A@zZNf!JdPL+%0}yO8CTQRLZ@~CQ>mEj zkquaGulG02J_s7Ncmumv#VUOoyag2=LCkpMN>@rC&w!r74tcBJQx5;P;t$-*ySL=< zZ(VU*;Y92(-WrxUux*%7-W{i5#25IVo&_9q;Wp}-I1g<{Q100~|AQHwaT6hTnaQXB zAwBlf;7Ots^S?aA(uc+7bH5{cMj3EAvZh3M0O%7k*LdVf7nqNQ$)2x2^ZWoA>gEV( zpyQSfyp>9z>uLVR1a3GOEk)K5yR|iAnA3?14D^2M9@8~a5=V*JKHb~fVUjIxY{62^bshz(B;ez_R{p%#)#&m?-)F;(q~#GlS{o@! zKcFIQ9i13+#w}jM+@ERs;x*o$&1R zcUNf@&xL|)ey`i3)igFq!C>8Sh;j3Bh#xhs60uT({|_qf@^8BBC?V~NzvF}d;IwyW zX5N?{WFv{Q4z8N&Y=lYQT*z+yN-nI4-Pw7SA?kPBsbfMB!%7%$@geb9cBd~$!YwX& z(i0%M8MSg9PEsw1c`ZLzi}IJ7Z;@_+ClT$E+5zBY@b?^0&yypb_N?Z?9pN6sb!=My9GVq&CUXK{!@ zuCjbqfr47ikpnW3g=!ZV$YInT)K6VI8-X|-NjD}M>7jox>+Y9A6a!Kvp6WTWOd^i1U z^wNiO=CjR8c1!-;YO^+$lkfl;t!N1_S(ihM_fW@eC)kxT6zS|=&wo*@Gb z%^93wG>rkpGsDAbK+DQHQJjC0o;>H}!j0fm37Jl`Fo|Y034Ay@rpYs&kQbR?Sm-C+ zvF%5wNU2^b*b?dB*@r|Xi1YfKIVyEuG;0v?eu4`wZW~YBy^ccfa}R4u6QU+%JSb{|i}YO@_`0P`v?ad{ zo4vkzj8R5Z+*rJR(Wz)fA9ErVH>FgH(tonap7L1uE-PyEZpP%{&Q{1xB`PvS^NR;S zG)pcAe6DdmK|EV^GfzcdR+=Q)r@(uHwOlWj$)mZ)^WER&8bn|J{Q;VVcokc$aZuzk z2FNI4N`QH^DT?(+ZY5_+g^@+ zKW{LR0u9lXIKf+gKWF$vR|4_`lg(0;TQbknR4x?>x&<@(1rk`%04cUNGiAV;cp!oP z>-Ws=NMEi7Sy#;QX&~_*FY5V*IJ@w zNt&f-afhiq7e8HO5owW!oo_4`#)3DJu$G~fq=JvcRif_uJRUgb!3>JnWy zJL&O=+I&4*YRnPF8ig960%=1Cj?yEe|7|6`+MN+NCGM2zdeQRcyu0&vOV2L{lO*Wz zw4ciHNPLe@Y)d%A=S?6Ln%;}lTSv$Yd|PK>b8;(T_|s+gHCgdo{Zh$+=8_Szk9UgP z6yICl2v$zIOH9rD!J)O@wJH((#=DkB$vkuP#@L-9Nrlp}u{=R)fiPn|Q{jY)A8#N> zO&+yJM?+SF1(Eon@psGUtw6!XtXeJlBYZLf#&DN#LcV2x0$Sz_`Nfe}!`;~+62T_i zLAeQVoYgUkcF7;`$C!SDKfD|$y}YjE<)?sET|B$&Bv0;HF=-AV1NCBKQ6Ifzr}FhC8nEu>wZY>#=1=+0Hpx#9(O(=CB=2L=GI} z=if%l6cz3nQ*{+Pcm*NPG20kFpEG6UHg5k#(LUxrqo_I}Ey@qPTa68g){Mvfh`FvvG@nqV!^Xo)^+&(ZxDR&*-TIkdpuXH@kvmagk6L2_q zIQ)_?VysNDT6kupS0$t@~5ZWkW3U)j^8(A-WCTviWg@lV>n*@uFhteHW}tGFMm0LjmC z7~UfV2NdW>kW~<6>~E)ZZ0dHq4`4D)!Y2}sU&OkGWCjUORvqB?o7|n5Xls8~^^xL4 zr%nEnm$z-NdcpE=egbqU{M-^DR%TFtS|on6Q8>ZUw~P~(RxJIT#wmnYM1x-yQxPHv z!5ex;i7ck=(E$hMSC?DF*1@R*HGl-Y0ir(>rvybv$up;rZiekVfWZ_NMb@b)BEfc= z=?j6!PAzU|EUtgK+-4C%(uqU2gbNVq1rBvt!^@;|LmDHpF+_D5i3)!*pv-AQRMlN+ zwK%cce^NZ6Xj4wVaq&jCP63uYKnnhcC=dKnAjg$7H8o!;h{QyF zcf&ULM1O4E5CNi9ws=M2TD}Usnfk`o+RD0(+w8f6pPpVB+bgOf)n01>?mG_Vf4cVF zsdi`lYAcj*K6bhCfEGj7YgBd&NCDo6AG`5isPn?sTN{hZetv5k;F%Ql>WL*&b zh41?}1M&CACJze}6=YNZ8`|B$p>xj=3_G%51fs-h*%-p#4ZB4-{vM_elAX!YU#!+% z0W^xq^3t-BSA)!G%=f!JMcU%u&EKeeMwCS}57%HH3M>ooLS2|rVPr;FEgzf=Jh z6S_qbKA6pp*0QTR_{(`Ul~U{2Q1IiClLDdCY!{#0P!jp&p+3bHLBt@M`g4*(CgmX%K2YJfwwTM`8u9cLjylR z=5==~$eX)C{cBDs<;-|DN=`0?z189HxM1OSoxkecg1YwGI~&yi=-p`#%-H8_c*CSZ z@4*IZp^jDNH=Uz#-F?rx7S%z}!NSAJMC-3=y}1^Tsl(6NFyp(kzL`LA4toP*|JK%8 z<+-li!y+ocUwq}^n(AR_2rA%JdRmb^YZC5+zHdal3z_`RgTP`n^7YqvhlFrkzz?x|Zb(6@7~h}8UzJx_TU~wBf#3(^_6mMsetcLa9e*8dZSZ*lO@f8*pj?WPZI9yTXBBZ#wJ*lmY zYe5!o72)^^kcdL$*0d%tVjZvozC8dviv$6U^1rG9f(uq|0HiQ%9d}&^Iyl-a%$)@i zKT{%NS8gnhD~zZ*HQ8_Sr_YLFgwf>AsI)?l|047Hn3jh;7Lh!VQpyu>%#FP5%5^wh<*qsF|;!^ZGkLucGwdyjK;!`r_oI z;wkH0duKREAOY}YjXH9dl(Drxn4wVjUJc9FHaC+WHa3=)Zbugd9qz6VsLB<8(52^^ z&^m*=hKrPV5n}c);$Wqv75)p7Qbe!ClO%w0vofEpoTOhLrfPQ_gqN6aH1bI$?s12+ z)lTg}k+;&ia(~`s{~s3skkh~c9adjuy+L56vDR+9DmWBam_e$h#Qn6C-K&qh+QeKG~P_(~uFV_Xc$AmK&8>SvGOq8mQE#yAndtEu>-Z z1mIIiK=Q#-Z|6~sytmFZZ~5OXvLRXj~>)~tuK_-(=G5j zLi{(LBnXfmTmsUA7cT3UJt4|gd>L8%MeFj;s%*sr4H4#v`PkZwuk2Jz(={gih~{{G z3)nOI>Wd|1WdOm!a1d{j!O!J;3!6qHSLQXKX{zU1?`KE83@MM+uU~__=cMB+jJ#p9 zgFm{t8d!N)YoV|BYo0O6$M9PXC(X_v2q;h`dI2dVO6N5+HQo!_(6I zBy1jn+7FcB$dv9!3~FzSPrpQ_HmwRXyW>4_b4GUgEN~Jyo+Ced=GA)VB$V}LG6_#H zharC$T3cS~wIzCC^?rN10bPf;9B@V17k1v;+-rfLuKigW;^*b>a^3gnmPKs`_(>fm zS%{0wEc-1`KvQ#-dzl-dQ>j~fiX}YH5u^M*(ebDE*0C&jpuuBnv^yuD`X~ycgI4__ z>;y|}+GPk57UX-Dm5I#k$~G8y-qpiMwK>R;AVJyp za&tqD54gAbxra?WqM3?p*M>zy z#Qo>b%CfY&&EwA@b@!L&M@cp+?Znr=ay!^X9vqe4uH@WbPUcGI$S~|=WI#;}CSdMI zsxv|0?|bAt1PJ?qmw!vvJ$Jq%==idGu#}QpF#r%|u>ge1y`K6olcv|lbeMV&U@-QV z`V55Y_gN2R3Iy){Jz^3>He*fy)bH`Z-psV&z2C*c+3nk*mQ(&@k}2?_sT`~U)zc

Myn7#b9>=OJ+uxVQ_1#S?{fB7g8dy+zg}yIN=Mvx1+w6mhLAXp z%4$SRUUd*=hnyhF#M4Yw&U4lBY4@jTOHR~m? z0mVcloD78P!~TA$zV_7^bZXjXb-0KsC^c~#1wINN;#j~L+1?>Qm{?v&ZA@u{NT|S^ z%#o(GsJ>X;C>UD)smV93U}<0DokZ|QI6x*5)*0>CYu-ll68u`i$$|mSnwB-NGfXRT|0_P@G?ZTs6*{&yLW7~qL z5eTI)5wYpuLRUW!)3|^+5g8=&bAL@3&1%j9dWWmt=auHw3|ey75?>Bih5dM-6Vv-L}qid0z&{00&VK z-4Dk{Tu+oYmAC&cgx#U?C9&8~q~VCef~pSTbZ=H_cRElj{Pw^7zpH^LT5nr z!AGDPn~Rfdnq0rSn?AAIa7HM?eMp}2J&%p1C>(4YI8zAw zC|uj!j+IV*Y7D(}NrtY7(=DL-5dN~i410C>;@ERT;*crxlV$6z!pX+ zUSK?_DF*d9C8H0?Hmqk|KKm6Wug?-Q4eSqE6uqMnG}rE(w5eVev#myRW-Gd)coL_` zl>k8yqxh}JQz;0}x{~PQCmB3LPAgE}w~a2DrDb?H{6d$=sInMhPzH92I+*B)&dgHE zRF?6bTB;KnCdgek0gw+xydI)rIoT~PB3HmwY@L>OQ&^I6N@_SE5G&Y*MS1U;D@e}O>MHMHpCYkD&-{14 zH++9SD(B3z6}Ks^XHCVG+m&uQ3x2T1oqGglfS=slG;d&J6mYQPq%-Bc+s>x?3649B z#DW_YL*DkaJ2N-!V4%2+H8Zc7au4z0#SdLKArGLRLqcFwMZo!aSs4opOqAE=yqx2? zuv2ZI&vNIQl8(xFRH169_#RS@ar#Hl09;9qiq7qho3%+D5NL~YVz zeFfCpUXruIR8!cS#o4TD{f4rc5D5bY{6UD`4|CwaO{7Ok$H;>HG!3;R_NSDTgSjR! z5N|{+S?*t^DpDH-nu~lQ0)xHC#ZIqk%3vzkic4dv&IVAo zsgI-PczgT10;f~mFfs#WbJYsHCX2rCDQ-oL4upt!a$&D+cu?`Urp~+4k~BTz*@-Ey z&EZ@JovMh!hB+bNIALyuRu`5wx*7TS_^8Uwn26sMH#E@qQvJfebJ)zav_5FLFka`T z4(7%i5p6OdJTjfdeZ+rHO*4{}Bl_v0mR4K&7oL3sAC$?;B4HElF+{dz+VNsG}wH#ZlIFY!rpq^$4F^}@2(KZs{7OBQoCE& z`Sym)`K}?Tb1(3!EBgE2{wZ#-81&?NE#~1qVgY({kBsqZ?}hoZDP5f!+VqXsx^&^| zpz~h#9K%*7Cxsdgz~YqDJB*CaY0kLm{&eL1!})b<<$a=#7nJ8mvjom-2kXm~T>eVU zet;ZcGT*YX0l30815-$}D)_VO<3>qIIMxiKO_bkVfCf#L>j``9w*5s3Eq#5)prqIv ztr#urJpUn#gSR;C7oAM=mv8yC_H$uljB(^*hCwYp4~)ne(?a}{nqA|vN1sCGE7y0< zbgFm*uctPuQ=z6y&AMhG2VEH%88g$$3Gr|~wlIpJS)*4TW02f35OMB^)dsR%n zx!v-zia4b&E@^H>G)S`apC`1sY%PQOhN*JmdoWBQPCD;O#kdVr|>kmXWn=QKO-Y0rK7Yy z4KcGH8<16jP-I6#W^J8jYTb;@SSJra5*X%(FFM{w$p{&#X^zefyX{S%|=ytW?7!8($`aN*!}9l2oyCAr2rA|)6)XoUPqb61y5c>G}$!UbdsC+k*dtnYot)jx8o=#fp zTm~o)1Os==WSocEl!Fs{EE&P>+HSmg-CS26dm;#&t_Y`49{QkKXeA&9Nu<&psm&1S?wfQF>72jMf zMa=c|Pc*%F(3uWY&}X|pHsjr!Tb~n31&rsJV&L=DsS}`SxC$onEB+ZOlNPV3P#>z6 zJ3S#X7uU#im{6Of2r{sPvK!o6f^~}I1^o7JJ%rw|Wyu%5$-c7RsU7Ia{kCR3vr-1z zSm*pulGwRiptRnfp|c=-I&pNfxhNb`~&#i@MjyR-|V)X(i8>b9nvQ;MwmkNkkfyw_AxZDOJTL_1h&{ z6Ixnj+G2Xzn!_VrKU!KUwybr8_V^r&D@rPv)r1d^faHh|aa-DaM_c^!n^ThtD5ZS= zNz3&1SuDSPgxulo;0B-bv|7#nOC`*p^CP)A(U#oJ*vWH`Y2X@jYDN@8gLu)-!%HFe zMcMx4-||nlC$5u|$)Y^!eFHpIR0g6MDeI_>g3j#!Tz1EQY|9G@E{P2!NF}->1)L@~ zWfOpy{zE^X|GHxBYekRxw^*%ZBR0T1YUszXDmd>>;iuypp^+3|vluQB~N zXN@{E46mk`4q0jlJ`4_9nh7_r4KZ|Bu9y&nf`i@#UPf)NSBid-0MB@sz$3FV0k*>m zsh7|iJCr`cpacz#GUsl62o~&Yx1~4FM#>JeAN-Q*%{OP{w)u;oi|)5p zA!?{8Q2R^501w2*A|Tv#Srsg7F?S%`6%-V_xr~v!VCUr*5YRQ$zL#cyBtARc#x|KA zGh5B-?b7D0=S?TsbEp%w7@{eEv1WUlsqvf>^b9GJ6tuA@xd`~wnUllgp~DFjD~j+) z@bE)pN~;vDDR~7rc?B+mfAlNjubXaz7>&=PWk_Zb9|@?Z*#E5ih0oS`jA&yc{L0dN ze?w-E9LwpVeOJ2}luy<}B)KU>IfUj>$$9+Z;wGc)^TpX`VH%z+5!X_kN}J zv-S@X@|B^i)K^?y{_`8nSMI!uo86mD8fqDru&ZcfZK>O*7AKx6UyV<6a&iEa!==#%9j#~AmfWv?bd{53f!W$U9KGJ1|c z&`*X*G@r=T!1R^CS66QzH+)~)Thj)Dm5wJ3jf~7fhTk|kLz26zoCRamsq0N8n*|r| z{ui2_JspU+fpy?pP;#AWgby56(yZ!Rt%27;Khv*7R~cCUeX~5|d8~8$l!t7=$#=r- z*eYeN(1?ZytGK7YN_w4gEm$VVg6iar&kTRR^VSTPS^p`6rE=zglTN`|oVL>uiQPyaB z>R3Zbe!&9|LH7_)kD2E{LMu|2?;MxOESb|wjCCHsuI{fXY_OK&TcNG0Qdx)2HlKWP z%To{TK*S7IbQ1pfzR!|_^H%b4Z=r622~s!wm1Et9Y3pcJ8gM(jmfZ}kJg>#;jyGxb zn2^b`4Rn&ncld*4DRV`ejWhygB%_M;1o4cG@{pqP%JXR^X8AxI@$uUH8B^tT5egK*h!>lxm`Z$FsJmx!DBOQLYX$%vvO=IWyyjV@<^P{yTrZHCpb zqYs|baIulmfB1ln@VkPJT|A703JZbT9Wv*~t8Tn3$EFS6rh6XtcxFkuv(r$I7+9X3 z+vdG!ZfsPE*%%7E7=Ts%UvXuKqKfBPVS0M>@p5I{IXfm*Idl>QJv;C|teeW{zB}lu zL!OI5@z?YOi`T*IE%g5IW^n17wY1Sdqvx8HEYOxGhf3h@-*KAd3CH<*z%_P?pa3Ow z=;3-Qg;BLi@5lNSg>YXC`rD}c7#mdzArGb{kc1XX*`#x6MY`Y*nyfb$p5TYGF;$?o z1h(`*cw_pR19Pupld#i0;5quvzS(S<+`sis%>TLfL(7F*j(bz_`R;s{pnxna6*ZOV zaou}}Hxl6_G=pW4y`O?j(Wc-@WfPYjUZMW0BW6*60??2%t z)MNo z7Z4BvRM|`fc)WLhU|k>(Z|)$i0`~7z{)dt@iH40X^CvT9zP>_ljBPpOjf~E?Rh zZadF@lZ6rd$PRLX^&h8IOxC+vczUX=gDffv%;n&qY6eocGX*`GK1!kpP$!fS)zDUIdd_oTqa(=jwO)X<)G ztEC`>M?1=M{t!t-cwM8l(<&6Oyh#)xjc;hT^^&FPubRQsxOc^#^~RyR@6~r%ioI81 zNb?YSrOGU^?QW$q78u3d^2v$Kkg)C0;$qO%ocJq)@0%G_&M!6U8gEERNfTb8FJGK` zc!I&wx0M+KWt3ShE=%p~>^ww-gwDnKHxpAy$l~JIUnKhQW)QeX^78V4FB(TF>c2Eg zf(*6!_5i)xrvjW&PA7C))ViT3;*PVcs1(fb%Ek(?i@Rb!r^44C+xqb-7we^?AE&XH zhchkpG6*Z!%8Im10uG~#r7Bg=cy+-)>~vUMV^m$&U^Q0tw@z3 zOSCSpp2=(LdLS?Qie9#WT@1PtmL<-MQ79n!A5wH8#*5W@^c4=AZ1kTQ zmi{3lAtlFL8e+~-S6{6&h9XjH63Bxv#@S0!)eQ|V+uetN{6i2KE4WHSU*A`iifE1n z*d&AB>e?1!lfZ6eXXz-?RT5cH%%@_x;g{GRRla#QLoI`F|BI}zjH;>&w?1@Br^Es2 zM!H)-x}}lsPU+?l0;0e{8l^)@x)e}ay1TnO&bQwy-uvVG$ry~i_t|UhwdR`7eCEWJ z{)b#S(r;LV@)lWK;86r(7pX04r@xSO%>Zc?%qH_XV&;>53@PVc-#sqQhdy(&cnBPi z$W84T$c%T%@Ngxfve_MdXe>y+F4N=og56tVO|er69YexbhP@nPPX$EOal>j1vs35& zD%}tBRUb!PT6xKsTzYdm)yApE1TTz5xo-oDI~A-dC!3Gj?8j=mncCf$|IVX_>ULa@(Keh6^FHw(3y~W9k*4BEB z;@a1@J@LP zsd~r7&D*&OlSbMHJ$` zAjT~sb}8)y`wqJh=*`EL_!vYG;HlpA?3liPGmRNdUVG)okzd{4weH-=x1Y+um80SX z`|J!2j%3>F1j7~&?X~VwopK<=OaG>8?Q1vEfg>Bm0fjPub7+U1U5UMY3A(8N9WSFvG>HjO*P1lkp)Wo5-N-D`9aRnK)f zygWP%oe#(qX3{@n}dzb)1g-BitGOa%=4u};2R@u5O3-O zNP;492ls`VyNVVTJ(<=|p4Uw`gjP)1)gCIdov-m zch*aZj%=s1g%N^icZ}5*VE=U7j*g~eykbP@QqYQfA-XejffQuwdkS}*C zx9L>}<{NC_zkTumvmgdZP_xIo)y&ZCr)7G=~L5$wY zUrFQx)lsxc29K)6XvRn38RdWWgIA4yfM`pveOf6t<>d6$rgcyp6|WO478?C#G0Byb z#y13W&`31d+-#;&^C-KDE0>1fiSi%t9~K1~I@dTgpKciKw1%=5DLisu@3_8Yuf#L6 zQtZa_DnY4CoMxt{UZuxq&+F_-88s4c`1UtkP3D<;~`MX3{5&nJtJd|ndM1)nD{A0u}v{$f&K|^{K zbA#-_cR+hHCA?)u=+hyIo;v^+#}&FtR=**W=|fEroQ97ff+6}v6a$IfpkI2zN%#7N zqB76;b*OAVfTeEPGn+|q)!iEr6B)bQK7Z<;j$N3Uc&a{Zhj2MIoA&kfc>(3LzXZj6 zr$fjnCN5WW^0aE;>3qS3YR`D+A7o@?0CK2g3QbIm*t$Fu13+`33pzxU&W* zbr&MNnm`CQE&!!FgGWwORtA*7^uaeOIh!(A>~U6n+{=7icuxcM?%RUrv*`E2{HUR) zr>Cv${&htSV4;`n_aP2*vuZ7Qc^}%|Px3!r9RbN);uJ!G1^$b9OOIng<$Coc9dzS{ zsNB(z%?~mJDzvXBjxn?FQQ_-8I@|V^p5y-DS09#I8igm;MuTjg;QA272u3lYBQ*S} z_pplW?F$ufJ$zQJ|MViaANd^x!7pCCaBjVr8ZS}i4oU7+ZlK*Y7LYS?JlfIXmxR-=$>_K<9ZiO?92wiIKqdKb@sh70(A`ay2xy z8v(Lwp!!3X=~A}%?Y@*IzY@iG4gl^-+Z7&XF1MU`83aJ0ly|Q|~y1xd) zP{eF)+LCn2=W1<#Hhpquj=}kY?H{b0{FJ7;AbY31B<9pKNc1>KF_&@H9O8FhN!pL7Y))I_6%{Dpy zyd9Inew^F8YeMa-oct5=aCbGF&I_=C6pd`0Bb%uT6^_WRR#x-@0SE{`rMaz#V~A4$5lE)^!^Vj; zN~MYYb^ovI@3r42ou(OgiHV6OwaZ&wcCT~8f?5}+T~0nLwg!ZOFoU;5@8*X#Ft!eY zyq7wbO;I_F4Vx;ePrH(&iHEXUT-SQ6vjaBKQrM00($n;Si2JdNQYq8!p7V-F~75>zsm-{QYti`I*hw^%j zhZpL;T3hoB*@6GVvNpiAAw59A9# z4K9cGL$m3eN+$Mp#KRm+%p!LebX<&GlG4{BeB}R@wQXfNL>cnnkdTn8#fRa6kd_wg zEyG7xIao^9Wc=QGglU9%u?{&OD${0oHC6$~&pCpU?Rn<_vY*N^8y@ zuEdHeokP07@7A|_lPnr?-#q{aTTMvkFg9i5%)%ZNBZvC_awsb-DxvLspyxI+vVYJB zC`4f51w02pCVwhQ@p@fahKK%m?ewPuR z#A|f8E=NS7bRxv#>N5b)tx9Q%(;?x+Hh<}eh@NQ(knp{d^5E}+G#a#u0yO5*P76TZ zj*gCQUuADUe%S=xLTrCu(00WCtPKnCJHO$2V~N+J#0T4CPNzPuCWE+(D_&G^# zSLwlq(Gq-v0~PY@yM08f0%Tm$vo(e(wxGfM9+G${)_ zrD|LH+j8%u&$OvfynxoTy1z$2+z|xFS$B!N7VN=KutdlTMBy^aqFS~`RyD)I>)X1_ zgPw!ZV&up0^O zd$Pu$*#1XBP9?>Ch5kW=tI}eZHQ^|&4tGJ__*L(;{eoB4(e%lc@K)i|g%^U_QjIw^ z@19>;D*0?>XW9Nk9CD^YG~oI{`ZOWW<)NQMjW-KvO5wu2W@((8&Fe4c10&XI_#RUX zArL72jl0z^B@z4cGw+5nrnY`N=)==MV2Ah_pvV|nVe}ZKJJbd3Wr%CU+5f%qNJ8IL z8-|rqK+J#&5sHKo*LSgK{HWqmhqysKlEL=hTFB7#K6HuY4uQwDE2i{eSMgZy6( zv6B0JYB}mnTYrE5NxG+u_O^_uTqfV8L#ENwYS`@t4NL=Xb6zVQEvcln!t)#vqLY@{ z1zET1pD%_ALQi*-E8eaKSVmd~(ct&&Z7-ME)KSZ<5K#Y9u$%|qwgJP6OyTC{d>V;E z^qQWWi=L3i``w}Rd6SbBzV%R0@-R%~a2}0^U>Jg)5hbzV<$MVr{oCe`Oh#O$&c%Ck zd}bB%&wS3s#U-bPuWR7m&xhu}LG2Nf)*J>uqem4%d^yvWw(k?~rj+OT3f7g8hA;3& z;O%`l;rq-nTZglha^8u(3HXR+ql~{CX!LdNWCpRpQ@4 zl2Kx#Xz1+h1Z-H1_LY)4O+nrb?}F!f~*PA_NC8@%lyAO%50C*870aTT~H6jEv|n8lCO;XH-po zx0t(K1oY6%*LE(ASYmcVIGcwHh(jXyglR(7#JbQU{|P-{r$>QR z&P#1wE^e#AhWav$#&6%cn#k>iB#6uV(plC}>9u+dItSJt9-AzRdp{~^V8C^N3RTSE z3*{ktul@1i;CTy)fG?4iU}~i766gZCQh4|PWGV3^kH~*k0R;r?Imk`BwAXAb6FLM?TbJU zR>S$hZR7E$o(F`2>d(uVNF>ma$4VHMb~KLf9)lxM#KnyOU#EtdhZnwMQ8;5x1W!Vf`FDezxy0>1qCM>@|A!d8Gmaz z3O)=)fMG~_yAuu-7oml*=9MzY^B5~D`A9i8ppwYc?H|I%%F zybDtK%EqLxt9#{eQ389YgtdHk+Dld~&r;XV`<6BI9RcMT4<8Rgla=EJw{R942P?pb z84CFA{>kO|@Qmj4yk;)@o!v&QmF-^1|?YODq;tm{90B6^RBogwbMdECj7 zfe|)ON%YCuTeN-umCA~WRt%U7G;(TL9Z z>G*=u*5Wse5xGnv#pTBBz`h=LS`ApU0|_|BGlya$czxR`Fx`B?aQ<`9YS$=-M?MP@uY_sreSW+$> zhBfx3SHNv2%5HbbZi&f!>(<2*tiPi}%}=pE>JIino;HO&qu$-ma=85c*jBoXWpAk9 zgD6GN%^Bp4+;VRz1MNGv$MM1?dMVA_ljF;x0{10*kae4f8>T{^$+-(v#B4j;`)urSFo&&4cazK+n5VR)dhhRJ3Dl+HqZ~?j?9MVhpmy_k zV_(>Qy}jptvRd8Q6&;3d>M}CGoQ3bH)*-HZQ9Q0*-@N4n3`5d_kNxbU?G(i`2UDsH*u z#O`mU%8EQ}XnB1V|F-_n(VQ0R|cOI9k5g01j_8K6f^Q#LUUfzh+U;XIb4ojC4Gwy%~ zfUeLjC6J4L@z)Zcq^vYkt?Z%~*VSaCeXDmMRD~hJ>-{ZS1(Wv$L~j8)Yjk zH*B83ZgToN6tJ{RnB?MZ9hu}3`C#1Upf2a9C_)ctZp+0z%fQC#cY#0vy1|?>%EYPr`udw%EpIdp zyz5Rlx=(x@C(IV`6r&v%?FsX3u<9QD0jbXje|f zk)@|Dmj9=H#sy1)(fH-Zg69--Izvoxeh;Yb*d?W{53_Z99I#bg-F7j*$8ku?Bwuo< z>y%~llX{~Gd%+Qr_8r}KuIC0ZF&LMJ3m9Mknag$wrY@23-3i*Q!KX=q8ct)XNAp$} z582Tcfa+R_@vNBp<-8!q79GkvKc|bR)faQgND|wo#~Hf=Gd0G(H-)Kt&#x%1y9{r; zjpp3SEeVgSchGkJkZUcEZxnoo0!}=2LF(tBhV%m zIoUpSC+3hFmHj5#mUkFvY?}zpZi%MPjE} zwVSyqW8uL2YwK((Z+vD9IXGJbS|S!7Ugks^!nBAlk_frrMCMw)4xx~-8@+GN zsjF)At=E4iL9mTltWAYqm{kvHG z(O1H}GKj0-A-!eE(|nK>-yiV|5(PHnwuZJx8c^?LmnR5tNU7c}CMQ_9Q97K zGjt=txG8Wf3WK-R=45_xaWP3NI_-y+%y>i=a<1tDXfp`+v80yt)`&UwA; z$Bl=IA%m=xFQMx*vAXrnGjGkyfe21b3Q|h3pv(D+rY5kwiT&ZZ*-2N2gn_|x88}v> z^WOv2tL?@O(-R}flW$(5Yv==XCQGH50WTmrS`1;-$)i^f|4piu^AOM)=>nAewuZB* zWF*zqL$G4Q^tVTQlOhy}hO_vC8ibS3HLTm(ngaTJ%=lgIV}FABn5{&b99-qRb#uDU zHiuM^L)=xC5r-gYHnzSuALMk;J-mv;-8To73x0CG7U?YrjF#*|Tb7YhPm7@z6!F*` zfD-dfVPPfk)LvBT*V&zf$}ReCQMm8qhTol@N0iU;VLd}}{7;}*^PfO5$ain7w%DEh zlM<_bi_c|CgmItA{d7!Lh04ghXW`yI6T)r3^Mw~+{AzKUhk&r7*1PPHt$iYFNn`e4 zCNyGY|$n@FW@{OIO_1tz!8+&_%oKVlwtnCA*qx~H$y3HK;vjJdY z!IuU&Z0z>no5VIs@Te`l#Kl?aY@e8T0IDWpHC0}omX^y77Czghr4PB+Sx6?7qNlqU zb5o8*_P59G_!)Pbb{_43Wx0d`pet5h=xMBB8Boym>qnsq6cK=r7p&q8&rW&csvxUu zwD6+FDn`+(w%-4~^HIjF3zB-!Vjju)PDaHnw@V3e(=0{AcOOF;+j;(*r=p&%uH0Qv z8*HWSmOck$2FPolDf3S_v9hr39@@Z;7XozmoWnJM>bvj$_6Oqcl?&f?t{d&^JfCk3 zGV3Z@R(|%#GsjbXgZ|Wve<*&sUy<5=Wxw8vC#=qprAHFgIf{;t(ACNF@eLz7C@-40 z?Q*YGc`G4X%$J6VF84b0BN_;+8(S&h&HM+D1{r_!krZ}_zRe#sGvtUbAOsNF zcTnOe32j_6ZSY3|Db~Xe@Wen#*cm7KM!wWNbf6>yVp}wGN(yj}$f`I!tUHLdOayP{ z>AG>`L-2j%;ZfC1aYC5jxri;z1jcH+;JR3m-T+MiSp)>Fz_Y{?zt0u|p2af6XGAYc zP=FyX*9OcLO&G3#|E0DAY-fv|dz67DwdNwOPuf1joN4155pB{sKUV&a z-Nd3?M$B!h<#;A>3H32!kNg5bB=W{F(Eya1BcA5IE7($6L03A?sUxZ-VJu^{{-vgHFL$cn&veXN>)T$@$Hx^g<$fg-m%MM%JK?{M3aq! zYc>bo-)OXYX+sKzjaojYNHL%T(i2Q&Bz9y_G+$%1AOw%93nLFZSdVe$1U-!;Ll74f zinB>B0)mefi@hF_T*+V@Anoe%w!$F+o0*|^SRTm_MsP+5`XCpPE*ln{muQATQd{-n z-_56WkV;w8jA!Qw~vU5BT_zk|rQN#dk!I;N zfR1*M-P9wv7+P%Vp)2oEgNJ0pn!LB#o#l7fXsJl-u~##u`3Qs6$j-OUvsAaa^;s*&TMGm#qsX^^E0m5^uL$19m1C>2oB;-^}FN0{@u1B z{t@vT+STVH$wu$C%BTKRgtz)MV5gYjHB0Vir(#Qd-lj^cvYYZif<)^ituJ|UMq>61 zmyr8qS<4#!WX51v!vfjq7P+v?DQ5Z{P%k%{qY#N=I&@l{EfBLGPumjQP(408d;NEa zq6-!RJ|Vmu3_Ok@5Wj|h)bajSZ|U&y$#Q91%v1FRn6G>5_W?I=L7z@q3DK*H?-9w) z{g1yhT;`ObAdQ&5H@Nl1c&msr8Sjhp(CUYCddDRO>1wv?*K^~AP!DnL6!=;PwUwR9j+$0zoJQ@kqaPsdp zEX52E9(nrz)zd}WwCoI9{%*C}5>xW88m55stOvD@m4Y*NTZLe5jo+krZqhy}c{$bc zBL>e)26knJ`_Q)*--q3(?g)#9?9byL(=6bwrSJYbNv&T=mhvpC8m^=~ffiB(@ za8Nhw?yl6uk<@#Kz^Kox0sv$a(!P{i6nVD#l0k(}3}z&X9vIh}2+!<-7vI-U?y4>p z$;hhJpoFt-u*!r_Q=8uW#n-G4PGxOt~S z%bS``dkGWbk;RnGj<}v@{tnm5GfbGXaEwz-B)u}S$isp+JxghtP2W{WO_DuYX%A2@o%orFDr6S5_D{A14XE{Eia*18Vt`Dy!vJ%Kv5r05o`- zW_VTv<*4yTW;1%4A!Yq`Y1tN~h-a>mz0#aCtIK{_u&(_6tpR32_YtPmf|#_prPOf4 zlAh_L8O&FEEJMGsOGrYhS`Ykxy4qwtJBVIy&FYf4xr- ze-z-#Xc^=T6M!7dJe=)n_ltZ1)BRAHL~4q-NN0z$Oz;uRTr-) zSZDR}f47ATt9RHsWh)Y>0vZ(bQsgwS8U`p@9u;ocl7Eab!#Y)lCbcyG^cV4!`#^G&}J@i$KU!-dek#o<{ucr6X=47`3mz{|e|g z1BmH+u)LPN)LAu~)QL=sfg`^d?D{7z6OYfg9A{Io%s`GxhP4VVJegExk#&eZBQici zv@l5*zdC-jAU1XIeq#*dCuuFi98NA0oEV{KI2<1$AJl-fA0m?It6d1>187piqjoz3 ziftCmVe5nR$lF=3Pf;!Q&GY1vd8V8q7M^+z`sW<3lNE%XfJljSI`*XwNfg2)tR9d1}x^Nlb zJ1y!>(&gnSJW+fkc&@@=?lcL^EJQPxXhHQY)0rf*Vw<{LaCmAxGroX8L=m?q{j*g# zGo}zK0G()IdXL&YZ3Q;B|20Pf{dRzX06;#kg@V`mUZpYS46#DxdH8wb`IT@ z>T+RM@U%NO)+7KQ9o**A^tapb|Ku+YcAn9;B5j_Uju*Fmd7-NnL(ew|kw8)ih*=DP zIp=~}BcJh`>Dd`wb@f@Kw4yMEw(QU4%{32a!(z4$O)ETyH&u?G`L)Uos0R7+MGBC4 ziDQTt@=LJ6{{%g!uYUa>X>M!p=+wkZv+FpoW>gn=5O*45lZmA~I4lNRHBS&dxjlmU z(n6oL{P48bIa+MI33^z$%KZ6O`5+R|Q?vBNHVZig1}>YLn@6QF8`iRQTlOVsIad7b z>>P33KHi$DFh)rn&jUR5bt>QaTgkebZadhEyKOID?8Vb_;riar8~~EhG(8>WO?`c5 z!`sbMK&b#UCme;I7bi2c3HTEXMc`?Qf6x66^n@xO7W47cBO9-LFU&4CtSZYZdm{t* zw>%5DJgF;~tQF8)_-=Xjo^2v`CtLh}y}!mp*3bH|UMu6Zj?#r%&$-Bp!UW3u`l%~VY}%`q zdox+tx66K?zdJ7JHy{2;jam%6%VfFW*J>F!h;>@}?y70DQOu#*Qgf7f8~N(QKP)oO zmTmf*>)z$;fP>KkM3kdlO4v`YdPLX|%>Fe7XDm(6yw0P<zHE7!-|_`y8{&?4z%>%@$t#?_aC#BJ?c8>vqp5ooTX}y9kt4oI zA-W)DGZvelx*2+2*?uRN#_dvimJoEm*5Ju>fWKh4`>ka zqD6PAT{L`HZ;6N{^7s6T;Ie8cE&b5wz9j_-a?j6qcqOYm(Zr(5K}bYgURfd(=mU$W zVTiJP1(R5cnR9^NTb?ENOb=Rcb-*?qYX|0@42YA{F=4~&Mh_~*4p9-}DajEDQf zV)BZOw>d0?7x(#H`+m>B{=kr*@QZjEJF9Kl&}U#tctnN7;|hwVk;PA~4DLY<+7tBw z2sQ#Q#Y46kwU&gg<_(aLRduokr~pZg*SCQEm2v}d@!QA3D6!vX)ffk11 z$#Bd(&r6|}4PA~UzoT0Y_AA1ePgqxrV{>CrC)CMGlc2|%gitN3srPc?V)u`!*+JCH zAyrjNEvI`}dy`pWqNt!g_|(^5uMVegC#eR#almfB8r@wPZ~Ge%{BiNH_}1EL)Y02Bzzu+=4PuixRf6G*_aev3n?N?Q zP|&3ZbMiu!`&ClHV%gFJTl?)u>)xb^Aa_oQ`Ok*~;-UD)$)u_i*iE~C&_VjAfYe_S z8mvl~SePlHZb%_@KX`lo2Fkn7vAH7BW-7Fvk1B?Rg`r?m#)+Fa25Vqo;i#LaZp~G~ zs^m#?zuflujD%^bRmjTEPfyF>AWC+q$hHm>zm{ai*DeP~m!_b8RWr2KJ&Ht`8!gVs zxypCF_Y2AucxQiDC{1zAWEc3N{+uo z{!>t3$d?+p8F3^E+vg^HjZfSA)v%(tSmF5xNW~D~^GBu@&09iPQd9RWzAijoN=jL| z(Sz@7|7#SK}zpVM{YU-U=S_D}#IfPN)G*-@x zbg5=e)tG10n28(pU+R4Hb!gcRjnqm^`(w7vmBmr0!6flH{pF6*uJADM=rv1NN%WJ* zZ0ptiOKWMrzABM@W@j~Ck? zhUPMfhkbY$m(?f-Q}N2L;7zTxEm0@t{xB*(GZ0RYSrjfew+AebUHR4|cg(>ZH#pC|PbNo>PhhT%s#7;z(#AXS(GlaRO zbze;;yYEVSk!1cRR@^%>@>>~TGD~F6ZTr1-t3-%HgT#WH*70t+suj*#zG7FH6_)W;{;&lM69!?oZ6KTd%u}IRyNw#K;8mVi#ke7D?~ zl&dedH3&S87r(@gBY>*-pWM{=UB08rvTsNCYE-)2t5l7-b}K%fD6UZzeGHHDG7<8Q zgfO53vIhSEQ7Se*9y6Ou9c7iq@hC$ULpZtoT2yZqA3U|gMBlnSwTuoM)krZ)sbsXs1< zSen9lz5jQ)u&=kkO{^M$0RDyipbL>`MvmaZDI~(e+!ANK)|JY@n;056M(TYElE>zt zThU})fDJ7Yi;%<7kSbF8Q*^)rHg0TfK3a*YzrPp`M91=LZhm|)f!&O5s0^bOt|TRh z9e0nk0x}hmKie!D7}lzp_qThreR2upDWhGFtrC=R{3Rcs7obt`eq5V)EDeYupuWF8j(Ypn;pot4 zj>K*@b-0x3 zXe7!ba@(`#oGIc5Cy#(*21(yxE(w}u>MR!ega?m=q9N$E{qT(wuN@okZaiHjeTx*r zfE-R36HMpQb~=J`RNs0&e+4^J{rVA5R{uK4Gy)l1M*Umx=c3#0?h{fxI<`v#p?gzO ztm|65Auo$Tri|2DE)k(MQnWm?4FX+is87#NdhGsk>@| zq!2%W+HMWLF>FCTk$`Nsmop4M*AhmyWvA@5}PiugedVI^EfkZdv!^s0=E@ z09g4ongmo6-RuG`@22C0`ifS!WzgeTwmj@cy#4VPI$e0)DBXVkGI7KgwU|@;G%ffe zNYMb3gx!_+3R9XFRk$Hk1Gq*AsM40sjS;O_?zJSMQA{Vordh* z%9Z^I@tbaq+uRNVnGo%sU%$BHmFxXz{_c^633P`@+^YL8e&`EJ2r^4iqX&bm2pL5L zL5N6Tm*BD8Z)J##Z)Br$;Vt39$%Ab|2AQRni*g6Xm#U?}pf`GGVtq=~#PP2PCJL8# z*GnqX&kW0ckbP~E8P<&rW=xYW3XC_E1>@j=;JY``BqY-;EvP}Oy|Q$AdijmPgs=BY zJ++}Zh!HVhW*im*Bl;v7^>zl(n@?JAkh(&<#^1LoG#0IK4L_sPK4n5)6gv3Ao_zk(HDp=HXC9dmYrl z$GKi3lD-oY~KtrzYCg+y&>^K1CxkYGv8P_pE9Xu$f-CAxAZ3|^4;?1jw_;-I^wN9tV5 zhPXGJKc2Rbf%*x@*x)pSVC@|Ej=ATKPn-1S+`)IYIA>|=#Ku!)qNFH|&--T8e1|+3 z;|2ear+FKegF*PyCqZJ~cH`uY7;f>J^!;=4GM#U0O~)eM`Ft-C{LQT)zgrYY>ugyE zGa*B~OB1C3kie49cjh)7?+q`!3E_A7@BdxOy=Gw)6?G(T9c`ocB{_cvcAlm|ZI?<* zT9S2df01efU$xBe?z)ZQXM;`<21Ltmn04m2jL^Yuc*%i}Qbt6~zn}1VH|3SYVbjJ& z8NSZ7fjIuZ+xE;y&$VD!y{4BEp|W67D#)0h{w>2p7b`>BSJ%1KPY%7Ptf3^2&<~f9 zf3rf*5(K*L4HpA=l;6d3fB$Zn`>@jRs17-n&6Yr>gn?yN>J?$Vo)o%$9tJ-i2V7fr}r# zdar&DkXbhpVR9yG@Ki=r%WI*Th9(-j3q_2(DcQV}`>cO$z9Y2R)y+~kzQ(b{>>eN@ z!+LD-^qZF$`9Kdf z{_}sY(Y5{^aU2eYAT>&`7Rh2UcJ1mo{(93Xua!=pDUvPo)tE`kY5tKo)iuo&4Y(#j zT(^(}i=@SV>xKPP?T6Q_{%heGC6imj`d(yG&>6?s+N$z8t~O7)EF^9@BkV}}Q1A^9 zrvxb1e6mUe8Io;j;Zwf)71{~XS3#9*9nTzTyC`Y9$Rf6;C#a@i({2BYh)R*_ul>>S zUi%+KY*Eft34(DGs;a6KiHE5kuk({uxU_xDsI3SSWpY7Cz_uWf=SWtS@$3>2q=FQz z!yyM^paD)%Botm>sBz+pj}ak!{vcLF2@-HTvIUX|07kn&xM?DtVFdcwL8pEkB7UZK zVQ4xWbvHrhy}#9xK9W*{)fq#;uO|+;NAB&hY= z^Vy4s$=|sabi61^Ahn73t(|xdMv_p%d+A^M?2;f@p0V!(DpW)U^#T!xv8oW9IrQG} zK;-)J^C$3BgG2XWKy%Q$r3ct4Hp~zRc(l6?%f&=Tdo7cv5{){bPcI8izPRDk;-y7A z_IbFNf3RO6a}gooAi$$PW|Q^DsPGY$kpDSt zhVVD*sR{~%R9@bz3$ON|W7X8TW+b#!d9laotjsxY3k_&UX(rnsq#qpxT^pCsC6f>K zo`vF}wf*sCnBURRd$$W57M`;VPP!hf403(^u@iPgN#v&7p8^B^zFBOCtqef9Wtz;l zh7kO`*i9zMBUd^)I_`qr6_z%iRR&(3<&IRrt`oeYw|9<2F8=&9NZA<0q}bo{-CFL9 zP4#ms}w{x^WElnW54qBy<9Q`_Oi1rFHu*LK)1SiSWR(Z_X7)x+uY-jyq%TXg%$ zGW|MdMIKcens}Z5myVBLUvF#Uv4aUEaxoK2LZ!fnIf$wuQ$^py#a&E)X{w4}%rIoP z1vI{x7(Z<#&N68$Yph?Rd7*UyB6-vN-P2=zH`@gE(a3D8&4{DzJ~`-qF31k`T5eeE z@&>Y2ncbA}+B){8s*%qrZBHVGs`Y-2QIw&t*!N;WeEv}3-%h1RR?h&R7mUNi)!5ZX z$dVO!AN*1Az(^M#T97FWs7prf0QxiH!*g<~pp8$ula))$EL)d{^OxP$R22SuM#H^s zB|BASdie*`h;%qCvi3R!MQxEFcuQ1ig0d3x2Y~Jw{v=@Fv13u-dH3E3Ag|WCw&c!g z9XuQ?`YVl%+XD|eHb%+~>H{BlI#sbL`p&8q4=Aw3zu8%t8z6n0q5GhaYUAkWw_5y* zbbMsyfT@UZ%QW<-Me#F@P&^EQxx~e0_dQ(`@&Lg7QIyVbtZR&rIwFigL!#s%RJP|8 zGP_aJI@9*&wCfFyz$&MU-PLJR97GBV3Q1Xt*p8J1e$*PNeJu!b}g>>QG25E&>eQk0UHFI7zho9UW{zr{l zS!hYf3SmUwv?D_UPFmtLP&bf)TsQLRo2JNVQN4H(4S|bgb!}H zMQwgQ{?FCyn>GL(S2wrwtfiZCp4pq*gI4KxAHLpm{K)HObiW>M;Ci!-E>c17L~=P; zy+IAqjuP|%OPIo+f)cdhjZMdeB5mba-o~jZAb9`$LCd1YkqR6W6p*ACX<^((D@}T> zero^uI#)$PLIP{~%B-!+jO4Pb`x_(SAZk&4Sw0&<$S+io5qCrJAQH^P2twbsq%k1_ z$>ofEGsa#4Ers>ynp`&sPzg%bTlI1Koeow-FZmrT5}=P5GRIJnFP`X{_-KDPwj4SA z$oCLfPB&r@#ZWA##)GzpF7Gn;Kc>0wMM>QrSD4IQ|BgRsftB)zs=r7~OKJdW9Qn8b z`h-O+3L;WEy@PdBs#yowMTk%csSaVRx8(BApR(xEq_q6x<&znFRaU*dERcMt4Q#*r ztqFy`+HWm6S5Z+7wuC7MJ-1&A?22Q`J10-dr%+JBwIbGZMMWdHc49%y{vAQoTH)dS zALQi~Pu4AlbVN{JI#Y^1*8g606nhj=UpDW!A?xrJ!cmm(rLx43GqC+ds}-FAw4Jx$ zae3@#bKrXw?xW`5^?N2(P7VWOvPNpv12?J6(a!hpv5ATwWR2?6t}o^PTo3ZiuFj@E z_|cuGUV#7pbBP0*%iXO$S}Culn&#%~$cnbP28XQB3EwR~EUv^6pVQOXaM@px6(hS# z*WfB}t1gz|^zw9liwE{wjLXBD%ai*pF);%1%f*^E--psDgXlpiC{S#@j=cEj+!ONS zHP*I^<5`L39^H|;^SDRVTcO-)nOwh5`l5Z#R!O{CrvX+9z>F+lpouPY6d-2+b)c^s zxKZhxGO{Ifc}>~t&DP&o``ur{lH0F$`PIaa%)P9vJZB@cr14VwH`UTb+hO%^kX zUemU7aUYbL24G~z##6QQ5B8pa?!ua`6npaLbxKRMWUI6G{54Z zu9Om#|J&gUxAvasOoJ_OZn_LlMUDh)0r;FfwS=$J&_>y^$5l8m7o>rXR+LZU@Y>A| zkOTG8V^5dxxzrJGk@xB1k^J=&AeHtvv?@n7?v;2i>nYeY%f7j&N z#OobK_cO3EL$wsJJg=@Wcj9#?-`~;7z@=H0<>Ak;565EHq;5LTfmWT{UMdG_sXKYm z=~}DvJD}~lNu3SvuS?XY=+G56sSAKoS}t*&bz@)O>=2VThsqL>hZY38e9#2|LAs^s z$C~A~C-}8ulq{1D7HaMr$?ORC+UHIV9NGkoq4HW4=pa2#Mkk5w^VguMZ8de5YiJ!Z zVovATzmTRea?CC3er#9*#s5M9)s#VzgPon2DKFTuD+ouDof%}I$_}3tdrTt^nhY-N zh(|*Vo5U#H4|DrY`R&&b;J4{IHsV4d8hP~2+xqgq1`|*}Kp_w|O*AFVHqs1@PFXx@ z8BLR(%=E$zG%)t|D2X_NS#{?`%{rF;bW_8wzl%Zl*$X@_hT`rYPN2~-QG)@JumotF zv^ayUan;TA{b~E%R-G<)M!UM}?l00;pkO|bKN+=Cpcd*W{A#WPD~8Pm9FAVPVu1d; zpe=P-r$#F^ib#O5}S=S?bn75_DB<93%(54%}Y`Zb`Ot$f%JZ!y{tw;)=eM`b{JVkIWsD zy$Tp{Ot>N_;poRfM5+)rjx$$tK32&y=ab(ci$kD;np09d>4DL)ZQ~Y)LYV%xy8P8% zMAGpae$9or-DvAe!=Y7PH$m;FFsXvu$^V-96tP)O1RC*uySc(s?ft)X7uY35*0mc} zWr_D%I{wzoPZ7`M7$=S);>^M6#9E0qvE*Y0CMj&62_xN9b~Kg2b;8&=O$?_gs3jAS zaK6{K_g*?+iI1$BQgAJc<8A9rsJ3O9YrvR+f1qBfPKE6}(7aM@YcP0OUjE4M;yYb$ z_)Ep^=(GWWg)~yS!f)qJ#!Paj zNAMVt)J(oNwd}*NpQ1XIQzpiH{v`l{v+cb9=0y#ra6yQ}hE!eS$)r@+e=sCupJdK| zFS1eYI^^03bO8!=X4E6Hu3DSm-7wMZiZpTW!R6IHYz5sve~xb5rjH>R&dh zGJ%S8UMc+9IGZhQkJd5lT(~UofpEH8{r5`yT+7??y0z7B{@!DWn${d;TVop|I&#Ifz=-wFx`2MVzv4|;S zd97=kH#ezQYpG%K&)CF92fTDDltR~Zy~=}?CajFJmZ)C@t*A0-MdiM}rqF=KUG;S= zi9L9T=Zj6)a0Fj}kT?dn@4+S7Je^9VXXnh!OwXfD$-9fSU$GzGlQN?ugcr@x>53D1 z$kwWbzA9;cS48PdpOXbt9VNc4qX2qfsAVw`JRBDH4fp5E1bFyp>h1_yJd6-rM$4%6 zY&Ns%XBArG5K;}KEL^+w88ZfvxIg{ns05>kYx_vhkNIWWnHN+4 zp-Ql=kP&#XOLx$j+Fo&Ah${5 zdDWs{-BH{9!Tn_+5VK;5%bRpz$$}{@BH#B1Aox6V-kpMEA#5O?; zruA&Ev}KQ-sH2#{?W?{>2CPz90NltK48RIP#R%>$M&CxkCeB2~K%v(mALYP33*J82 zRZ_-du#jIt!5|UQd903^t?~f_r?VK{0ELhk$cMbxfK?xnty<+nct*uWvQtPLEo6C| z_&+>f3z#S}3DOZ?Zb>G&2U34gysOgN{nKKUW0arU?02y}dVf`MuvmEq)GPc>zUYo* zin$%O5>WhHsJ@wQznK)fKbiwb7;V>oH4SNfiuHI>OwR60Q1kXvEOz4sG7rU{hpW_HK3jJf=mIQPV(kxjO3Gb)s&%Q*xsg# zH%sm)Gg{v9oK3~+>&@)oWWT>=p>NAS0&XSnD{9k}QpCe<|MJ;&Z%NALaX<5Y_#ZJ> zQ)W!2AGk8;0|u?HK92EDhF7WA5u?fR=I=yITy-5LL?l8WDtW4d=tStUM#EeF)wuFr zBss$Z29K4&qtl9!86SI3@=BreTOFCy^!Q{@l&5di@R&_wF5Z(w6_Fqc&?@6Vc&!3% z^S)fi{5bybo0i83xjwwp$0FxRXr}*epIaJz^3cMn=2Rf|78Qm06lH#0Csj_LP;+t^ zf|&D;qZD&H<9#^+bU}8~?xFg&fPtEAINn<~S3&M?dgbZFTk=C6*~FHwryn-gU){0o z;$(jk$;dwTeyo3-<+Y~Rh|1++g3psF(Q{S4d8`kEVoDvSKH&fRHeUPhQ16ZDEttTV zI5WwsjnPgoT93KHBhHEFXJ>{&CllkS)LLE&*ugtPdR6E#%__Gil^3Jg zc8UMHF>a{_A>H!S93awDO9CNRgSTUAFIcSnsyuFmepBvG((oT6HBr9ZK2LUcfzLGS zlnhZ!u4+Xb9O-p1Rt)K^J#6GpmmAHWsbdU@-|+uCRRi9*3B92_#!q6oMuQETFP#_< z@zzx6RWO{(m?`!g5Ny>&)wnAs-d4el;^s4MiBi`jm@()f>i&Pz3Lud~xmD7k|DK)y z)CS7`;}U-SCg0cpzhtw_(l^6+$U@Kg^ZlFZ{YYJo@J*W+EZJYDZZ%oyMuj&v*3~P2 z{-#)ieGBah7PkVcGu2r7eV2%-IrmC>ySJ$X{f5#k$_h!|q?D6fkUb+kTY{fo^0(xlM4 zu{}G7&pZ1VXdrMN!tG#D=zm|5;*iaD^XpBu%k4&JZ<0R=gkJtHkYq(j<)TA%{OPOu z;J2g!#MGb|P<#c3x*@-g$#aB?0gYlJ0&AGs?^mA+hdLCq`x^4D*01s!wd>~uJ+_%= zXb8_q1le=`RJ^{V8rWpY+YkhY7^oCg8om>~X6dsBmC(_)NE1M1WW4llYi~C(@4SE$ zYA7o!b0xz@Cdp4V_dAd_8Z6qaotM*Q=Hr>HjL};zYL*+1dn8rk^)!g2Bqb&Jg!2$% z*L=}7k^Xt(TSi>*Dl8c=8AA7i?>sLOIY>xILS)}LSt7u03h^}m92qY^+Y!R$ys;n= z%`6`xQi0I;jXDS_kkTnv{3-+O-&i`hE6v(^;=0|vEC`5n;W42Do~KNjBri^$comroi>{p3iW78@S{GeXrCekJZbPLEqkp%ojbl>OHkGhGk$`dQnZ>%@Vlp3 zmqr3C!8vbV{}uN2CS^!#yUVaj-|;F%$et|j@g#(Y%W^GB=S)^kPW(Xreopp!*2rO5 zI$PaT8!Jv^U;2jPv;A@08(ncBR?p7`5>hpD3gYiIMwDA@fIXb#`j{eMX zk_I~cof?g@)%!P3>NcMOlVO#fhW429#ySFgR!X{2K(0&eOZ9Scis9esZO{tWL zUwg27XAbS;G#HK3Uh>hWZ>BvFU`AeGFYFXDNoM`Y6_=FERAJFA)vHPg{|6g=YV28+ z2}^Bniq)1NA_`c^&_OK|E)ymT>>kMv0A8!_Rj91iuGY#5d2es2aWaZl&hT>(>B-=t zvda~g&pTkU9A>sQ*V}vPxs2?IPjhCD2e4-advHT`P)e!nuuJG2EA0W%nUXndWk>!e zmtz@LPw)zX?O6W7*7k+(yHu6$Z9dn)5EitR!NliT-p8*bP0+u&K65wC`7tviTqcNk zxaIR98~qnWQr|jnB8&JVrBhxqFi-s`W}DHJWg|l1f-r4T;!yn=3*~T?LNruA;0#)* zdc8JsQ2ma4{~2kr>y+m1GydTTW-^lkK59)NsyMCE+;8^s>4q}bQj>z-Mtz#=kq&yB4z*nJ^Zfe!+1fR⋙f zc}4GbkhJ!r5Lp`=8!M~Ln@zz7i*18UI{LN44mz#)QNc}_1L`kKUv65z11RX_dQ;ek zz|mOqq)psIknRc>Yh?wCTfE)OXQz?h%I|)u5M8|TRZoZC=GeK zp9FekN+}96o5<$E{@y(8TyASPxl3@?!DhYcJwYurEt{ejaUfjPf;eDU?OEc_feJORgUy*D#!_*c z!@zDV@EBFuTSil|f z@KKI4ShEb8UH1Z7ua|tz7cbVeITD?LNs5J>kGmhBYQ57Q7YTBna|oZOqyel~bOROF z{iki5-_6Y()^wc+1}T}-^Lc}*ddcebwnRVglS9X1C@Qujb4c)SnwxEblKb)TW~!i{ zn)SB_?@H@fDC8|Gx)Xkxl)V2QK z-?izf(Bgz)%ahBNNx6UFSc-$7mHKk zhSu9(g%-ln*pdOqSBZ_ASzdg}c)*mb>tESs7B~qP__nMFNA=ygqd$Z7Nz?axZn{mP z63|S?vvh2|7*EHcjY5LmAq!{^^loC>#OG>&^$_~bZKWe1Ot8$xS|w;f`hN0R*pkD+ zHhy?V=ZIajt)H_YjLWb;y4~$?5h(qPi~&J?gG5WC$QKIgKN8LtYvtuAx>2ujh>HJY zIGUxy?Ci>P%l`@%1WlIc;6En7Ma5umbOEvsqCcBU5vC+2?+``WZoT@sJDR5OItUW5;w-h!e$_ z^NGw-7uC1_0s2V%KGSI~;;+r*roHSjcueYXpjjLoH25=r=OG?zR1h+#tqxT;q~sWw zFnKS>F>~C}ic_2};_VhMr_ z;kysli2+)2<$mucNDgLOZ(|i{MbC~{zkkmqI9Rybjg)fVGx62s_%t@*ncomkjKVoQ zYu=&jb1313HNpmHVxPZRxwRTer=VmnvW98a8>RjE zHuLC7>3hS=jt#|;tScvXbCnoGR8;n_BQm>lbEv&-j#4?p7aNY>Cs1~@@ch=#BKH(b z#5l3mVyXHLZlW2!H%EOa>Ea&5{H~GVNwOR;Cljb!ZWN0& zr<-j${hgdeBj)t2q~;dccp{e@66(DFi}Q<=)ZNyArisg`(dEhy1`Fc`OMzm4a<<@) zLu>)m6~y$HLe+A=9H^$HbY7NxX;;_3euw=XEFeQ{HVFDoJ z$HvBX24s(Ka(`0$4K(!hE0UcULc->(hBWKU_FV$^fq z_I;skH)!b#V5*jsFjloJN#7gyzv3D`bWg*vPiT&G{*`)>4KvX4p^b3OO4OnXMw`Bp zK(IC&;^-@V$oV$AUYpbKoy7c-lv7Sov9(KV{sPAL9|o1Ho?Zo7Ca=M`X$IojC@7B_ zPj`gG+kj!;Hp&zOdGSkQpFNN8wbeMlP&QJCu@QnHS@~sNJg`3z#MltRWXPrgU+RKe zru$8CUshl>Ay%*o3z~|tjCPsyW*IXsIXWPV8`ZclTT@6A4maMT`?i@mSzJ!b8A-pL z7gR)5eeV0-wxzCaSmcQht?)Kn1zSF>PvPLhfUqJ*{4})qA~v=vDQRP=WO3G+kK#xF zVTzG%bL!==h$(3o+|PSQpBCv2m>z(^x3SRpJ&XYDWve~UR_*jdd~+GP>fM&Y9VXcW zh$^er`*E%&&=4*))N{%hywL5%wUEaJl^C^5Eb$ZB&o}Oj(TK^qU`1yVTSUL@)G264 z=c(0uv7wxpM0U=jO44FYuW>+}Q;=inybk&c@hVBee&6WQ$9?l`~QKtN4(Rw}&i`Mh- z$osN;D^i1bqK~eP`>L;Qr6I+btg^OP=9#5x^`tCN?$nr_LC9>lX~b+ z&N+^5ToCbm5D9wc@sx#xZWCttStzwTdRJ$!iwzLG@&ePV=0}NpY^XbiXI1pl50{=3 z)spwqyC8t{BVX@uNpX~y=&P!Ww|A|2Q=v=GwymIq-~^o~0dfg?u7IRKVJD9))*8Hq zkWQqKw_Nu5-6%31Yd+IBVtYmCSgd8RI`Hu9=<_LGV2Jqv^zf^x&?p;&bDEiUhkG+J zh(A+|?0v^!SXL$*Loqh`3rE?3Y8C>&D{UCdOuffiz^zv~{%50aX#|U1g^lat36c`7 z-is18?e{@+Sy$U9`)%(q=5$_AJx>rT#}Drxp=nmj5ETe8my-)3)$i~}MEHk1<$#k9 zLDszi56en7U!^%|vcFG2ymfSRRDyQPs+KX^Xx1D&H~UdUi3z%eEFOO%$9-9CQfKL7 zGgKT9zz{vJMFPeN?zJsoh~3?#OYeVz9>98*@Q04t*U`sNPEIcSD|ljNVq$74slpM+ zt2NEe78+rwg!I8in&X4@^-vL8gSM7pUbijV<{~u0&|lTP8_uGdTtG^AhL2Zf=%t35 zzh928V%OKKwa`j+@MTB#OIjKv|G2dMn$GuRu>6gfl-QjJk#^8C2xwB)DoxcYG*nT} zf$D+3v!SZ-rY)8PH>tu%EUki#I8r$oZlMf572h^PRVS;k;&%P5oTg$G|CoYXR9yV3 zNB7Cy-_FAC@>;?;__QteZDhl*b+HuH-n2jicJ}v7n(x{QjO;z;5j9W5hKwB?pW%HM zvWk<_ZB7prKc6BBtgo#lHC>(^b(wmfNR>3SIhx@z@Vj7`z;3>2FW_3p5}0Ik*LL># zISI)nmXOppK>jMM-c~iv)!AtOPg_p;yP>4v#MPPgGyJhU<)%GDy8Z2afN8!wK5vMB zyP;Sw)nqr{x^qB5vAxsoS=Zc9L2WuDM8chE1A|uOCT2x9VPG|` z3^rpoS)P_a2;W4Q5)qLm2B8d81{HG!;Gm;xRBoEvQPaJjFcBmaRzoRP`>H}HLtiTm ze(M1n4|()tLkhTn{et`5PYdMKsY}wid-cus1QRJU2#>ovGd^1UZzXBy9C!NTUkTEK zxa7i&o)@uLgFQDrkCL+?c?eu5+B|xYssu`=p3HCJXz^MP+ZBj|o37FTMukw@yY+x; zK`!^b6d=p(Ktq!QOdd52=d?}6R1m$UDIF11J1U~RV0;VEn<^F>`WIXx_hjDwNiax& zR+y=CZ}J+V1ot4$2!Msl3nqS7=gk`FL^@5+EU&)$5@cc+!4XKz4{H2%sJK}Ai<%il zTJt)cm}rZ-N5FLXNG?GsMSAk=1u~5kRqN@bXntl!)inr&arO!7#x8owdGc7zjZ=RGZ zkj9w+?=YB8myIZ5_>B5io8Xi_M7D!E@lYMqMX1G{s*OP}|aYi4FPYwPMQE@jvK!7d`FU18L`W)?7OrXSDt9>7~O z`($etKnCCg4JWyQG#8&ZQP~HyRG`5-WU|MuE-A3f!wF8F7}*l)F(NU_%ofCS*D6WA z=+{6_^JDG|`%L4{*E4hH4s+B%T%w*6A~=gpdJ&{7pX>Rwy>G&MXw-O&&NEWfz=hyr zI?d#>NL;VsI38#;T9GhRjfoyZ_j!tSf)Y{dRJOsi8Nhc{7`7L*o%yk_@RC>5{nx@J zQ1(Hw9CHy{O=?9dz zKKIX;EA~m|5b4sf>0=>r-&&^%`^P^#Wy1BkvztWCb;=YTuUfT^EW}e3B#ovoe3Jns zxF{I0(4rTgN{7D|%EZY57a8OCRoHAJH=_{PpwIFO1$OmC6<>27E7#yTaw_k7Sau;9 zQK32VApzU%Ni#=vabIPN(r>Pw>!1*8EQG;Hw?S;G-aZ9a3Ty!gU3suN0gsaWZE?H_ z8a~rPCPD>KOXjf-mW@aY9f|{5}P+}^OA2OBlVT^rmYm3}G0q9)QeA*8dpg~oD zFn~}}g(c{{oXTzVc0zmaT{l+BZ8SVfQ1x$3+7SqZKxk&^zeQf&PxKbwy~4zYZlqqn zwfvn7S0Y4DfgKr9F&Li+oatFTQd0?kq#|LN$?Uny6HfeSgVN7eh7dk+HG&-S%7WyR z^Ec=Z#}Q=z2PFQ$tr7G5Y@pcndxp2Zz$d=-3}`;aBaex`==-|t?9OYEp|);LYq!^a z_bc7t?7F)AYzF;etuCJ6CmI+eHID)T2~oI`nwnabUiYe{?Zf@qfRiW1EA#6VAG`*! z%cNi3DEVm)%@O89W=pIEqR~n@Nb}{u+8jz>ji~I}(u%2--@yBXYtv*q>+(JstP`0^ z+naCtD{0vVQ-6%xkm^`TOC-PMOoAU ze7{oX8D|x2a>TIA7lPnmIyJM*l%1T0x)dNi=57jc0?h@#aH@RDO>*O(t#=6pwn}~d zR9jn{X_c_e<)_9Gv1v1`A~+ukoZ|~Nj>g|!n2$=aZ^yQ7#I{dKevm@gA2nyvhk_{KN%@4g3|#91-1GlhNe=5vg*hRl??jIBP^N^?0Dgf0z#}}#Zyx|~;Q#;*4FLeY6aaw4KBMis0Q?22 zk(`tS0QU0D=`4uAo=zGDPUt~^3%-Y|6kyA1M)(dwyCC!nSug<34V_TKtaF< zAi?ht;9md&8Q`Tm@H>Dk0{Q>BS4Lp?pFD6LAQJrF{P27)A8GjirObb>l*)J9@a_X- zBt%u+SB@d(32$Xw_rjpnsb0h5(%3W&EEbhKN+Mxw^XqDQl|+)larx%&mUp@~)4AHs z-Ecr`#Umc(W*hdOttT4{d6x&O)C*Dkm@Ehz7PM)Uze)tV^0uD^NJ)4Wh|&3pAuv2q zybUQ1X>CnyZGULAT0AW@wORo9(E-M>{2CB*8eQK?F%FEU>6sD7Xj`EIZ~=ZYNL0j~ zw>*S^8j-M`Ij<+WP8qw|^lLgF$;VlT>g{>tk!IoMY{+|q=Ebf%nQ6bj>Qfi5ihblg z<2MR;4}?(;HeCh-is;Y8d;CKKk?~~k8f?h%C6JS-=%_^qWB|kfGrAah2p97LU-H<1 zQ7!PM8af}ZD5RyPX8MW-5BpC1@*i)>f189LWp+6NIQpH=HUa;lt~^9m=ip5!c*aA& zD;TVpn+;7AE0;EdJnyQZht+W-CLH+0{%HLbp`oc^p-Gk6*nX)12G>mQ_hcBblRMZ` z0;9^?AqPDTtI9^y@GB(LfF8q^Z2Rvh$VPyT5K)}CIQNnvY$H_s2SmwbCeN8VZdPx~oWmav4cmV` z$w`WZD*k3`OBmqyPWmwf?*d-!=J(AgNeeSqy=El_lltY;)6<#!^k!QR7+3a#JYD>Z zcCw`B7uRFb8e)6L<-gg_7unA}l+XHfiR!X=0ZN~QUh3?F3?7BNItx!_^1VK>@*6Y1 z8_C(zfcJw#fYbfMD{4ei!@`8^&x%5|rB%7MN3SSXnkJ~|hJ|5QXRHQI-0|`(`VISc zzOXyaRO^f9a2OW6xAxudVc#yIh5WsK0z^8&9#!M{Bmpo>4pOY(5d5irQ1{e$)b@Bf z|DC$rQJ4xn>lmsCd&hm9E5@?womDlAFLMOpJT8fgntG}nr}o_ZKLw*xdsv_$@%jm` z4@VQkM?l^Gj^k}{Ug*Rks9cHb zRz~oQSG^mZXEhc_Im*k+Uw>TIRhbgUz@xU0g1Ui=xa#Sjett z>&(8;2!qKKp7sQh{S7`k!M9f`|DoGB)LmoDV+I@K3--zzrf9YC$wN7)C-q7Lxe&~P^m~e3P zg?sO~Q62yMqo~lWMPtSUPU*zW@U^3j6?mqtf@VsBY4k0{u#GnA)eF5 zw0N!Q8niL&4OjSp&7ME;E3fvGP@b%>?8vKT_TGBNWTVdBrF#J2WZ#<%n{Y;LUmxx~ z-9CU%*I7R=I~QFJy^F|F;y^R|ZeRc!J1d4uh4dY6Xi5%xmHAX3E*MynuoToHaN#hbj0k})x>Q}T8OrJ}(=Ox}QBh5Qm}s{Z%04S^KhQ{5N!q)5 z=l~|#R}-XaRp@uNO~r`4X$QV22riYyNYq4>i2@dO>9A{{Dh=uWsz$vl%G+Gbj4{1# zN5NPZI1d61ZJe^M4lozBx84+rAhX6Y7w|*jTIJRa%LEB~?>ednH`b&)p1>wNIpDRn z026|)f$beostpBgdhYmrcl5=#25tbN-6S`U020zpm-nKg*CFq-IM z!<%7az}qKyk{i2zXRaX^vh)x^CdpLh*feoPGCY#j`=@Jc)l|TW-+y#kNOby><2O2JG7w5)*If@*r_Q90Jv|ljK3DpbJtxniS<|#!+lYvX zz4~4!mTJ~tuf+dv3>n~O9S;jS?OKH&DyBk{oM^~#(Qa+`6+h@QLcr%<$R&6Tc%4sa zv)(q(-{tdkO9?#>kxODZ02BA}cN*|I04hMwyS%_ldf=su#`KGIAVTBl>2x@mEyUcj zVZK-Fe(Gk?t6foW%jS2py=?f2O!QBny%p5a3BKXou(pX_jDCCd7n5BOa{UMp>G$8< z!2~`Q1GQru4~@BCXRw>-4;AQTO2d$YY$&7XR0iKI>{%E(tUfvAdt0B>1UHV zpYNVvn?O*P7ic5;b%kzgU3sPH(e9mY`EB;|9Pna!p3C>@ksP)ub!LI9q{ z;YB4mT}Q9|i9h{vS?GWL32tY-#%lU+^sRz|aDDlyt#GraUuLrV0z*vVV8Am@=w=-&EB41Yd=7sDGTsLRewn!0pf$*k82O+ioQLZ*!e(!&l7i`d&9Hz zGi#6Mi<>kADnhg>pTi){;aMt@nZq-5qNV|w0tULxMcVC06eIZut-q433UK_OK zTl;XLIZh8&)dP~fsY$?C|r)MYdw8Ki5G|_a2w{PK!v+c8*f#4 zce?&`Vh3D}hayzywcqN19v7gHr>DVJH8$N6Hgmm%n+7KJr0)9R@(qW8_p zJnWhhX3P|CE(b_U=%eO#TKT(jXfN)5d^p4l?IY)hL5GCzCzyqQnbKZe!^fqthvRyO z2oX~hq zYrR70pZnT)vD}zlDs;%a9gx4$eJYk~x$3s)5p`g;vApZL5+qYbdB*Qu$Gs*-v7SUy zLTe**97c8at0(3Mkj@s?P~(D8taQ-n2m=AQ(}0MM1VD&08Sa}m2)GNWCSGy{WpspS zZQpjMVBNr*-L42l;j80r=n08wa^GxBIE%-)kcLdq^zwWf!?&q@}K) z5P8%VBqquTzG;@ADQvJ=y?U6FOWeLW2PX(>`OLcf$JiOxn2+aorbN*ilT))juH1d` zWqWhQ!PQfL@#gE=y`$|ow|iX|2p{HPfIhKP_rd~JUmYs;jzybD?@{_&a;%ilex*s? z|Jcol#jxb|OMQ&2t}w{#gHpp~r^%hD?@2n$hKu_pL(TbJty_iS!JRn3LPTeL^y;-k z@*)SR&;()S5t>N!Oi+)#B17t(SDOcnU9cV@k@43m`ZYwiFy$-OWBGqf*()~|>9t!4 zBY~&!xjc<#q3Qmr9z)W=p847rHH9-Nn~=y~@IcV(7R{ZFqQpYr@COZPzzIzq)+$?m#<*x5B5xvEpnfZw5ca^w{H2`NgkuebOhluDi@5SqpIbq@hU5Pf(5d&qZIZlmpgFkny(8aaH&A=4Zi3DQ zfLm_T+5Y{*!;-Qn-&x5QX~9V;`J;nMrP}Jg!wor>TXd>8*MGZesH~6N;r-zF22EqY@`#7# z7XbAU?*|&kuO6)6RS_1_R^5fyr~nSUbU)Ak@<8Z|w31fSof!Z*EZ`LmdU8-Ocare` zum`m-U&3GynkB3hA)RN?*e!fOujlz+&cSooE8@}}C;x|!@TcZ#7Y=thTofMm|BBh&h{kNfah#hDEuyA@>8q1ko z_h@&E;!4vAUgQU{O{$z!Eip^Pdc=RMWW*IbqBYYdw-eA}BT5mWY&$C(x3>edn0#%S z&6H;+m-y(xx3uiA1I0N5Vr0OEcP0?%$6ctQmke5%6*V)9K^74oHs8NFbYZOq0&;%V z#Nx-Jige8oCXcQ(tF?9qWF4vbOg*#SR7rxiA?u8K0)+x!JMwD|n>|ob6%5wS_BL)h z*$2MvjauER;2?$XAsT^Z>$F| z&!>zOlz`CFV=s;6v9PEpwun^08{x<6fXK)jET8Q`9`)BssvefJ0EveHg|-rmn_Z0Y zu4Q1s-AJ5^i^~Tx@@i0r46eHph0iVXG%nsBiT;f-s2@597#foL#JX%>*zjavE7{b>&8-ul|fKtRp&Y>a5^fO-DU37dU8eZiK7QGyUD zggc=6*+)uAJT;k$y3z$S;||91=k;ZP@OW1OTQdPM4jDEaq_P}NwN5icvGu2|j=5_D zmnSB{z6XUy1g*MNi*``qL}euA23c1W!H>brx7fi_IA-dyKHj6l(WT=7^!Lcj%74FN z=}tPTc=<*MmpX1}&7X>mcH9MBz#jfI?PNACQDoel3%#)OSDtldD74E$LZIf>xa8qZ z$dLrh0!tkOS12*4KB_p$EIR6dzmBw#l&F4L$K;}i3*evcC^nJn1~|afAZv?fDnFIYblFF-`!nXDC;M=mu&2g2xx)pY-;*LU$Hccy#6VIQ%X*X8c zf%={7Im5GBFh7~y{i#22ULm^;ZtoP4HbR!tQOvgL+GFC{GEeJ_OOEqTy;18HpDh^( z4-a%g=5Z`9i6uOX(?^s7HoN-z+ZbHYf(;pJ=%ig~+c$72HVWR-Nc)i%ARk>S-eEW> zw)>4v#*HE#41K@!l8nB_LDa&+uYkBAWH-!0eDxr<*+HWo7zo5zMj~}dg|Bk%io?U+ z``8FIpfZ9l{;Q_BhPV$8?`}IKobM=gfZr>(kQoxqnWe9JURO?+hPJh}&EyZMs!1Zq z7Pi@~wJgui%XEE-szbMB51~g5_;G*T;UhHXZiFv6AjI`CmN33p_)}Z!!RSbSv+bI* ziHNurZoHx9FFziKwb09`HOZmMKe?<=yZH7h#_aIfK$! zSgmqWMI?ui5W_VI#0f~ZUh^kQ>zxr%;Kua)j8@26BkZ)l{X0N@LfR`9`kqSw z#E-PH&-JTiZ!wFOAo1s;Rc-F|o_H(2&zL$?Ghv|!cgdenrAYD3`kFmtr#6Xv5rgnp z&8J6+rU5JJ=ev)!jE*OVh3O|Ue-6zv{i-c>y{i)ZZnK!NY(`|W%Qk!=!EkHZZ~+u~ zx<$GhCr$|4yjk61TSQ29tH|36hx#7v$%kHS5AjC5^*ER<+-VIo3?DF3DU$zbQ`zMS zc3hquAw_&6B)C?%O@iw2^))`mv_DFYXqyz)=~a^#^XH!4*Qom55B0`he|uk%bhmm! zR)Q52xfe9db*dZITdYPtkQn?;fgA3Gh&FB#c2)L1@ zc*A<+SK1p2{#G{bYO-BKPv5Uv2YCdi+n9C^(|42R(A;Y`m6TUb7PdL1vKX#2pYR57 zvxs;pa#D>d#`=u``p7zDo)~@y#{9x2&nEp&xxVJvc=Je0O`d3%L1*DtF?^N%^xRJd zNjK8^=vln4RneET(>|}ixJ(Vro-QLcRcV?sx zmH+(+Iv?ipKIj&UE`Q&kce7Izd1WqN5w#Ro;VqQ8Z_V#;y#LGR38Tt&gxI?FJc$Y8 zEL^_eecpWSjPamqNH4)qbh|5rDJPY?mUw`ghzCqY+;{b^2wfXDz&Xflg#f$qA4&H! zxKyPW*;&6A1yd%9r2u*{pH2D2>kS$f;dRcLmG7&w&R{f&Bj`W=ZXQ0}`-drC&*Ntv zN#*+A>O+H3Zw!cH2?nEE9WOG*!cmE-Lo7ok`fCD-+z*${&8D)55?73Pv!`5L8}vRq z9sc=+ROgh0>a{*%qMrzOP(RMH9}2gb@Es|iH4A39aMFEU0XoiP4t(B-YgC^=o2!`ADvK9#f%`MqcT4A5Nyu($#k>cVgRE5b^+jj6Z^;tQqW`@T4$9#c9c5wAMx z+Gz}2wq?JjB?M7%4lNN;5p;%JJu#}8RQIFIW+|S*Z^tk+ZBYrr0uyXye=ovYZDhjw zbNRp=lz@Nuv1TJ;w~0HU1!fXuN5_VK7!LH=jYyktHH;+Z^4#nVs)lY0>$f|M<>Cg2 z6lg^lopg+8l@W$Za*DV5hb}$IwD|>71xq4}1du9YqM0EgcZh^*i!&_sk5ZHPi76p{ zx&#nfi{Pq0WF<*F7wvq6ABRF>IsqRP#flSYBu&#j{NJgXuxGBq)@FzgkhSJTb& z6G)^VjU=qBtf~Wie=gP2&C9>Wj3`8TrR~v2WOmDP(`g>}2QeH_V}^!3BI!1pCW6Su zjY&N;*7P@Aq8}IQ0Huf!c~&;YCC@M_|_pv2_x^ z8G$_IrUn_}cI@WG^{Fv^;uJ+l( zQSVeKDs4Wbky%<Jbr9$g@-ie7H!*02I#40X-V#nHTml8Kv9bz(23?|qSb<(cC*ptj!dqf zUX2m(+zZITnE(VQYP{CpPwBRpNfxRvJK2rwhHSD^49_1ee43x%(Xq|^;uNg(ciIXq z*%~k0RfnZuIh466nrv)i4UUmw8R}4;Wb7&rq++MUr$*A*?A_ae))lihG zJ~MN}MHqc?G^Lsj5eCYMwcGPftj}t*-I=SFlW-nuJt7UBjH}{1=zb;Wd9e0~Uf|PJ z>6Cz;!Q%7roZ|ESknpLa%-FR5gHtlQ&Rr~l)~@$+`!`_ zcN<>SAM#VIiE~>qcdN8GyGy(H<|u?+_ouV;_&HxoP<7-gT3N4l@oq9K^y8WY89W~H zA{{#qt-^MnfUBt|GZQ381B9^s8II29t-Wp-Dj&^GM_+5{Och);<8nGtRdEEwP zB|nfEJ*e)beh6tgTZ(5<)g3cce|ee4<1~LsJE73HW5%Az_juI2At&}zGpdGH=&F@o zm~z5I**xkP)MO(y`|AUuoO*>$o3(>DH*`!?HPEy9^;8 zv+cR5sBG?5A+j)JhIHAt{GRqhA^eZj>)+VIuM_JJ&l?nsr%Ns@Y>@(eKf)avpFI(iAC?apCG;0UgAB*2U?X-w87A|*2QQrDn7#Plzz z^1JK0oU!xTNXi5vaU+Nkh0ICSpcY0y|8zNxAxn)aeXhE$T}t3h@P~@9_~K8CE2AT3 zbq^L5W;bPEr69NFlkH!6cpQYHI@&TN8QWQ(ebiC%44#iwK9G}3MUu|U$LTd0-Ncit zAnsD*o5 zM>NK3qf5EI*riQc*}^zp`tgl!_7*KT}xP?IKQc;teBZdnY};mMfJE}!+d z6sj#pQupBhvaakO{8|~q)DkOVa+;B9c$WyWCT2;3M_?TK_~gvnw{2<<_uE{K%KWz? z?}VHSPpq)%l__*OEnC$OGjSAgLtB=HRfh}kGdoZ3MVQLrEebsy*LUCFRC}`aoqWVH zPEx|kmEkkN)9pb)m0m6us1S8J)QE?r+4bQCunIV|vZRnNA7RPEu=p@tCz{!GC*|?E?k+!V)Lt-wh!A#x+q$6KJ%R$RBql z;Y^igX>y$B`#hOs({9>u*gDn8U8ik^U-@8N$3*6W@dfTp5l1((wX_UH_V;TFw6?Wn zb1C&D83fpaA0yxI!hs{2*KYuS_KsW1`0%lUbyl=PQhsz_R{)rGNOAD%sS;)3Sex5h zf6yL|^q^7g>8b6k_r=x#JSzfk!;L9Etd|m*Q?9Dx&<%wq3 zzDl6Roc>nibhJ?EA>}aa6kPHutR;7>Y3w+1R??-y>~{PO0`JY~s-3U>`E!YbvF{Oi zcA|Jnl&Z;uc8AeMULi(YPIb2Mjhiq;tBk6-;||*L-5^!KYNTakuI#vX)z0SUfi$ue zJ&SIee91UF^n&tRR_CW5TT6Dpn}<>-GfB5hdzmWzPr?!h0sOAs=TI*cI=tUROBmOS zR9Q!gJuwxmi5@4B29$zs^O8zVOBl;oo9MBznkRLI^{fSMU9i)rOs}uij-;f+^0jyEE<1fyiwMhbZ%Xd4~#X=CMN ztKswW!LvQIV@a5a9yf{I!d}q9`wQc)hsM3)iPT1$y|CZ;H0^Pj&A_Xx?V)eq^lNS( z%(GL4d>-$8UVU>(Af%=t^xrINBa+)gZqeM0RoU}70Fx98ee&o{iQTKxKbb5TQd5h3 zBiIJkHhW%j*lF9F%$UbQF+J+7)`!hQuTBi0hZ<(lrC&Vv{|R?=dOeJqvRkL{dakt? zmhdt3F%wVG^Fktk&j+n@=3f=FZX>I`9d^gxth$}p&Yf#-Pb~V@?l~V3C|a+zIFIfy zsvx#cS}q{6sh+Mc`g&}RztPZSlVSVmpQ4p#;j|4LPkD&s-MC)|LDmN_iQfshEB*4U zj+j2K!8%apP}ZyW9?{itJM%qTKjFXo?QXr1tnce^__uU>@Ds^agULZ2 zM{$$QMu+KMZLLa?_r+K?1+UL;VC_(~UN@t&#~M?wS}mA4>1)hJ^LAge@KPl=V$-;z zlDMoZYoRaXNW(4L(rYU$5-ypn&!%~u+kV?+daF&ROIv{WQM+br6eS-2lO_{2AY7}jtkSL|991p%>3XJb+WSCTPR=~9c7$wRU`z9eC#GqM zw(@P!7_T;uB0RK!z(}ESAcP%-@pR+crdrur)Fp(E9`yZwL8B|7m;sY|-6q8J^V^Zn zKLLicT6pCI#q1Tj>80gOX;Pmu4=tGWL>PwKf*;E)N{YudyhgpR2f=d5Y>}z+0uCz; za;0*SF17NHE8wwYo_PRn1sc=dA$jx4USw1-;w%aq$eOtZS5!xqUtPn12z{yP@iDm@ zO)0CFs!8oLX_R9)K=T+)J?e$j#4LO)R2xwg3u?rl2(&pwGLi`F;R>~)E!PrigBDJ2 z&Bu~hr-Wh8u^*%w*xI5_`0duL3P1m3mrhb!R5BpKE)lyMF%BuMqu?tD|C-O_baFi- zK7I?fVr8WWyT~W|8g1R-aD(wZS}J_yaS#rRO6>GCfxUC3P{{qAPqd6BD&bEtyHZ{~ zXu?k#qb|)*p;v1$aHuw)%Vbtas|#L1_$hkhYp$uKG<}+QbP|Dw1`pH7IS=yNt#{0D zbU~TnVgEuo4cD_3r;9B~BfEj>H%M^Pj{99lM@p}vjgNS~%F@cpX)O!=0Yh&jM!~O2 zuRR!Q2C$ZcFKT2MRn}9hOzv=<$k8H37HT9e2=pe3L($Rat9H$Yel!d9&?f8t3o%M3Ocu86A z!(A7oakuU9K!vV3^AwpX)cPl;#o-jzQ~(^iRay}UKoWCjc6j=DezojzcP$&Y&NcM9 z>!A))IPZh@oK|NqQg|R=V_~tqG+TF zQqtul+?i!ktV~5ymKDw7IIN^Z(InJZcx@7-?2Lni-9!3!*>6ER-jC`%at`X~L?v3X z*{}Zoi)}H{(#taDcg6Mzp$U#SC!qLoIR*ccoJ46e%|HSg9&%Zrc{*V)8GUUxG46P6M5YCV(asZH4je=NjQ; z)*VSwKYNvtio?Qeb=g{mM=!b)o~AwZJSojP{sm-_{_!+#xdEG1-{*@V(0a$fOcuN4 z>_$rEwE!2_x_*2?Y6EfGnTRUd_wbyxtF(l%uUMCFk-saZV_j3jLA`-=?IJxN2?nio zyZzVKnkM+SM~>qf%gM+OLrd&l4-lI(sCgSAb2}awJZ}r@>U3E2^nNiep7ikO-I=I{ zU7m0DdA$CK4|VhSQ-TS1MaK?s=@0?h;?~oYR48mLEiG)$7+TU&Wu)JkJtu{)F{5Jf zQB@Wdp^nwzkNjPig8};Q!w3X&bGJ%xP3OM_IdT6RO&r^sf_se|cY1>&cG*%ufB!n4 zUZ<98RDa^-Rm|Wl{G5qlQveM3!O;@^sjER<4S%StOt44KY68BlyplyuANoy#j9xqo zh?bq4&gI5CURlwmmUVV9;e7L^DvJwPORL#iyQDMn;SDi?lCJc>q+;0P1%eV^e-5Kc zQH4&^=j7uy_r^lsofi5m_ujbuQ_CsSI-1Ig{AQcgCYP~xuVy>p`Hqj4{m4pp(WZ4icr>wV^om_Z%;AgwFDucT@X4buNe)4(CPDtO z1DLyljlLd!8CHvSYnDgyH2BNbNkN&6B{r+VOKo6?AiwL|XA3)~cy$Rwgt!_+G9z6l zV`IHU<+9FvIly>@L{3{38=$zoo+o}}MC7Jn`AUTQ$MNv)BiWFHZBrA*iEgL!!&fg( zZDFH|Aae)K zKqHHfgbijx4us!Js(#X7;^D^V;#NE{U3v-Pz(K3W@Rc?W1>fH%(bFlHB185fn(+MB zEiX)FUdedydnau6cmI*h)v?)jU73a(Ge%EGhdEGs zkRJ}l#nPBc>rP=w&&fyl`9SlbdMs$)8~&Kz%5pkEXH)BUJvKhsOi~EB9^Pcqf3jI26upJq zZ3y^qT}{~eIt{KZn@?pg10NS=0$wdt>eo51$73c{2E>=XHhPN6JhSGX&fG$hnyk=n zwSEXThXdC8_~I4zptp9wxKY!|tX2-k{xUe_6WTR_h%>n6mla_>Rc($Zk0p<#H?ABs-&@>ww+gLTt$A}zWOnSl zgADb==bF2&8twh$ZWS?#(Y`fVaX5|`7tAEK+Z zJ4{Ygx>inn_xk$MkmIL&Jr5s}KUuposj7XD+#D<o@I$QP@A5j#iv{xJsttPvZl!S^ zMZBmy^LaQZ$LHU_3$>#KHoba>nOUMymwZvLsaw`FV9V-(0o4K`3OIaBL+6m}Yt{Po zT6iHd%^K`N7--L&O3x(KdzrnuU2D0s{~ZK_-gT`J{8y}9T&6{c->L} zz5BFbo$cO_TaD$cj8)M6e28Kt+~2#+g}Es3rYT6F71Ml5ugzC*$W_O@t+e##dYent zQCBl;3%;bVRAZ4g zOq?>YgJv(&d&Up{$k(~{?^k<2@n^`DSi9-zNgRqv2~>rHT=ZtA)?0(8;0XTl1l?bF zzuw?pm|*lO%RY=-oT^Px#f>DGhPmfG>0`527*4ZE+8}#}QyMh0jb8-j@O`~pXM7) zE}rpj8sl2|Hyj{lI-|MvRDfBf(UHAS1Petfx*8LOiuZlWE5zb;R$Q+Tf0@c9|F*E~ z@Pz(3h^OaoHM`5|%wBDApLoPJuX?sz4WXoIZzpC?Bx0qoP zMgzSbACB_zzi}>8hw`+0xmqV9Mwlx`NE%f!IlDURb+`}wz+72wdw}FHjC)#((P&fH zouMJqI#!t|Op}dEw7dEwFqNpu?eWe_)R*>;Y!A8GJaO#RH0btJ$XLe3Y1J7?hxnsl zX!cUKfFF~7$BI;_TttageKA~CrcRf0Yii6Abb>3tc&ha;Ni^A$dqXF>nOc*XXFTlM z9JImh=Gg=1W#6Dwf-nh|w||3POpw!bvjNp&QrH;&+;A!IbVNxh) zGrFyYthTqcC$pzjZGp)MO zOe9kc(Tl?ni`lofBcIbGsi3`TfAk#>ydq&oR4+u5 zWaVkwqKG}*pRPAj%tE3e%%L^;saHXFiY^&w6k47|C50sykRqZK$3vwX#}KqX+q*!E z!-g!)A>FMm!WH!`594M1aySL*s_z=nb# zZAa|S63jb!M~BzoSDq@`M13$XCqj_m7pyMPM3OE;a)c=&hr4v)@6SjlqbrXS)0C2f zMrCtgM8Om2M_nCQ5Y~fR_ErhYegu_Di!n~Tw0c@Q=_E(CwgJ#X!yT%Hz%s0pQ}HG# zA`~mQ3*rSL`%dC>HlLj#;8;M_n}s`qTc#?eh4pqp#jGYyAIp#!Yct%>pH$Fywz>P# zh{Nq_2zK?v+m3;P!Hl0*#|BNJ|BRT0gqkbOWG0gapr*SeZhGu37i}l}i=0kjgsoYK zPmE-Y$4^DZfUC)v!;S5epyQimvlGTMY(|Gd_g*kDg+#UdZ4(wQM%H{hNzHE`gvG$% z>*iqp2^!fW0?!SUWPfP;8Oqp)uMZTAd%A67hJVO2G%Lb}5fEx}8-no>M7(nE-u|#c z(|SFsp7v}Qy4iLOM`mS}q~*L`5Qshv$h)Qqph1yZjm{c4<9^`vX+UP{9WO z*1am8O6Fu8z!>Je6Yp34fY0QvFsxcB_qt44?cKhc_@x}{7H+#=LLgKp`zevQ*z55#m z^V{E#Pd`n7LeRI*$+-5cJM0;Pos#rZ8A1p18eW$c*@qnUdng9UU%!z-;YYNh(zKE* z90K$F=;XM<;w0TcuKUfd%A+FF7^uj5TTib~KuzY|;_s&@g@4{Y^2F_YzK5`_g~8-rbcIpV#3V`Q;$UXH`nhM52>S zka|jRSHJT~{zV*pH%D;u_99GFU-FUfYljVr*PgKaOt`FTNS3h|(5Ev|E=>Yty8?94 za#SCsy=cAL7wfy4&6mMs&}sK?(4o`Q2}1Z| z-z^SxI2PICbo$vs9qe@{XqSZ~6yvkp-Apo%x<#je_A9p^d2AmdWY6X{wZJh1f6tl2JQa3@hqnQ(-5HOL zMQo`$56wThI}{Ja^wq1E+Mj?W%}`{ski(-^nT^pw*+S(Ys=5H0u6X6dGH^5Owpo#V z#)%=G!&mN7RNenI4SN#i*MMDno%~sSjN+bEbB4-Af`rQM{^pMW@;}t&cE;+2npi2v z#)SfZFQPx9+U0p!oteWA<*A`ATCV#{-f<{e{V~w|jN$?@lWPq*E(|t4T)uVsbo0%) z$?OTuddyC0if@3rFCH494Z9re^~DzU<{Q&*8;`rLezIhXAcN=!^LASIGs`MtKv(Q| z{SbVa_{1-&=6Od&vpKhKhnQFUv(p!k5+jhhdr+~R6W>qslBrH%s9b9zB&L^SYv1eS zFP;UAM07@0pfc$Qt1c|+#9~cp`c9YK3Ls_OW{!q<9Y)MP%N+cCJ1Xzryx#-E!g=(` z%llA(@UG=OXu^oKMT0ds&iq2_lC_XgilYTd{@MTzO{Hv+o9V75?rV!w50E@N6Zc5= zS-m9eU`I)>To!U%jcd83lE24k#A+rgdL*&O4vvoZy6ng1iSPM}hSS?_4PTK^l7L%J z_2=rkZgJ>2Ssf%dVla3yZ^^c0>{(VK(DS_ z0W}#_E-T72+1`TF1XihvA@oPwCU%?pE)c@OL4NCV-MA8i`u+u3Ji-y94NN(jf}@IaaqdJts7U)7>)KsB1xa(}wO6_xBtAvY7V> zUw8W^=U~Yo_4Xa1lHvKv;BGpKOXXqT3`%j{fU`Pw?#?lS|)iEey;ibDSLlP&-q8AEZ!GPH5U35UQF}Z z=O{32^OO0;kg_LVqVKK=jD`>8$ED!+6}08v01 z$?q5U5OiC7n)#02^=5y-i^UurQSeK0<2q7HE^+52<1RnV8^ybkSCHLZ;Yp%}1|r&S zAMrY&JHEC=$w!MvQ+YK7qP8!>`ZQE+> zQsz_M@`r zcgb|+*$)$(Y^qQBS%`}#+Vo{>pQ%ncI17JVxVf(I=&fbO!_|4D&*Ue*zpz)`4KeXo z#D!A!)sMI%0t0_w{ugy;;q&*FHUl7r#>Dvc+u{>hrgA4qzkIzH%KQg;LHB1l<(LB8 z0eZg7F+4#pyQlp^o+fg5{yV}CuSsUlk8hcSogYyf?aW_UE&uM8*RJD=t3$g~P3`1+ zmT!HoL~0MU61BVPfL*u{hF^`|l|w<3IbjkDPru>e$n><; zji_7&NMFw{1y5OV5o1_}ctIp}(DNk6^ja3#4@q=b?Y^S+TU32l^wMzQkI|=?FsbRR zJ#4UXhX;XE5nO?vW6diY=KuTeU2Fv5`U~Ih@mI}gv9QhPI4$vl*CNX`&l04RAj}l( zUKwojiQzkw-Db8Rs}|g%nqrCEYx=&?jei`hI!(~SSv=~5-}96*$R)?5ut!fYkjPAq z<|`9d3rj-}b8tpk!8UEQxYqNRlHde}y@t9zT7CEO5Hh5qGK@W5Ru)1P1=H0}SCU@! zOF-IbJVlM9nEYTr{b;=p!u|#9dAk3^UTohGVtv)4yC$wN{aW~S{{h|aGkEezY;o0# z=r7oi`b7dTor}d&MwhJlwru$Ol_B@StF60$*>BeF>9DJ_uie05fY}VWb`10s${hPF z`zI$@pn)n~0HHv*s$lZnK((BSBC1_93vfyF1`?CFx|LcLa%^UN>iBiw10f)o1rQ za&kbdxU#eF`$Otbm2mq%^;+%qBX9Adz?Sh7uigM-%ew-Wq3aIBp*l$rg11+D#Gkvw zNI$kd@4BDI9ao{#j?QGNe{Ah|0?yjSKTc#K9Y%zKKasiTt1+M1amX<#dBb(kxhg_N5C=4o1@!6VKA zO@yUD(RIj5QcNj;zG0x%Y2c)IyX&{pC@k3Y6;fA7UdT$J2nMoSdD0d%rrrmPGXNwl z+3%g?bNLydnsg~U93z7%lRSSg=96mNW@KY|_+a%`yD z^33xFo5JD%S;=(vG9JEqbt<US#raiLmD`qw)!Wr2 z4DtoUj|A1uYE7CZ4_XDXVJjwKYmCYmA|L9O(A4LqZNw`!9knZkSHkwKSa;s z?qt0Uv)2U>=*8QBf=RZw1DzcDJNnl&z<91z!@%bOfw}8G1l!>Cg;{^nv|ct4OexXT ziwGM!m4u-cI4dG^PM^XgpT_0byC+`4+%*L+ttkSzB-m=ZH{kGX)?QILzx;Aw*68i= za=fjtkvzuEd#@Ib{JdPGiE>ZRrX+@7tKQCFsu7`2nT*Tr>LN+zD5@gTRQYCK0_l47 zPHAaTIvAT-tI6@^;Fw0vu#+DJA&lw*?Us*)qv(C>$Xp(C7^!S_S7($?5)0}c z8XKKn)#^}C8lqgrHtw&ja?E1&rxUB_FmBe{ilR8b7FLQ$~s1~Kx66X&#yskMN z3l9@)3d|Y2pL?tG)g7;<*_CiAX5PDlDPnHN+aCwNK@rWW>c~SKTLMAXF-ota4JiGL zW7Zo`b@6hmmUEvgnoy_7(4*Fda+sIb@8xb;n>Ri^m`v%{lVI>)q|kupe!+K9!1mkL z)771eOEo($WbYtu>-EQ=LD-%l1~>`M1fm0V9Ll>B)L`R}uDBjToGbJ=&I%#FINU$@ zIlqzSEY}(=4u=a)++=D_6+rtOa`l&>(^2MEu2KOXE20-bNNiR1a}o3?-3($!8u`l- zoeb0Iin4~+8f6SeTgo=*t+#o|^lRu7aodjlJwjcsa%o8ijyiyfddnK)x@EES3yYjp zTHRHxrH?7q-5Mm^zH^1e+M^-P9MqQQ=}Q#`^DOzN%hh(S>{!e?b-o|kpaMNQ;o{;f zNU#PM_}(Hfvn^F{-wiFT+P5Ku=(6BU93iOp zAO#2BTp3f~;BXdaD$8fGniy9ei0~wcHt;k#4x&|NKDiQrg@;69Td%b+O(0<^4=_M9 zmG@AjFdBe9dM$U#b$GqJyr6u!B4RDziNkJ*B8@>|K~YvuPp{S4bfe|Flj(Fir=334 zwQiduCZN|ubK)XFO4nt7j~LZ&vR(}xUe>g2;+;h;kFL^c=Rs@O z^7L!h@nnN=WjQ>Nbg_P!6Tdjx-gL#8XtL1>5OQ;4#X9(I^t1Ed1zV-xQ%0oOG*dR4 z>%rfnGs%V1DTPd~tz_?ywItoLjJ&km@E_qO<0(}>?>D()1}3Vo|47|KItjvsV*bL<72nK~n z!9bAX%uv*HKO8C?X_bEG8SlcdR=vR>@Nu$hwfo!$irx3!-BdhVJoG!5XG4o9vrwA# zXJzx&>nlhq@cjDCuIq)P;HjWMiUkI?!Tj?N;7}o(XK9N_(52MrYGW1QXv(Vf)ir<& z5w^y-@nv#3hM1SzVQcI-+sE_OXnu&3m}e8LmtbL)bIbFx0G-opBJHlv37={*mEZEK zG5XRLeDik74auHkK-BU!* z{C*^EnQ0KJUwyS%|7)^0;kW^F^t`8vzy0ZA3kwORVD`*$>>PU^obE3|b?3v{)_gOO zA1DEkA`424)vY*j;6+qv^Kgo__P4v*uD=RLOF57pQbrz2CA}_A8_=eh9sMcw zaV`@mVNU4s-Qzwe_PgxfySLzVy_}?lHP5)S>$q9Fnr~jR^Y)(KqeZ8d zgTLUrqVsOBdfVDuFBc8a9PquVpZ>wVpn#*~_OdqUZfT>YHa{|J#EQ)B2ATWIKeL?(jOEqB_x?0 z5~!5TCxQV3fYd{!KXVQ4-!;qjD6X&nFOUdOM9&pX4Gl2eR;R%`YR@R7U;?w2o{+pyk=-p3$7C# zue~y!!g!fi zA!~1{>f*sejOYV>RNTmvlD{U5NStCJe5CAR-r>Xas zgc8}wV)V(zS?{v_zQJ*LFa}M zCk-Mt5imxD?sVPwo7eH1bT{j2qJN&me7c_;Cr3$lX05Gh+PKIHODUR>`NAxMG8I6~ z$U#r-a4>Vb4B29@Z;b!lp;%BbH?KF{p-lGj`^1&Ly1`8(&_mw>DaWwZMDONBW_InL zi&6xTcR+j0q%r?njw6cdbl$fga2jEyY^FcK0{{cfzQ=B9QZhRBqy%uX89X;j0rZ{O zltqb4m;y|laesm4MS4pYX~YQNxk)_kJ3yxMBXJaeXSSP6Ja&gMP`X$oBzlwBwmohS zbh}iD4)4092^xydX>k2$Q|vN&`n|5*=5ZKtm|(>zcck;|Xyx;!1zdjoVaThPs0yh^t{h)upT}vsPn8?RRLFdLDK zLqxEz%_lFD8^#C5ZD~`Yk?pS%Zu?Y0Qzy!x!I_V%ueh?G^B7a7Wk#aVciz&sqWABlFMPw`8jlzn+Yg&t{38(m)oD2UYTM!ayW zB}1Wth#EQA;Q=t5{+=Z1>PC&W>5o&#`x0;^uh4-i*BlL0-0&I#h*dt{{=V@KI{kyrp$_*Q*WuB7N~!Pt2kJGSX+57ua;v`ZA^#B7;rM1E zj3+`mQW&1(A8-89L4}rJPZBP9)^XCEFh6mjMV8b`O(oTTmJt5;zzZm8cc1ID9Au#3 z)j3JRz2WmvxjS_NT1HazJ>h&$LAoH+3nAdq#3>rxGT}4^hG<*vSq&kg7AP*W>2pvv zZmYymV-48<$+@sBmUwn`@&_W26hrFQDkQ(Fs*$Kml08CJ*5!>DSr(&Mxro;AX04Ni zz6uEntW;rrNbwn7JX!Y1Mzfh4XBpN+dblP#T?&pT?;`tmVRf?AM!)N93W?*%?9p1- z*Kg7!VY%VlX(waG8|jjZe_HR{u2-ZKX905_)R@;3##!l-BnFEAQ5#mcu#kaR%4r1Q zLN7_BRaF7G-U8QcSF3l&tliIxDFSb@HVnxZEg+y2Gf+JCL#{6t2rHuIA|@@vlB>o# zn>2-0R*>i2yV>ewCJzVF304FNHRxX5iXK4$BL^+RTj*z7uM0M#$JvhLiDyynAX~~c z?3x-~qluL`?Y0P9>!_m%<`LyYun8$aoPY#tLUm=8ul3QzBmWz9l!c}3UnM$^JM|2X zPv0x}?}^Wt$Xr`<2e*J7GF#yu zE8%zcaH_w0m?3K*zvn_VxFuh8!k6DBW-D$73W;epS$p)367tzMIAe-ad~{iX`0ub! zn4!VOXEBmI6d-sly*K=4ot31iy0L{8r^9Wb^{t`AI=edmoFGs*l!}joE$-y3%LTVh zQJ~YVhcc1%;_33rCzjz`9ti-8W@HTRxNBocW%qzA_j5KDEX8jbwU84ci704s$XM$Q<{h)}nJ4 zH3~G87f^N&v?wr^9|Vw_X;f)<84tdOL?*XNi;FCMUaoW6ZG%qB>h>;r^DB1mOV?D& zFSF#9Py{~}F2^nxXJ?fZZEfaHs{P)Y(P+)?!%~CCRek%MO+0N#bL6p*2>F{0592A% zi3~anam#h7vc7te4dkfXkcheK9T(@s>6fPsItE-^D6FqRVx}OQb@(+VzkeWHJ8taY zr+n&dQMKQ>x>tTzZ7H3#)#bth=afN%`CFA}d=lBUmpTSEj^k9H$$arNMz16dLCr2z z+ZvNDAMy2+7h{e2V%tk(3QbXdZ;u3TQ+KmZ%!!7Xly=?OiTQZDtKNKzo$tBMscM~t zagd((WH24qa5N$Q*8sextIT+k)v_&seNbh!F?QUQ`sq>-#5TVs;b&_%8Oo6uqu-`X z>329cimylMp1%6!N;rnyq!LTLX5BE1=0bFNW`pcth931`Zu?N?EyqX;=kA{lGL0x= z3trH<1m(7{S@1!yx{05i=dZz=w>ZEslck~_>@BScg2+L@+0~Ui1sdh%*9>v5@O?Xf zlw#ryt~O*?V^jz5@(du#3TOCjZXjZ`N^^JKZd)%C>iQ))vAS;}ID78xIyxnZskB&- z9TtTa)=<(s(6n;QHvkgBj-N+5$EOCpUYt=DGjKp|V9s10Mn!uKc9Gdn`Q;$VM( zDJ<2rPFz_;W)iY865=c$vV!JhqQIUa9O)2pdkjj=Lp%tCUG% zrFG415`@)-q;pU@&^9o+0#ho*I+h}T34ggGG0G+C;hQUIMw??nBY8H~47he=E}2YE zV-Lx=(`^65deh~})s;T0pQn>(lD~Ax8e{K(vp5PoK()~MMUcwe>0`qL~H&#pS`I7 z9+cuN$DX>}d{g36n!%uH?D>QGtq}|%>!wPT3c^1ZvVlw|j!{!)39*zLaH7Lm-Tm^& zORvs=ucfg*qnR>pQf1J+N~A!CHt;$zYE+&)VP1xGDdpTvz2&T5 zsr#e3lqyroEr-X~dO3PC8Y_Mk+{=cy!h`~P8z=YnpJO4M-dF-(|Ff0$sN-ozB zIItJHRewd_hT;EIs$Q5H{0nFTX>K6q z@G0C?9w8Yc$hanTT7V!tC4;j&Bf1%mb68r`pL8Sxon}i@3ymrwGzt+MjwLrTj~Y${ z{;jkmyUl)A7WC2pz_2NWw~4Di@;q|Bg@WN|qRPqU2|c7XW@O7Y<*YZK6P`4vCE7~W z9zuz@l=4V?hfzvln6^O_Z8Nrl6cR2r59o`o*=4&Wg&my10HXO*DaY#pfHH8s6zq_> zP_%pCsI4{p-*{4SL2fQ%3meLMdAe$xLeezwbpCZy8wVnw5M3P(+6<%w3c?ae-mmZB z@r}(?7}fe7x?um>Kp~{uFg-&Jl<%-8hnZ0HU@17VBE6<)cqlUma;Y^@LW>(}pxd=e z8D)}5RvnjuVrMAWC-V+}9J=|fQT| zt))g64l59%=8&`duCe&_b4w%jilA*icbj1kPq zlx5SyM@4R^EDTY;f434+@`qElqQg4{uMnC;#R6p9K~@6k>u?|qOFIdE0P@&_?P+$3 zAa@i|{0{;FRwK7AlsFX;+i+Of{a)@lmbXiIA{Y?KaMc)NZ;to6v)jIFY|2#e>HMo9 z4V*2D3{sf$=E;!$80eCoHs|ezH|xbC|la%D!xSa0RsGc&R*XR3IrV=r33R*0?P)Fy?Y}>t_9gc5qj!L>LW|;~4 z|2YSBjFzvy^q1PZ13il9Iv@Dk6okA%PSx@medHxIi<}JkOAAYx^}O|S4SVdY0z0J1 zk{Ij?l~UD0o+BA`RKj`6^*A+y=XV5|5>5kNzuQH>cX6s}J=BDz518(+L2?OX>-8oB zi*YYeN`gE>VR#fgVYLgeH+($Jg!PrgXHwhx3X52o5HX+@%VU0R#Bc=dwfgD^q~|YNtc# z!PfWfYOMVDnDh#1=ze`jVbbNQcY0$2v#AB&iN92N+%`=L>;2b5mJ8Z;z0M{=^oPBY zXUzRCD*h5S@1)Hp6vG%>ZvXD4YtVx|BLJAL7u5vW>j&SeRt6PhuQk*?Tv#nnUtz8- z!5%A}lT6fz(Xg1A$N{;<>P7x9&*A-eoMV+)op+#u* zXEN7h3egKjz*qplE$>;Us)$&n$`W zd7UB7N(3{g6=ooD%1+EBQR4n#DX;(>g;npjODIt_{VBy1g)PX)9!2ipAM?%OpvLch zNJ0kE$xC>=S}-ZxT3)wcM@yooGE|pMR0Km(Xu8j~t{~_R1;PRIAOX9l>loJ83}l9y zWt!D{F8pxan#{L9@x(J=0{>9})fSL3poixwef zW@cvOv-$d9we4$%|08mbp!PQO!|7;pyTL+5ZiUGhRK33V0-pcY!cIo4&}+7Q%uI8c zrHizGSUXfHk)KQzu)05)jm{9Pxh}}!b?sS%sw*PYtT*1x4?(ajH?pbZZ0h=2DxzPm zG(R{u_|U#RYHil{n!KzRxJDKCmx2X9)ur;eJ94QE>3i6xR^WR%cwIavwrh7NudGK% z*6lDKeQ&KVD;LqbWLy{6aCjJFghkTkvze+0sW+2cdV9Kh1+Guw)|*voy}2`3W3z6W zDG+XU+5A?gsOVYlemq3Lp0-`@(m*GxZL{;fq;KEyIUajaSs-6R0MM&fXtsJM`G4+o zx*-wrh*jLsAoU~Wn2z~w?OX##)x$=Bmn*de$i#xIAlUVL!jaxt%MgzQgC4wNjl1#TTpKEM0{ z)M`n4R3{tlOK&zmjd+Jw@8fi=qYBEVBxB#l*~j}_xv?!+4rN|nnTb+_YVCW%f+?H&-ot%82V7OtZP%(N5=AnkbQsg@UlRc!}afV zP$7DY&*U{?5ue9SpWowESO2F*g>Jp;=K*q4mtSXnjrB|XwBJ?he}-+i6e)Pgyx%8d zm#D)2O4^3Ia=-i3R$gXrq*P4#A6JZS^xCl)(gfW;GxgTp8r)o_3_1=*%>jH4CjNtPzV}Pcq%Tp&k||D zD%*U;ZF!-c$S|lqOo3W>+;+XUaRuNKF;|mCm$>yM(>k>dcT1O5Wz`nxIM;3NH#;6O z(WxM4y8{z{YrXAKxxC(f3WJwm(p1@_EQ+0fX~~1U+But?JEDcmcpSKS0!O&H32L0j zkYvdX5sFbWY6_b64b)4x6}GVzP%)=>pWT7<#18rNS7U3M`_E#_MWQfMGVKnp_osIn zwrh2VL4lTkZE>)#JG?*ppl}KJy|x0^UenpAV@wmMhyUQRxonj!RXI03M1-=t?6wsj zDN!R|j~qJgn^mWEzJ~-|e68ly11<#esz!n$qR=dbxwy@`0j3sXLC&x#)7H znUJDh6xOYyph)NN`B_o*xYHYSYK$hNCN~)fg%GcI)BhVevi*5v`vuW|yKzd0z7b|{ zk?HfO`ir;sn>iHx{X|-wDd*Gn*HLv4k4%MD@asp}W>g}#J1NZQOH<={yW&R^H)1E) zIIbrIj@`FUMR{6e9B6}GfqTo>JK{azexIXknXgakVXwcRd(5BUlp3jlP^h$s{o-r= z>lh|uc7``16;S6ORM@A+XF7d2$Femp%&p1eQA8F{@V_6?eoMPn9w+JfRHcF^Lay-& zSX;6!-j@o?-kObId-z4u`x;YS)GsQ`YggqQ_qHRo{A$HkNtLLmi}-GOMF!i z!$smKkjD7411m2=XA_R~TYXcryvHY--Y2vQsN`GS zJ5cppz@`?Ij4>zf?)0^p4BVJB)TM#74}t0)k8-JOM?7mDcJL=S}!=l+4Nvm9pG=-F+X)RiN+{T@v=p{=wN}c})wQ+r^4RGu(On+g$}CSK5B=P*RA4cd zGje)bq$ktGFD3Ta9TBFf6!)hcsKF5tcBR@QY-TdyhOmJt%NUkT^}aZ$jdr3APG07% zaEYnTU~w7hg?J+f#$4STZJN?aNs{N)GCO^J3D9&Z9$zAx$+AATY*z;A5nADJ*1p~* zIkAqHx_A7#Xff|^S*VOQSF_n)v3Nr!l-KQs!y=TJ=DT@rTWc$hu2L5#%a(R-(1nKL z$t+Hy*Z673p62}fpd;gaY{;%g9?qNv6%$4qo5>*Epc~kmAMzWSC_dK%0h{HnNG3&W z)bRkx4B%a|naA758~@nwkI#Z7*vwVL#D|+gvmU%5@Y|dZwj$j2*m(eg-~u~h5p|oy z`epU3(Bt_B!F2&G^)2oH003Jj8J?^`#9@w(v_^JHnzA$`%&u0YZS0w)BKk zXBHfarP}l|`-az>5`nve1O*-piW8K>@-oBZ61e5~5l;$i0Al#ql}0bT#Fq%cT+ zGdO`qhE{HJ1bbop+gz^Ho_Xah${O9ZzP$ezo{#bMUF zDc_Vp%D}5>9tqMb;Y%pA*w+^{m5xHWZCEy@y`#YjOEGEMhc$}OsO#C0r&xBIJE-?G zWy-plorkSKGm>Y^{2Z+csTj1NUWbeCi#B)d+(+|}F;i`0qbUy*L7zJyJH{A^Xv08K zkE#h=L`bfVV$fmG;>Kv;CPrKXz*ZduiX)|Fs!Fq7_#ZF?H2@hUvGq#Pk_5R*#rFR7 z2HjyZWp>+x0Zav)PZW9L7}U??4`F1iCMzU2{D_B&T-rWHe^b6apRoGef2q7 zC8T3l@ex1W7<31->sbL2QWl!|<~gMecUN!M`knSel=%LG@;q4&*0ItYUO7WZDnVd# z6k$plm=#DcRn9IJb$XR<)~k+(x9oNUa({HGs_Bk8rFyIE{(4=p<+OV5PD~*;3u9() zG}Ilq49v+=FuLy^XeO~!6!msI7J3%%l}HO&<>K7+YKq_y{Yu3FQv^-~-{eXDTb<^k zTya4)30xX28-m+hw)3?02#=VO?Z|6BrWj(|r6Q{#F()E1&MtT^_;{Ne7N!`x)*A-iSQ zEj2^Jm|N&7$?dNwf}(W8)gBM3S->+&!f}BkvBfov+?nm)Zi&KVPFZ0cgF$jBlvYdM4qG9Tai;j+|MzIWFjda z*use1Ji$4<8B*@XxIGum7w{1`rTPN=&uwAtTI>#XrD488`296%vZJNVneIRR}b>#EB|F?(nv(scQFM zUO@%BBx`VJV)7!JZjhq!;=)o0!_&^zrA?L=+1V7v57$|y>`lMf7;rEO zElDs&DU8IXAXG9s`F#v8p*%iLG>Eki{LSuN($HVD(U9Pa`#;sPc$+9)9t!24OxsDw z=aX%5*lpDxkD{F2HpB~y9nLBrvCQXU5%b`{c@oZomlnx~Tn2b4*2Ox5=FmWDuHThx zE=pHzzKgDyD_})X`Nrjjcsb?1;UcjKm7Gj}((0`bg2l zBFGhGmAWwN3K3gHSXxCqyXm6X0u|p?FPU<9;A`3^qq4iDtYOGVn<3X6kfpiOw&)Y3 zipBtM53zkY2xG5hiYPpQ|98rEEHA*} zu$VcU9Y}|(U><@=uTH?_{nLUlTx@?-?DTw>o9OcLL5l?)2uBT%AZ9=x<Py?V#hh$POcW=N|u2sD3nyw%KkK1*36Q5(=b+e^Z5&WNqbpnXuD)Iidp z-8HK7d8W^cP7#UD7&IZw91T9;=;#RG(yY%Dc2EguM;r@3YqDCVR><}y?`f4Cevkp> zWHQkUt{$7@32hr}D089O<;xz9iqP3iCsUM%>|9oNB^Bcbk$N9)n|DlAIM=I*GT3|^ zVM~p@yd=x~le!%{MV<7?m?P=Dn6qc;uM7-wQ+hG1PF~AU2 z)lO0I`0Tz8>oujIyWj5WLzd9sSFL=v8S}&M=J`0*MXLQV!HTR{);r!~(p=a-bOPEM zJb=1O9|$bZzqAqmIW&wCetG5~Xhqw^%*h2~&&M?eljr^jrl$}^p?9nYX(V`zHjWAF zD6rSowSDiVexEN@ydUMWUmV<~4HVsB|22o9g_N0&iZN!?qz|NGREnL52sVtT>oyza z$dY2vt8*S5OBWIFx>~qFr{u3*mnP8yl(&&Z{dGehA`z#>fBQI+V0x&t# zN3|q81-}F;q$0%1%>>pB{3Da{2WNPuG9yhRQz8U^%XOQ8hDK{@Ff}5crfE4D*y#v6LjiAIQiRED|_svl@uVud; z39}fn;e^U;3jq(o`~xrZBjC_*^8*hP8Y%e13WA`rvE+bfz;5&*Py(nX^G9H4-o0^HUUo7MFD9Z}{BFK^XW zm?V2+cK~@nl?Y9?-|TS3;KEvXh^Kn=f7!sb9a>2GJmA@c$*f3;6}idi6j6l2oF)=Q z*->FoV0IjBh)5}muoW3{oeW{~{I0DlmA<2-K4KXGN37sH;i8lF19;IRuS1^Bdf^2X zJ)nt{H}%E3f_U|!PKIBJ|A`0K z1I@%AjP8i0m5c)Vk{s+OANMz3Ps8~y7!U7@Fh+jPxIX?nqQhV~cvWja0Ix-;2`vg) zGBH~k0LqAdhXcnJU4_^{x55Iykh;`IzFz%wa^McY?p^}*sf2-G!}$EkPAD_LrJ5hR zafzQWLP5WT-Bm2``XW*Ejtt`bL!=2*mnA{?1Vkroi?JcH@LeS4{E~`@H8BOl7Qki&4VXF#$l5A;ZM{K)G~3yHMXTPZ65!4Dq!+dA*|_(%_4?3&aywo|9)(er8JCS%3Hif)tm+Vpy#8+p5GuL zfcUGqfUHejir|#;@C^p~WBKu_%Tesd^GnPv^};_PwjN$&%AI&KyB^sO7Y)QbRG{Cs zr0%VDuGp+p+dtX4w8E=`*nc^aG%K0diW6ux2(T;^wI$hC4xVg18GC`Z^~G}0VLLZQ zB|j|Um1%DKSZ+|f?EVLE;qa}chbV9_fpX&{dRbpwPuz1c1}|ubBoOJI6k+-AN^@jS zjEHJg0W`$K0|v2GsVfg~vJ4HTJx}MuO$1Az8Bugm%9^>rQ`<;*vBFi#ds&xcgKsGi zOp7#-KC9-6!er>Pn;e*$=>4v!E?gI3EOCp~f<`E8n0>6Bj_P+4qXiJI4-ZWjR>TC; zB6Un{Tud>zulQiFfyu1Ao~l3M(T}+bB!?`|`!UR|6b@(#g9p>cshFNIy!crWv3q0x zhMj2r@?Ud{UcSiisrOrcgV(J~Km{t;v+V~+z(h*1hSAxTeUbZ^nWG_sp92W|F|DD% zQFqoV(%6G5VVcZn5I8$SA-8UB3;$3H-DmwgNz&;ag{I;7*V97y0|(Y}GeQe%?jKfx zXFO+XpINE0&_i-(N)ILk04)TnzN9LXTcAfiK%A%Ab~7S}%jZ;GqFLtGZ*UaIDFpLT zL@IHxZ{NlgvR8h1S?-tm=xs`=BL&tu*7qW8gH*xR&y!t<>aIqagiH>n zHBexl?B~=J$fb56@b}lZhMy5I_L2I3Gxn>QcyQ;!jg z_l6?v5d<5$WFH)f1$ilov#u%CI#qaw|2}UxlrGh@gVgs9R3!lnk*;>H;VS-3R=m=x zbiS_7))FitZuhH|Q7c-deHQCN24fr7-T=N~UAW?jzj6C}m5kV+6nrk7sEF1>uq=uc z0upT=O?*kJXmT#!N^Rq0thRRGM(z|!9Xt}?_Fnu#QW|Oo^-u?VQF*wI>?qlk4mvv4 zlI1=$gwN;7`+Cr*Vk|%WztLKW{o9X7isl!ASLnodW^?J>PE^iHsVya(YBY5iaHF7A zdLV&X!T0KvItUF0q@`+g!Y0obA60_3Fl6kRu%{6ci$?&jkYbgLqo}FW1IXWC$Q}Bp zZ0*@rQUqwS1&eZ^Q-Cy8uuAYRQ@MK%(869w2hLK>mMnZ+7}%piTA9tUYT8*m9$Sk6 zX>3H1&{{S0xgE2mq8>E}xAY6_l5j2dsybF3Dt$HzIo;lts~Z+FNAl;Ok(x}|btCEQ zjUH1AaJ6S;SH@R8yKEYE8w@07GI;!!b1B^%+!Bq$Oii|1ZqD~7N8e;P21E)s`v&Qo zU9YK&XPZ%y%y%dF^!S9PoII;zNPIf}(X&xyCS@}3z_Wq0}-2Gkkx;>jB zPZ`_7v1c~L`<8>omsT(xTmu><2BLTJ8qYeV(X9)=B%rO%f@f2hbxE$ukBM~eHuqh+ zPT(uATVkO|Fr-4+UEf=~K?@4_&|P$-V4GFrVy1B+kN|I>-*1@S%+ zNuKN^O!GBbUIl}EP2IO=86H>(CDR@+gV?HO<1v2k{oLhC@|k?4c|2$&o2$5j4trvE zEpYq8QJ6e_ryHw`8Jxo^RrL9hDGXYz?yf#+VU6pqGP2#RMO1v$ zMw%+zHP%nbe?MJq=eRI5n`f)0i;^&>Mq&>gCUl91Dqv$?>$ToBQZ6rYW_GJ{+3zjA z{{3jN_7>2edWxRXq3nDkEhJP%~kE%%_{>bwg5ppmyZ<96r6 zE!*A`DhFid2)E&=ZV4JiW(ojcY~bPw-hGLFqeUzw31p&CLS|L_b~5dJ*Wcy7xi2&z4|4e|6wtBkP! z;Hc91dNxX+62h(!vLCttF+N>e34+EXD2UMen^GF3-;E@EuF^{I&hr@#o~07f&#WNf zS9PnZdN<~IoXi2NE0nI3>9%RrTk6fB|BVpy#w&$|hen=JUT`?^^F-^d63@95u2RnOmu*>~Fd^TT z;BY|YI^h7QAR)38x75|T5Jsp5-hl_r&UbUTmblz#aTeR9;1@RP!;r}u@O1DvO(~&f zoa9hN`$EYW3&g5Sek;)Y!AJ(u*p?z8TW&jl{_3uQ&PL4ao4|5v;2~h_&&x#t+|-$jFEQp2hX)g^C?DtTf$4yw48-f@HHP}!ZX{ves5RO43qJ54&9IfB(PRZPd1Qmi*nAjg%bk{-pyDH-3n-!L+PLJCuyg!uNXW zE*%jB3`i*N;`cxt|3|D?u-E}eX47_PBH@C|0B_9y+yTAzPu{ZiAkA4*3z-za3M!}h z^VeI?%?tRPs_Sw;*0Fpx&sKVUGJ_TfuA?(|TYvO{m-d%rX`jmLhl%%4js=&dPZ~4P z+XGeo6f(H|CJLoqWHl!8(T}Gn=Wq#O&z0X@dlkzxfj#R_@C-w8nH{#9ZHjqD%P`kX z&UYZ8e7Pkcw;@y(fPz@rFrE&>QOC+8{4Iw|;E=ynzO8mG`@4y{6#a1*@TCy9Ad6hsB*B!Cis|2+p!V2%g{; zG)QoFcMt9sf@g7e*mr;bSM@%;`9wicckbMo>F(3r=OF0fj{8?(;sk*(3Qg<|7uF5K z^tpa?ejYdgi#5RirR(JJr*i3pBesOJdF(4x9-=yr)s}x=m_|2k@#|=)@CdM4?`6V5 zd8JMxSxJuBZ_22n_fmFeA*0JFuia0S1(f6{x%>P%f)$s%X=DhcX9MU-2fsTw6<5;l zdVOY=TBvrg1N;~tB$JRrd`3);+9LG~4AK;HKsE4-6L+Eflz$MU)E*%viV3AWe*Bt% z97kWSNHz0qsat<1(tf3<`$FkE-$u{pOqfUNaGh{las-(E+KxNRrkNrotOA)ABjmU3 znz5-y&f8NU(M(=|v`MjxY>er^Rb_mf)?vNZ{?%imfk`$vx0?P<@0)P+px)&+jgPgT-^&2NnO*2*eKSVGXZ&5aDZbA8EF3sv=$E$D|k( zgh*VY_#a$auF;P~>1=-CGRm9&w?>{wv(a4H(eZ5TxES9TzF7I8+I%;xC0airqY85d z&^GzA-k5`u8L+t&pT8}{)r9o(dt)QVVx?o0AN#$s4dnu? ze1$noFY&B=W_P(yvv%1gjp_>d(?P?gU4GXW<3_&7#5mMM-R8|z_yh>pWkG2doOYq0 zXvB|^tg^`i&xoah?;;V*Ku&w+KWPzlbq|FrJZanN)EynSzI_QdT;Z>@*Hj&vsCSnh zPv>=;jY)GKxHf{Rr2bs8eeG88nYEZsASANB&Uq4+>`^3n-p`JcJbpQ^f`O)0i1eyz z@D}`L+|TJ?=dH!ANF=zCi{w+w%^mE}Ds@gHqvP+|L_UkgA1W;plWO-!y_@qoppEh~ zVS(hI;l4_VTW&uWd;7bBXsiDgSOMAO-uLL=zkkAfeXlR!cBfq-J0i6Ku4m9S25p^` zFjo7Dh6P3hAy46Dwc6#Vjfj)17O%wxC+D1n%HyWThvTNkSby)+wHE%_^(H&R8Vn&p zj1WdC8OI9DVnj{WVD_ZZQf2bM2?_AQ?*!I35Se6tut7RCHFb78^-yd*aoQk5!}}iU zuc65z%1d#!y9~TXjJJX3yGbKNwu?Fv&qw^;e;%%;7R_+3{uniR&F^3&r=*;m_&+0Q zv7)e|bb8(QCQ_0Q>Nj?OV|M)3-(s**elg}bl zk^OU+Ok*4&E5mNn$yI|OWHV8uI}8%W{u&v%vtYXDjbK)rkDP~0Bw^B9_B5T{+g8HU zfOfo~eD*}%fS37?DwaREkh-RmTk-?u3gL+FL`A!f>3sehV$7eWiQEI!5OlPJy{}7? z88!K2yy06*L76LsFqT$?U}9+}rML(>=V3xbjMXg=Pj!s)09NbQj`|2X_uNTu<=LkPf zh4L5iosT@WX_$5%N6b_jfU>Vnc`YMe52f=vdX-Ez!xkz!J3kr7-48#i;4hPuwQh)f1I~Sv~+7jX?bP z<%$iA!G<0j`uCsCIxi?{fb@eZBcmZ9EX0rY1MYAfeX#t2Fs_tV2sjBp(ek2B%r6Q` z)4Mz4w|&I&UIrrc&~?1|zlHLNdCg>OX6B0E0@Qx-BNk{wrHnNoNLVHIhF%(5Q7EVh z6%x$Kxm1XW?zuVsd+EE;uaX|wa_PgrXB{6!@;F&n9FG$?QcN|c`e-K;g!)-MJNJm(csU~%{DjKZ zV!Ea(6NDa$j7+E;PP&S#eU5r>fohJqlg99Np5X6}^&>CgZ_}J+q+A6q)b98A|II#0 z#yvsQUyKaQnE2ClDN0hR~odn=7WkCix#dtwhL?f zfRo~VC>=fKd;MqD09eb3x2MhWkw=GG=jcjsnW%Nhba5ks46XOFB{gGb@lw0!(&Jb0 zZPD9i&nmBfk#0Gf19e%Ql0$a78nM=K5@`2c@_d1Ku?h*NONLh!aUA8ggg4gIL*S2E z-bovB%&6u&v1e$;ys25Sk^%5KyaYq+V7iqlku@`93zi3N_|pc2xpbK4K0ZZ;TF|Co z$m%QMko;UMb+EIFWZa(+Kez-o91;IR#KZJA2K8C)%dQx-_5F4HmNqvTZ}jJ$1gn$4B}o8cgwJgkjNCdRa0(c|fyew7vdH z@*w)+jEv;ousaD=i-COSWr8w?5~J1a0PpMH$MqNeT##_c@bjoDa!t@@hdYn1sdz4B zriH{FA=ED&Io|sL!q?7B)CgbS-D$T%iR;PR;g;i&kFTMl2yaZzf5l06W?g9oHE~^? zosmgu6CR~yW07JcR!>EsnE6$c2F3h+qp2AAV&&~K>ToMZ{MMz39$C!NB7!=WNmbL* z2^>)4g1m-93?md| ziPEg%;DGQtLp>`kg0}Q?=!yMM%B2sPK+Cq=F5TTU|L}~Lh;&l zo#F=EWajymq^Qqm(#Y>--%BsPbgFYFYrRLb`gVUWD?Gf|>CLDHWuNIhIsJnwyw%PE zzpMxcc8*`_m2K;5e0%W)`fs#ANfg^MLY*&(m3tDGe(kN-Nzc@HZ%u_t!_zGnrDI(a zq-a0go|x!Aa7%@;S?9}7**X=^%yemH3#W#Mhoc1r+z}xBO}4SeokX zM^mdo<;wyUCCW@=V~ufZDJ71~d5E(PACXR`>}YmoGEUPC^KdBIWeN@>j2{xWZ|MJ_3dYz@s{gU8u(+e%%*?>a^w1y zU(YbCKP&}&-@dcgmL{2YeO_7^ZsVHI2p{^Ax0LwUmN~Xdke|ey78Ee)DzBw?7lVA- zNoL4C9uQZdJb|nA?u5e*o6Sf%-uB_VHD7*HH|Lv?%DCzTF)_q@AbH;-vBUkHuJH{k z9Y`i501GJ|Upg%MYB=DUHre?uZPi&!tTUhZu#E=eLVgMf@}D*;;CE~gfQx-dxX>*qZhSS1-;BWDqob}M2vFfDNTTcCD5RW27Jg-! zN9IaZ=IAL2xl*>VuBTOv`}x zrPG>nVwKZPyD#Npc`^D2M8dcbt%PC416X3onr_-)3b@cdl$sh3_m|AP)43=$9-BZ~ z8ARwRF@m;I+k)84;b(*ytL9#4N&*k@s*Q%&^%0Sg5Hch4(N4W}5KWb+^0%iF0aZr% zhqm|HE8t8xa=ebGA|0I^_;9h(xY0To@uTyxkL6g3nqvg=-=7Msv|*NGU4GBMNTT;2 z31+wOt`ELd0irB+51j~D2}eOGfzL%uW}b)z zem1|D1MyQ}H8)7PUWwhflZ5BIT(-ZhBUg6xzX%=G3>x-WyFEMeS&F0LC~nryPN8b3 zAzavEwa`bcH?L4?L7Z^5CIX%9-`^h%hj!{&nYRU(z>|)KiQ{Q({Aid# z{kPlu2}ZQ;PBHILbMyG>J@zL$av%iX$b?3EBfygdX~ z51`@ShV{z1U@|NO5Vd_y8`yT zQzvtj{_!!S#1YOo3YGhw__zIe8Hd|Dx~Q`n<+9pg8&pKG2I*Shei<51OBX#A_$(HL zUmy68Zns!&Ky<-IR>aLW<^H&Ej$~HLfD4C@aj^qEHj_9hxBr*VJM=xmj=_sLo$7?m$4(1SGi$^DwaZjKT(E;{lOmM)h2={N%S$etrLOPLXG|SDxdF#(~vBP?pynr-a;>;%`oax0%JE~ zG^{33@GD3TOhQ6pVsiU(0UIJM8C{7q60Uq+)&RI`Tmxr?XjON%89_*0`1Op^PUiec zVOqVaqmT&n;1N4oW#V|Lv{3Z0IDO0a7-J?`a^0#LJYmXIW}6TGXAhv=hug{yP|~~jsDU9 z@rGqjm#IxuwAJN?U%<#l(r10rVDx(O)Gv>oI6@9>B1hEIe512CQB?{gS3o|BtM{_=O*n)Zkg9qr6t(!c+GNvL z>glMjucPPP*c5QRtkC2lY_yvg_;Q(pQ*Zoo)ptve*nR7)aTrk~d0Mqe>P-5rndsQD zV|J(dyRhfY72eZgYuuy&d2duz_9PT+=h;tf52Q1nZ@&Oa-UdYm-h!^uma>S zJ@?V1yZz0-_?G7u7p@0r0abO}tT2|f#$aQA_VRjH zY)Q}M6z!&~gQ;qcByP2L^vQL9pQek$Yv9+nL+IUw3j8KxLymqSQA4c3^LVQ-ywkx2 zmdOhlog3$fR=2`fUTt(SX$l-)%sGV}JFM2bJ&#E~O?Sa>HvD_zD?}TChVGNQo@Z~v zRc4JGqb3`1;~~09#J`*8AKv35WW>K9G~>#U8AMe3{{_^XQ@PEu135Cb?|KI=S=2Mh zJW3K1Vr-B=w1nTFS0h94G48<5!;&l-Xstr1GO7u$79io_D8yyZ^xD!Xa-(_q$rbwN zy`Disz=+e>)iM0@Y2^D?4z_RZ1}34}2Fe@x=1!5&tMkWH3Zq0~0YYq@P@!_OgQ4#D z=SN&>2~%O{@?q8UrCqPVb9N8nZnS$xz^btjYJ;VNRqYt-*uNob1&zUU3j;0R3z21^8gqo>i2JU z%ua)a;7)97How)%dUVsvaTi=)B_U`dE_oSQ%(k{ziuc}j!tBcIWf3k4VvbIX zhnznDbfuLEHUu%U5s1AXNLPwfG}C`P4tqc-g&mRK4RzhW?0kIRW<(D(--|*%Z;T*4 zTA{CZdf1)kEi2WMjn2cdIFO*#ob_JU307#5lF$?Epvo6n_}!Qp(Crw$eVvgq{?79% zx-T4eXtQl$rq2F+iwaKn=I8b8XE@;5qEvXFfb+Eq0{*Vwy1!bFf0;2EdD$6B>@olJ zr`?x6f$;jP3(#-3)gS3bb5AA=>M=hXcx85UbaL+wmx#A`q5^bF7zvoFxz6nc-c+-5 zl#Y#Z5@Bq#d+A~%lvCA&UR;DCa?&%FivfYWAaLDq)nCH!jC3B1{P-M*FI$FkxhG!ZxxF*u$-87Tx_0cyT-}XsFLeb`)v3vO2cmsk|+^Hx4tL7#{*EJ06>N-|W++s|k?3Pc+IX2AY z;3(q=>3o&Fe&5SK$;XLQk;GnBURhj;`B2=^BHbVHUsrqFjo&rkm_2>c4_}h%C#$oc z&bunC*)$>0E z#G^kQqQ~&}h82}yr$~T$WDl^C<}>C(qeD9C1))kDg-Oaqe#3`!g2|(5BjqX*1H`q1 z1kG~GDoiZn_hYlq|6R+h^zn+Eq-AAg40%p2fx(A}JGZNlcl;#4Sc!rv;+Ub zU~w{wfGt{d#GxxEcr=p8G;mj8d{Mot!&`$Ux$Ej_)@7JmnxnXJVc|#6k>8%bH~eb< zE=aZ)IXLlYD_UF)M}M2xQX6xkTZ*PnHz<+pBIj9)@j)0P*(Q6B&(F#MRENUZy}S>XLH7I!<3 zMQ`ZD#02EA%bK^3S^`!6H3cO)vMJ}Kw{yJXK_>n!O$^eHx zIzgF`k(G6N;JXqtwj*@Bpag8mVWrgGRaXb4wY>+DrY7oOk6&u#WfcO0^?#1dev_^_lLQOFe8RtDSfy)oL%@8P%B!lI zTLthiFCqHF&*E0wkemDO)bG|-);?EtIGdZ(5+O17ir=1eUKdH$yuN!Wb47Lh-#K&94P~S`HZx{p5No#R+I^2jn}`=~<(Pb-P@l^dWxa8nJKYAoXT|=+aPySy zyGh~5fA{_Jem|$FzCPgPe*gP6w*S!*Sp!OUZTTmW_S-t{ylIiO>m>W?W@KjN1n7CG zCp^{Wa9z0@xzq3EsVL`E)cR4w{Hx0Nbd2Qq+-sEBXjtmaUmGUKt0NgvV zK@60}%q1ZCLHavzJ2?m52TL5v$jG2`7~Ls}4-OuCHHqq+tO`a~vUnSK6t{O*Gy970 zjQ2A=Y13GMvOz}&B&^0_?B;S`9Y-;j-adoXcta7Tk@3&!>V47e?emeH0G1;QCc1ytW)5n!pnk8jABV7>C#Fc;i)dCex)Is_f5f|J#tp!c8i1KQRgoCzB?3 zZC##~4e#BpuYG8-j0Ob!7@{r|PBR1iyGnosqsBudv{$|f_lG{xK-U^=wxdYOfL&rs z1v@wr^S-Fpj^Y$e=fTvZe|_vs6L|NNv~LP?kkM+zOm4Ju@lVZdVr*BFbKgNj8KJx! z14O^_mWLB%A;FqHY~KC$PpaALr_G55VZa1_ZaW-#0;#wz2!Sj=9uaJFdMT+iG8@fKa(icHsfXsPVL1 zQT|c%-4DNB0=aT9mG<}AHBFpgRN^HnNq2DG=6J?}>q>i;T6Ln*u7b=$;ZN4OuHG3UD+^bwG{^$cr?5!e?DfM0d)bf5TZfe^; z=}=er7)-Iz|N6gm!T{^&>4o($`CGzh{>2VhT9wJ^w!6es;j3JdzR3`DOo(rrU9R}vR;3cDl9E-AXlDmtfp ztNIQ@hQBEWw_*Lii-w2Hnc<`sTCVgMe^{@nwu3FYQf;!e9bQm~g=O$KGE)h+CI-VL zBvL(R0K`);%eBsqpt$%$l>|`*>U~7&NYvaA*lyMoKD0ep51Snu9bMbaoayADM^t2H z(40LsI`^THTb5ZDJXU_b=FKs|02I#efQQz+HZn7loN$M_oTe|{hcxq*pnj4qpUGhl zuSNSY7Z$2pnoM}xtxAbNM5bIQ3bqXn+chu&uAI(~iWjf4lX6{&lUOGVP(3m}Hd_{c zool3PkZR5OwZUzl=$B>2f|_|F;*GMUL+Kl!*GKqk?@46+H#Tv=t3ta$wYe?+P(Hl8IU+6-$c0;M&)L~s<;>bzW}rc}fK469W}z6( z0`<2bdQ6X~JvL=f4W~mU;o7=|yI0_#k5`kh`gO{Mf_7%z*E3-FOckWwWa^6S&r$`~ z?)2Q|$9n6x%=qoL&+gv9(GOnG3hxZVGAGx(6)vdWOfwfjDR!QhPORguI=;)@@8nDO0(^M!#F{o#(%-GhU||q z0rP7)dPUts;+ZDB0;!n(CHVvC%;cEn-$n@Tq+`Cll|rD5;9@ydPJ zUHZ>GkEuhFMW{C*)gferuMB&nVQGs3oPlCC<8L&oy(2%%JFUh9cu28=npXZdn}EK6 z#CJmf8RwrP$rot;&*MCe1E1zdgMQoaS~x|1yNIPw=E!-AvVL00;Jv@IjPReah?K7H zI_XV*gR0usNQ2?K@{daV^Sh4vgnGHWP>GDSpv5b*jhp&!yBBhtW1}UJ`31w2G|Q7g zS5K+c;l$&=b7^pAB)TS#E}2gKxgX~K0~SMaY4Irnn3VHFL1#WjSf?IEyo0(Qb$0&; zDvODogxd?evl#_o9|eC}@gSVZFaKtNe)PCw&!X=JZ6kAwXNa<>fQ8$HXdV=HwakghPN|9xmKa5@C zJWpr_V39;W{ythK1mX{Y5$oEL5*Wf~v6!IjWiww&5b!m^!yj+^$y0H<(MkKMoLkpo zxnJTQeDv*dW;XEn8nwq9U4$Ytg|biJbj-qg>lNo1S}s!&1!KOpod>*%@But4_fykJ zTp)JsX5E#|35jSPp6qOX#uvwi>{%EUJpP6RetRP4ok`i=H5Fa(1$O#f<(cMiGebR+ zkFs$r_JDAoFpc-==H~e~turK=3^TJ)JpEyuHOkHNZCHoVlHj)%olEn4_l+ws^uN7x zA7&h+y)rvz_VwLdfW@0KGq$Pil+QPynh5LqYx)p^Q=%wOMRtwDQIfIQvXb@OrH)Mph7NGy(am54PjV$r}3zRZe*>Uh^DWXXFjSvxy%x$H9%y1ZXZwJEobdCw%z;z6m|RvF?5Se^ z0aVMqjPPUug&ONO-`{hqJZ^C?fIc$kgSfD}HJQ83t+yvI4wM}O#N*GWWf06QyEY`f zPo9)K6UWBJ8Wun?ywA_iuf10RR{|icqp;@x`*qD1fPWMRi-(YP3s%WTv7kE!P07_p zay@EvXxRCX$p*Qk2Z4rE#aDJ@;`!s-NoE9y*;^aH;VAmB3&$Zrq@+_FHDPT?iwQ9+ zG~*%W+(;D7$$me8WWuFZv3^pj1ZG$s#E(Y+t?t*E-fu?F9x|W!!oL{dd_yEdB=9vb z>lEd&0NC2f7AtEwa~{L+W$S13fYLl&-T6#kkI8I_*_9QswDO$Gb{)|E%b_zK00|4N zv21+8dSVT1%_m;ykm~`?+^Ti^@1J;#Soj1OtR+a?Vvk3yHXewipnj{G%A2;-Hzp#` z7y7zwkc=jep-AD;tESp~jt#6_-sFv=Ah$ zKX#+@&;DY5Jm+&I?*W-;#!vYmu=Iz}5f*w(PJCyBLtz&YU#g0V3N}Z)+P5WXb9kn{ zZy{KkhF@7g1Up}D|L7tj(b1nuoS*sp-8uMM*ZN6rXQkOA^>&7@Z8i5pF84w@3ECL@ z&b-PSCFvm)muA_=jUsxSi(_o2XYXh$!ix+yKf?X3?()sOF)1O}K9tkVQ0@*iz*%pHIJ`qPtVJ*{6CxFb&$>X$X zNrSZ*F=fmn=2ut$!@t>xN-HzP5Q2WDze)hAye(;@B{;#+t2DHu1LbMD?t-Q6^d%KsnB-Urq;ki@D{df z^G_XhR>?+!%U+W&S3H@ zvOf#J&4B3gbdQw+Dt#q+Dp>LFB?^1sHJ)S@x&aNKk9sZqgS2Y-=}W5VKgZ6CNBEsd z)p^ZDg<-?zz7<_hB8SNB>6eL&Z)!OKUcFlNRzu$-1$S$1ZpKopPM-eM@$Ze7aA76g zm;aMGExg9vInQSHn-5eW?pnT|ywP)7yIbEdeWpb^OrX6m&M@-Jq zJ;vRIz4$0%MN4}@1p+oZ?H9jp4q6ftS`eGt7rzKrCH1*i86EcO8VZY2c3qON#k{a# z*ZBEmJAd`8xrXaPRBc0EjWEs-D}D|Ox@c(>jgBoYY4w4GIYHY|y;iPuRC2fMhzm|S zxp)kEm>vpyWM+!&={sO+i4Be;YPPRwDld*!HVVe?lr2&Sm75Su>5!h4q1%1u%j0Yw z_0ik1w+QBwp-|#NEogLxTJARrQvB-3=Ps9V%bfNoc*LG4UMho3sv%b?@lu_ zt!RA{0}DGEYdYn%vjOQx7+zx9&Dq?pG3cqfK02Y zS}IkS?JJ(9$4ZR(Jf6`d;`uKcP=%hN^&CppZkd6?9Cnul7R!-oZ3}L=dV{nB)8K8HF)`#~ zjEa!@9&-Pbcw1%M+IpUzD(q@3?A??3W$1fxt|b0h5gl(>pvbM>5V=XLUp zj^4CW3OJ?I1O%4(R5iKISureJ+g8aj50-QkdWu zL-2pXz1E^TIXXty(16Qi4GF)~tp;WSLOF1yU9k}rCr4V_g?LSd<2zjwM(WrcQSZuz ziYz4zT1(7w927#RTP?1o8|VTpJK$oZVJuZIEI>T#7yy6<<{gDUfLQY*j=BMwcA+XW z!JoC|MRmu~+?~Z;ZbW&%CbHmr8Lkfv_TN4DZVt`htt}nVa#)-;20%_^x!TPHFdTQL zt34e+?W7w6?uM-P$$u`H#Wn&dA_G0MN=&xW42pS{*PMBU`aHJ~@P zedD4Eu{3SImyJ!F5(mT|GLfS^v~LUEf1R3ANWRlSE)=59R$A}dHUjRw0Nc$`IlbY| zPbYc&_=5fat!=jMcoZAm;)GeG_uzBFGSIEm9B_Z}} z$d*9$WYq{C>BmupD}}QqA$pm?&WEX8XGfL4ex(E}_L}%U1;PPH`8MFfpt)!ph){E& z=K~ASXJ-+@+xviVKqDUh(xfO~PZrK}i4VMeBD2~9ke(9nK+o`_k#!~e=lfTVL&o@Dp))J?kw?6_#sk! zGyUrA49DbTh9t`lw+8rOGE3ys-zkeSzl%xe%jK*Y#C`LqLk^LyATZ4s4~hUyF4I71=-VGnHEt_X@Fy2!#Fd5C6W{(Bu7x0LpXvPn{uHb|o) zTXDF$x{eM)>S7e%^i%aU(s>V+pVhc9m@vHWvBLcUc-kg6U?hsfTSIs*mtqpK!P)Twty?;+xnql0wKURjzd4bXeALS63x|V#o&tXeYPOA54#Z7|O|ZeGx2LMJ-Mz(ap`NX(^G-*_=L#nwG8RPG1L`H>ChOvxfpqNL+*Os8f6n_W7gAQg z;Vizk0ux89t>qyiCNZarJIZ^z0;LLYsc%ofw~7_GZFooQKe-=i)*tm3^&$#C5mf-) zg_v1%WFk8|J@J=|uVw0X@fVf}fw_(HjCSYsl%kY!!tO2By4<$q_O$`DS z#ylNek1n(86O1Sh%xXV$gke=(kfNrhD*wMXQNL1J>WZ2KOlGN~uRL+U`ONI4qtlIo zK+3?sqUqf`QG;a=_+$9dN@U{xoG$z^=VfaC&#Lr-U3l!NpQZltPb_BghPUL0hbN~1 z0VSS5rJ=SkWbf;t!UlAoi%WG!!=OtE^s>`;iyQE~J2;+U>UYiJ`n+fM;O9qVUGyeB zNvpF^h1A8U?djS3k2<53{L$cQljV2SG#*&XD>K+(^Zmps^dOc$H;tAytT4&2(dXt@ zgNo#zXFW!xEjbX7)4}c)DMPC*!3)K}RZ92?K&+$5zakVPFXuZa$LxpN$rB&*AVD5i zXBIOHKMF4$WJrK8{Lpl$Vs+dt4jpp8o4Q%K@bz;4juUGnxX^5Wkay=Y6$QZRUe0L(3q!Zr(Tg49 z5t_sbBM4X#8U&jH9$+k@O)eYvb2nDuo$tIJ#6T97#Q@PKGwC575@+tGjiB9IqZB43 zZF58p3k6nE;YMx{Z&K-Mvf7v|egsB%-yudH#N)o6R4{&9A0W2x4l5xvQQ@1bG6`VDAL@fJ3MazDbh5lcn{sLb8J!gOIm?|N9CU*+PX7V)EZv=90_nR&7-3 z#|^k&-keXNU*5hW&L&GJ_<5thNd(mf?s-K6BT0{N;?PYrrzLD+! z8bjB1c%dAzlUBBVBQOSqDRl<}M2K7PhRJp(<@`cxhGzgK`P-)bx zx0`)Ixw35~d5GcJJL}$Z6 zZ0n3I!HEn;l$NIb1SZn5VyEC#(OE`2@A1GYleJ8*%ATjHD5NnKNUZ-)H%lC%h zV?_5yr{cH2=JI6vN~gf}I$s$_m=u zwXxZq&oY6SWQi+n-#rI@ML?9zyqWM<^KQ!qbObwcUY|WaYdEj_J&z73avs!qGUrSF z)vM6&(H!8Plz{BcMH+n*#%4oD8I`F6G;Nw!5zVi|t{V$ogc?xUOHKW5(uwxD&AMhE zcM?UpGesf$3u=W(GNk18d2cWGKYLxjbE+_IEzT=_P`_L^m)qsv@CVAw-m7`L9Q$32 z0MXl8C&b9HA~z<;To5iKlr1SIcXrWF|uJ#IKgj?tM>%E_k z*pv0qZ+aP6>7}6nFM{U43^Z`vn6MFFL!J=8Y?eBGw?{nm6V7Vl=!GJe&`vwxYqJ{9 zTc_~E$zL}&H!)RDW7qPDJaJpv}YiNJnUIP2)UzgJ!FmGr#g_zJiGrnFHUj>EmoUTOybH zis28*m1jLW4XB#hHGr;AxU83;%L6Q0&7UVcM6=g+9nNn9Yxmc_Mg!u28T`;rwK?CW z-F!UB&RIWh9Z~0p-v(Sb!tdHfnKc0Iv6D3u(QKVh#0fxZPTc2+*#Cccs`ga-*D>4i zA56D(L}bB=S<9f9s5wjxilBr7((us8HBGevO}?-~=izBTxgKgs%+A@zGX<_<{h5EPjd z!Gw#EMhxWVlTI+p30M&N(Lg)synTfBvZNRS2~mkrG2O~>G(j3$VB+vtX(1{Ug-9e; zv_Mc)0*F5VjJzE&qr{4Gp_J5(oda~_Am$ECE4B_Xv!QneV#AuQQdpz5) zbKG{Qc$(^-pI7N;(^o$~GuCAMBhM$@;(u4rW4K4N+j!;PB{oOQ`uP46qxHyXCZzd9iE{hZl1j+v3qS~zg)@;G!?qC_ytGziED@wE z>tAi{rCLZ`7pKjFXYnvl?)VsfH-jQ0j4XeLM34nD^&brm_e7ve^9{B7zY4aL>IpVl z;=Amo2PqSPyEQSDLCoG_zhP*Ug$jrGg$i2$XsZ>1R!B(}`Q`fWZpTGNj|u7ezmlJd zskCHkL?7c~#a!bQQhWWn)5UbaOkFbse?0$l=a88)pOs5ehIJLCZM@Kv5|7irf|Scr z&|RXatzV<9SDO^v(z^JmDbu`>$rO%}smY)P_v3rB$RxdIBZ94bWiwJRon*C7Bs}+c zQ6;HT^;fcyKLk!3HMJ}su0arMa=l0yn|A~506EDl>6c#6T94<=$7!f_0qJx6BDzWw zu@cZRv)SMymJq=ncQ&gHiuQ*l?El*Z?7mie0A5RO>fArgN@?0Jaf%~IWaP_|rwyyg zpb)3Mmzn)>7nv|OZFeg&J6Jg z3#2$9fbiZ?_kAnzXsg*inl;J6TV)LE!spXd7DZu*)5&t(vz$;tPwqBVkAb|)$_CEJjPWF5{Ti&R=6wB9J0x)?lTiROlxu$Oe&7N_YZT}CiY9kus~r_JT1 z%9GJK%x;HkcH=uC0fmE0IInW5pTzfeTnMb)>ksDM=6N-*si2+5q5u+w2SVJrA`*MZ z6Coo#eH59Yw3EAHzZ4Sb7jP&sN3}Yvu7Zx4L4#^CUBs4oCaBiwc@(cp(rUCo{IqLv zX-OnB&n7fE@~v*v*W~VX!)ATiYBG9;6Eko`y-~lZL2*f`+Ew_r)(}3UE*7&<@eCvi z+ue?}u%gIIT)c(3zAL{Wzh#7ARmFn8v%u!nt}@5bi|k4V4jN0PTa>r$Fe*K!V5{=u$cOfZ2EmligJNL-h{(<^D+-v6$K_Yk_5u~K$?p=+-$}JM z{LgpQ98=zAHN8A9cEP@^xVKobYtV)vzwrsaAWN8azwAOEl?gM3-(C`Fj3U>G^MR>g+W;AG%c3*-IN#HyxS!za~q>ll?a%^S|~TZ!v;` zlx&=SFd~kKF4MNc=j^o8;(zr|c{CC+)$YA|193TA{lcSfHta z!s^88yNUb(vdG=G?+&8hNL|?z!j4XXk`W0gB8@KPwFVOqz5L7T=KnQWT2*C75v_qn zFt4Ypp7pNb+-nvRW055C{k?&~{mP*vChC_HQKkk4?AA~B4!!ZEQMD~~znUKcr#nk# ze`yq#O7Zy}U)V775EmUS;bL%j2~Z*Ov!O*rpyT#vS!?GrG4uSz|A2Mt3Dk78-FC+# z6=`H?;xn+3NJP?!P$)&mtWt7dfr2rxl?)R;YlN2bnPLs&raa#bRdrmg-R_O2ZQZUK z9s=qKHe)FtqeE${!LZMg-^1Qdbs@IV7$npW#NqcK(jp@&#xN;GV<+;BUW45u5|D== zTWx3z{65vp?!xk5WXs7?(N1gYSITh2^{SxAv~IlMt=a9iKE=?MaA|V5^XBC}AeUqU z>zdk9UhasSV(oRo8a^IkvMS|rO6Ly1d+Z>-GM%gCr%_!Vu>anuNHq)0JUu(#w4YdI1SGCpH{Shhq<3q&?!6yV z?JO)?a9Xc+&~|3!o$|Qf`pOXg9ZWb{GnTV*f7QF}DM;6~;i`bvEUcJU7EHbl z4gQ-UO3oNG{E6a$-Z4z{^P6XWJM@d9OhvHj%?AqRB|7+_c2dGz`V8rIo)FU&9(kl$ za45=K(aajkZitKyAKN|X%YA4(6vevb7+k^)aF1^{nR)%gIlfOIqhwTp@Z zrg`briK?b80nR6hV?`6$?@bKcxHDNfuzshDswiC@By;e}D(!Am+gwhtG~KI~S2fk; z(bq#BZ{jxK=lf=fwJ82xwodAD>Uq7>Vw;$|HIWg zg;&yj(cZCbb;nLR>ZsGPZQC8&wr$(CZFg*|V_RRn|L>febGa|}+Pj`wPt}@pj`16* zL=fPNWl@@JJTD6}Ib@lxKBe_0^T}0CtgJRYPvV*PnJ;%|<*bxYBzDvcH5={5;2F=y zAuVf=HD!G-GV{<_g@&C~!?jO2a%WX$5BbxKtvUUxN*sQl2@5(MfWq=N~l6sTb zS#$f^>Bc8H6C+B>?d>>y#xhrQ!&C2KBNKs_mlr{dP(Sx13Z`1KL6e75o$KT=vdEsv zelfl5#ae!7LY=Fd@M3XQhB<|dr^gcyX>efAM0$@+aEh)sE@w{*GVe3dlcMbF({nK@ z8cZ(euU32!A5$FKtgV=u*o~U#KOWJ$ zLi1VQKXxD{rI{A8*ys=mrFjpRU{Cl227#+vP&qo}@mBj)V*hmp_o)_)Gsi^Nk8uKg z3ud1wb3AW>F$o}6zAK2~g;dCsxf~G3;Mc2D^to7joT8G>NU3a0*yo%jZcNTj{AHt;3BJQZCg2B z4?FKviXih<0yI&UE==>lM>k{y|+uwE;)$}|Lyxbkn zq%j=|Xexv~b3E=+5`ODgYqwagE?J(*;qr;TT3Kz?k7f2H)|OYtz11A7D=s$iOR{gl z9{P*)G9h4`vBYp^xO3gjAo+N8yaMg^G1CyU&=BfB0Oi5^xnuHyD5`D;hu3oveY}6a zsBYX;!Cr+dc2EKAJHoD?hL`WlKJWdNK@rE(?Vc5F!hT(cpP+FseXZ%Ok@*XEUpF|c zWH>bNfBzIJ5JOS>^`d~*HWgQ`vvGv20JJ?}dut`C=64H~s@%`VGgXEmesbBL;TuA3 z67*0Q0d&Sz+s!||$9(7Zc3)V&B{KQ-QHl;0KyWLarPcAreCoxa#4wnjM)xVF0+qoE zS8SP3uP6})^tYSD=E4cfjCQH+PoL=Rz#5&lSh61C!)6Q=eo&-P;>+EDHG?8Xgw@~D0e#%~$G=a}#55BQ^neZB@1ac9 z82`r-aC*AdVnZn5_7hDg;^gAAx@R?B$ryld!LKZhlRJ1OcIs>c8pz&*it?%|n3oT% zuk(?bukFu?uZJn!_pDpU_rCi2dS*}<%RV&&qGZFr>+8T%K-&t^`?Z1H=RHApGM!r# z7KVC}Bci-2OYQqDrJ>jI4{f64;iJdqX`QNv1?mLtWotx11A;o6rY#4)Pv4n(y3jEV zd&Q2@epsg*rluzJaY0bZt(?mq(h?t(+##yzlkKY(x~g6fT75`C6mY zuCBZ!gA`1Y>00=?=N9W!(biSfkYd$QZCww!{7qXd7PoxlIo{)q#_ti0-Yt?AjQKAV zP)$q4Gl}+zf2ENdE;fC1_k6AQeDopfC`h250KL>DnNpflP5!s#>UBm3jVz#6B_{g| zmbFZo8`M-d&5<4lo*xQubsQ+ADnFcsC(fO@pXx-3ik7WbI4q73mj+QRtbzZOiDC}%6>uWkB&a5+iky84zDE2Ma%eLc01xUg1 z4P>628LPW0Z&;vGQr|e3EW?=ErV~o+(2Dev*91xBz(DAJJdFYA{FuJ{^Yb#Al>K?s ztCNWw*YkDba3~^0CY{q}?AG|sbg5iu=H(6>z&4N%=o?O+D3VR4}nP z#;`M0ctmtuoKOnRox7P#&aO`&YZyO=&Q~SPi(dvQC-YNLJa;pOV8o8JQ7m@+IM2~U zbGO~QIGhZFo28y6X9K1|N%9e59vT7^P&n|+aVpHMRw*=O%?;OKL+vpACbvNiW>)#< z(cx^Z*~NHJP^k71*Ucrpj7<1zVQG!JRw-IQl`N@z1ZRG8^Bd9=%YgRr-R56$oP46c z(J*0}sF3-IYJSVQz$7Q1Qv(4#u@xVLG7aasW&6Fosf+zdU0Ll*h&Ukwr;r$BjN+b{ z1ZE?!5E6-HdvF6QD*_^%N=}8%>-GLl*JoU-|6`;oYDYx|=QoY-W(ZCILA_w!&Uu+4 z(l<`aR7BLlSjM96jl`5_CJF+ikb&L%m0*ck6}AJIP1q~gr^kn&s89^@B)4O2HX4KxTUL~n}-W^RPH+z8LlkHw!`u(Y=If$Ul ztU>-RhGX^S1v(X13b>7G*^3etLV%-%=kxUhudDUcoN>hQOqb`OfO)bWkQVa>y@!dJ z7bKay&E@Vs-6*a|p@fTW-LjZy8b%nR1hv)Io%+V*~=$2WQ0u@p1_e79U`CQWC8J(O~~j-qvt@ z{_&4AA=Ed4n_FKnxp#`eWaHkz`qo$wkNU03qou(Fi%L+%q|b=TH#CYOxYRC&9p_AwLWA8qT+lkqFg^EgC z=&iqGMtif{X}{e{wpyblJ5A7g`z8ST@jJ&kmfpCdQD`~EV%N)22F4ldY(_N-yP2XnZOgf*{=~FWyR6J@j{CM*=e+aY!4PH6TAkCe!~fQ>jCb9C7|xKkGLUt= ztARB}mjIIzs}CqHj$VUJXu0t89teKGx~uW|QgP_K<_Ud#$OJ-9kHVrfox5ti&#Ha5 zmqG#I$No_kmz~--ZE!%a0A!0n;~r0-r+oigb-gJB>vO%;Awd)#I;5O}rG$N3U*jH` zsb%vfaM)fO_Smx3_KklvYMnkSt~;mubAK(JdTjJU_x%)mxqA6(-Deb4S%>RH)#u3t zn&$T<0%nP$yw~MN8Hd-B*d@S#Omx>W9*le^^ zwbkc=w({ub%xTSLXrD~mC_Lwd-tI0oC6dOTytw;_TVV5LAN#(Z5hB37S zy9D1os)8bpmsbX#WZI2V{n#-O=>q+CcINT*@g%CY92-c#m$6*q;#ir@(^g+mQ1K~( zo6xGdb_E9$Ody-TKd$_?(F1ZuCQm2K2GoZKR>fq{2wxh%wzZ0VF=0G zCN|LFR8@v3k*VPNIAqeNhd`~PJDa@O|F-+DKL|d@?Fg504Eyw|usGuz47IUzhQ{*p z`rG2xjF(20cFS!WwvOxBlg~M^^)^HnwoU%z&rM6 z{4`uyvst_A#hS)CH83_3A@J+X=L@m^2OlS=^XP-eOEAC-S-MwmNFvj;VWm-{bCYdc zP~`(=xXH1Uz(uig%dE1; z_Q)u5+Z`K z?Bb0ecsJSE)knpe(f=O12W7XP#}eDc*=t3TQ-BQ2Ul5^6xY5gBTHm)(xnS zZh+fdkCDUX(vu|K@nAO!9}yUb5xWn*5kcoy2&3RbDg2GA zeXK;GNhFl${vP@k17@7X{1ZlCxCsrWV^KpZWlGAi*59wUcihYnH!*a1eo(A!wmn#> zC^m{&rGJ&F)lTnH-oRJs3lEHmh-;p=uP~;bKteZbJ=@-WT1wL7+N_SBC3bsjUz4Aj z{2GnP4MEmKSxLf}%meDb*2ikd6&ckZ=!s+o-PgT{ZPn=VnjQp@{6M1LTGHzO!o$OJ zr7v9Ygb*6p{^keYDmp+9Km-F0^lxzgl8evu3=<`4ojTe+!pPD6LDzM)F%h6NthW_p z(69&wqUfzEl51cfz{Z3yP}m*f{5s-!35vyvjuXM`ZXYZjOlPvKEUcD@*uTYq(J}_4TJpi))I&D>y4*M?4E=S0u3-H$#UMCXf5Yrd zt=%U>>-#FLMCDIv3JUkj28c5%Na~v6c2{JzV=FJ5V{eW>4 zHTg*)~4lJqMX#7y#Qr$#@u*)V7(4i8l$v!Niy6W<@?d3|o+z4%NGV6FK&x5?hB zBlpEQTvlb`LsJnpzPKN2YN^ph!*O=Hu(oegl2kYW+1?=O9@2H!fFDi5?4x%2--@$C zzbIRZhI9KCtozAPXHlfV44}xzbXoOpCm4QzJ2CUPs=4aCTYmcWs{>bJ&_U(?`dI|l z#YJ^-39n@XI*#hFd-MYtNNac+^~sogA4rL|)|h99AVn9oyAnt(ZP-<89cD-`kTq>h@ij=q*2A8 zL>kt`Nyrdf;D8TS-FJ=*E*dPRJiGTor6vjwg^%& z@mvW3GpBkvNUY5Z@|VbF5bBNgHO7|9%Jw)Cjr_L>%V@kJBI z(pA6_ESv01jLiOa@T8>0iZGN*y6R#g`= zZ}W@V)|hfCk)^bdLS)4*~elHFE*M97lA?pytk^wKQ_;7laEpn86#;iuyBCR3&r zmM`H&^OVY6PnXb>QX0qf?eRU1p$2u0%_-A4Zcj2^x<7}~+a^BgKv!Dc^tIil(^IO(bk$L9rAItej9HM?7Di@Wrp2A}6wd?fV? zEBdRRr`?P77XIpK*)TbZ(_hN{sFl0EE5tsf?t2U3O&*Wl6eO95+KSaL0)DInS=-Ha3n-j7sVlSwuwY#bRnJ~bbONe6%8MEUTf7e|rs998jiSq8v z6cBAdzHG`NN~QwF2L%PSck>MXD!f(F7IQ?e#b&WYhx;JI$Nkp`IpDC(AE zj@^8frkB@lefX`s6YdE-z%cSTV@>)U%m@4Q^pv8k6#dAjYKa-EnI}H*hy9GsxGG30 z98$`fzf6f)Otp*dKqXo_h!iwGDDKCaFr&~Qg0HqNwjXn@r!gnh2n9v%!YOejMku** zGsS#~QM_z~0VxU5`V>7JUxX1{nkk%Llt=c47QBwhCpNu?V+UuF=X`H;#ZIzQMYbCE zUCh^K#uiRne;pqxHK4{cuoEn2?lK-O8?uvZ0Iem1?WK=yHrangv{3nYap{p3B1LL^ zGcnH|f-wU*d2z`}wI-^ALJUtyB2a-}q-S~Yyzf0=Yie8qv}Yy?O(CM-Q0Ab?$XErH(`0nJ2|RZI zXG2shm38^_7GgcWHWx287{0B#gUe!~)#mIqS^a!j?ppNui}7@BTcWkQ`_r2a1lbw+ zn#j#j^b#0C!|cH6tHXnkX;)xEm5pv}(C6e^w(lx6IO63wBJ-8^CZFZ1$+>Ge{NlIG z(~MZBAC)%97;B;s%CzT~Kk+n<_vnPd9XI-(;K&SZRNy^NKdAX!pREaR?cIED;kT5IHIudISwV~^3Dd}D%w`Be#@%H$%S-uE zhF=xO3$nY|5au;CcpdG0<;I%!A{xQslSUHehESb&z~22;2V*tlfP22K^6UcZClQWB8ergp<-1fcgHym zzde;iW+{)0?pfe3GA$7LBAwg2OO2*6h=mD>8`N+;WsHH2s#rd5{GIYP3tl?)b59qg zNJ7nsbZxT*T7Ijc!w*+~d(~dgdWL#G$YYC1OSukOl7ER+nl5yCQRR|%DK(Is^Dg%L zvmUM6jba#Ncpk-qOzO|PqscK+F=#KuJuw!Nj0P~-@~TuV&6mYG%{EdDCbI}3{R>o= zk$h!cj${t9-kn3DyS-_`TZUPHm}u}%0c77bzY)#!KqGxFQXibL zvye4%n7ha7B$@OYDTQArCSBhEJX|!D=9dcNv?rpY$`h|+rGR&LI_jj_?Bsz4FVx)x z!_rd>{qf8{mUoP9-3^lk?-%mR@cT*hj`ZdGMa(t*?w3;Vn2}60cWX#K93^)<8O|xg zR!^j|?TJ|Tdjs8<4I|V5ZhVRFN%8{6glC`i3wjp~R~Ru4Qte-Rb;b*ZgZWKt!-a@v zm!(SPiF$l(-mE~joRGDe_2*x>Wi=Poqw<-J-{k_<{p?9x*@-n6M1$6<)=nw$tzPCT z5LoPQB}Qk(hlEL+C`UQ8cl3-!m4nu$!Q_c&L`t^{|0()VNaYa`5kz6iJ@LQn{GpId$@u=7e5EvA3R`QNHmlnOaypzeupoN^R?+O_5jhSY zj%fcgfIRW+jGO;w|9W&~Up}l92tWNEvpusTY6OaNuCAbGg%KwYlSQq4j&i54M_06d z^sl;pv!M1Zj%&x{2X)u;QC!JMiX~d)o~VFG5X(8Pa&*tpj_l=qS3K{S+f1=#cY#zo z($i>VwwYzxt!TUtwJv$;H*;~#D8D}~?bb3j0bZhM4ag7V64pN<>%ESmAN4me+*calyaAw;-@=OLHqKtGKqN zviFg1cck$yB2$-}=~mEI`62h1!_ZHk2*8I za80~ndM1I!)e4J)WD+{5_7f7}W2m|F4nT}kFpsg}KemT!VZilZ&wyRtFy8D4cU?(U zj~Fq#0Zz=k^}+;r(M7ScZ zHXY_kEc2p*B&>c}5QC^@j(KI`_>@}mxF4T&vk6uG(T{AdB5Q%OMOrKRDsf{#g>Y_k?Z-3GYqo#G`X^?3^s|wqPW%1{738 zboJdKaM$0AbMc2%_4gopxHKdbPs&O%{tEhEoJVt!u-cTdDBG?UIRHW*A)F1lE4`*?w1()hY3Ok zGNEXQ1BytZK1uy{5D3}De1?4hmL7Dc|V7O-V&K%bOu0heD}ycN6n_*6v)riB+Eh$AHwh8hS4fCy@HJfA4om1VSbi zM7f2xybTu1<3lVJ{>=(BX<~sLzkcX+x}FY>6;cH=r};Ber;3M_*58Jku@t93gEg2= zw>wZOzs3y`i@CR%5Yrp6Q1|+ANT{{xJPQek2*KdVAt3%!^|?Lu4|4FztnDSq-%*pf z&vM!7vh5lYSFNY(CPDN^$myYen%^4+%hx;P{XDgygxpy=gPS6y~_yyckPjCuz!^lcCpLR(l{mEX(BZaG|hD zDd|~J5wJ0#+CTA-kVuHrpa;#yON?>Hi~bcev1#ojw^nf?`bN!PhF4lG9~79_dknwQ z^Df&{O4Gmt0BG^6+2LNIO8d%biH>b#zpm`Q4o?1v%11CAQ(=79d^dxgb@=NL^?vz% zlOb2gIv<)YGC8$qqv5!-c=mB$M{?HE$hs@&AHh-plK{PcjIv(0NazEE;@ zN};0S`TCL~f6GFxi3SpX4c~_*8o^vUu;~SeNV0zCz3z>%42raIqJ)BAkx(pmWMRg` zg*E;@ef2Jk|D~wnRS=j+Ob)>TyITlO!We0Y6YS`W@L!+@7z#L7mh>Eb>!=2V1Sf(D zUOs<*AZ)br{B)Qsj8wFJ1zWfT&BR3%UvML_FbbL|ah`C*zXevA%2xYR2v`0o+ z$IGQ=4_M>NriLyikm3CKLbwHD3U2pvh7@QK z3BYyxiAf5e6WlNndqw%K10l+2OC$n%f1uNI=U67KYXh-hs0-*|imW954w4zbh=^_2 zXG8c?;Yh|4>Hir^K7LR31`JV#{SGfwxsCoiH<+PaSKEN}TkVqa($3Kd1n5>PtD~-U zdAN|_L4)k+F+T4v>%fN$Ty;Nz-U{Xl357kAxt}iF#B_5cFZj`~w_L5&8DDm;_JKRk zl|L`mcq?OvUTt-~-A`R6RV;VjyeE59(f7POA)^-pK0O{c0p)|st_sx}lS5%DaKe}I z?3HG_hjC-$&-%|T4^SYY+f5dfTFD0dZ@=nH&f489v>m3Skh(8%m(k`UfZG(_1!fJO zPg5W6Z;ux0)(`?-${Zh(_sRln?g z>wx0&AmV*@)*}*c0{kGlh2Twdas+lYhY^Ejlk-`yvJ-2b8ky%y{|g`%6^S4If5f71 zZ`FWV{(2`5v>8$9@`xb;d#3Aj;lcY^eomFiVRz0n$E&&6eNS?R_XD7b7-lBL+8maX zXRudhDQRNDKWTCMy#9^bk0T4a<$X9$xI3DxgoOmx=-18y@D5yV?@EJYneP%Xm5GAg zm;m)!Gl%`ClvK9oKA@>WwNab^G%HBTMOq{t7SA4g8*=D z@0k2J;KAK=*aNyI;McEbHP939g=9E@glWwJ;pM)88SJh3o9Tmn|t{w+)9+vfKpeV4XAjVtIlZ0&xL+#sH>MHDwsQ}#@VNl zWb*p#-Y&iZ3~9<7ck`pjGJN*RXwld1XG@#@IK+&W&g;Dfbo285U0W)QwQj@x=@REy z-_)gQ-MfGk-Bca+xJXix=0n+f7&=hCXh@1$;vqRa5pn-H*dK4NuK<(Oqr*Lo{M!fu zg+c!jeTI(b$tpFvMdW@UO$D3P=HvLp%v+=H|1_u+DywTk24nT%<5rsN*6W=*VR`xI zo1ChnAewsgmIQ5{RX6iqA!zxOvYB%1ZOQy+>^5#EL&ua$Dgcu3HoP9-W?p#ObMa%` zpL05tQz}`ICCrK~28M9~mmYwfQ#VQ_(UI<#8p@x+X5)G?w2(ScQO^F~UqIGrOfmy1 z7}PW!_B>_`l>SQ226}S(sh0ek!HpMqibW-AvV3vV{)6_CoQ-bVEvbQguTMZ`?s;`W ziTN{Zt}NO*+?R9T1#+Ggw3TC4eAU>&%^ zeP~D_9qsLaj@*Ir82_MdQPkaYAk>BBkY&_{t64mjTb)z=lsG{^^v_QnPv+3GOS!y!P?% z=_Rhnv64vP(JDz*QK`{xS$A>obF;b?P=a4Pnflnm>(dvEfL}c!H?eG~)#P4<53@e! z`taoQIl(vg3H07gW!}arjV-*{T;$U(&NuIzEKGB1>|fg-&gdyWFE3lFw=AGI4<~_!X92^)Gq1OV{881T&>XE1Y59}J4Y%zmv{{PB)e%B1YfC#6 z4A?j;Sbj0wXRt%%{{*SxU%Os!NQB&+Zd-GJ7Jm2r<<4{})4hx}Pw1WCV<#adTH9(9 zXrEOh-W^Zp>~lG*5QxY`laRzGeo~dhBGz@_{%KoB73WY+uCA_vMG{)7(XC0Sb8+c$ zYTBKiRG;_0Nw{LrX#dxn7ZFz|$Ov~dA;HteL;n-r!XHhia~9pmC>jx|;KNC0D7ZYyIKJMOg-=c4v#W=O zd5L*nkBOF8J9gd-<4+W;W&+NWD((?BtF`OBgbGU8jPKVeZ;d{8uf|5N+a7NWh|4xo zY23%hHl5v$WB!7o(Ly+b=PWnK4yJ~E1lzL}K(V2J{_*@Ve8!HjD*nfg1?KP~rr`!; zK$++}nSM@besAQ6+~+k8{DJ!j=yv6o$R%#bJVZ(<7uRS}ytxAW22-o+>sAJPm3X^ak)ujZ zWE!Yk-#zIN~@ljylj_usZzVTp~aq z?(yXFQIL`YpLBu`gdcahh7AEX5-~`C=k#+GpEq2wjhG5y1vH0O=@1zLK*CaN`F!4s z>$8o^a8xsJ88agp2giLwQ|tR1Y)LH-8|_L~i`TexuE~`DeyuKYf6@FybPx_g ze3g|ISPDq)@_yV`1;s2O>Z=Pd_LIYX0w2U$@i#IZxN4!l2o)C%W`sphmvX+r2HNL0 z|Kc;ONUj3-zsEk^P0Xksh@C7WGacZ@{>ON;Pr@Lj%6!OG&FEtX32+mO2p1`^>G^o9 z*6!f>GSH>N3dzvenpz?`Fo~K3xGiFt9iET-UL)uj2#YeWhhMh|)U1o}$1`mn581Tr z7F8_EsmcfF15GaqVG!36)Y9Q`VP8`oWQ9-PHSF8TIUZ7`&tLuxw8G&jXsi(FiJ{LP zh@&@xsVz#SV%)zO&MG-pa*Q>Y$~;hnj*-~KL+$>_XH)76l;RIKuMCC3h+EPy4PQHA z`=Md4fH3!trecRlqNfSXWh>>}CLLAv0}$!ZTSsHQSb9)wob7 z^B}FEq@T%3jFv1a&h}?^c=9yC1mkexIh(`cB}mvjIgHxi_2Go-@{cSgudR1uyN*eL z`HQ2^ZmRdiwzgmdO5w<}uA#{^uUe0>;nevOnUVakRyeTAEH#@%$V3Mn=HbF-9lF0D zO27E!+G*KGXzivM=sma)p9b(c-9a!t3xhca_d5n%t=L0OOb1tOpbT-Q=BaMsF;$7a z`^+u=R+_zXrsHjAa$7ziuf*?ciHkNpP+lPZgg_VoY?jIe6N$O7d{=B3+xgce?kmspmi0R?VBM4gm!jne;aj&Jdori#30Jsm% zmTKSNpPC=;C$o7j%Cxz7pN(vETe+~pRG>WLgl_-lp^J~et|ZTWib~^Rf0N)p{Ge<`RWQ}7ojxJJ)QXG>C?o&`Vpt%{kxcL^NFf$A#t+akp<5DM zdEI*4MB>4d`-=mjR)dQpPD%EaZa$)&c_a6Z*Xr#*dxh0Kvm6!^r>vSe8nxzzy}<%# zz9@4=9Uyjt;?B-Nlrq)pb%r5KM-d5#$J08ctL@GokJ=MUHj3vE^lY$6WIjWIe;K@A z5^`m`zD9W2+1Q4XtmZNh(O@cvXqOqZ+O=|F&i9|&B$s(OJx@G3vS|tiyS+TBn-Mx+ zju?`w)wWhvTyM?Kf*OvSZM0jg-{xuAtVt<#P6`A=VtgJS%DNeIx^2DAk`eK9R+{wh zD46QR$R5|XyEjupH(R%qRN0*_mP(SVwFJ9!3FD{;NcIZn9ZTHS0^KyTy{#9Et$aV9 zw!7D{!>D}&S<%2hb^$e|jn8RW6<5xBt5e9#m1-#a|Ip`979=3333pZavCgBq@<1$} z&sXraDB=4{RdDdS_)0KFou2{-j`>OXgJen-(YmM*L`g@%M?#nGAOY>#GN9jRqin8L zZ*V#uf6yEYOMZLE8=&NQJM{-;5)<>f3QBSLT{oeNT0d&mwK})3VIt>N)8Qd{`7!OHwIy+NN3)E1>ZB?tRl4$b!WWBNZZkJ81vOj<1E%bcNMmr~5=bptys($Hg zVgNjU=7R>1--fT1v=7U}j}0XGr>Eo8ip4)!S&u6N3k-o=pxt2d>T->EKu7;srVLCX z`%5Ad{#{(QF52!t*BNGYq0V8BAQQs%s@HkS8&icE9q6dG?m9=HLKQVWi^PEf<4j$P zE+Y-&Z9atl5G3z!7guYw&JTraa6|vDF3hak*il4tCn-1v#V8EKRH5YhNh*8w{0}x^HsUo|I57FzdHPi_Jj5<2uL$myWp- zI}O>Sy1*_T+|p zk;>a(O*B^$6!VSqdl;wY(s2}L^7M$q%Ulxund)L-JRnuQiXo3`#@w)|Q-2Xl4MemQ zWP?RV$i+zpAvx+@4sq}Qz+a}z_??=Mn zi37M=4mBw4bz^G2N&SyA@9a>{=ot;fo*pj{rKG;X?34cErQ+hFULRuToW61NzrB9G zj5EA~oJnQcA8!hNJFx;9EX&o}gadK^jhlJspU`7!5^~T=8^3yJQ`BxkFEh(fj*rel}&P*J1fA-a;0*RhMj)PXw^C9-5 z-&C4SpXk6QTFED2#G>TF*0-`K(cr=C8XPsDxU^HLK|p*##D(~k_#}3M{l&f7%Fv6e zsl19n8o59cocQIV<;=0kK;nF3Suo+Cz`(iIk?De`dK=9Y6BW|VAQLr{IGp0ZoLy*a zNg>39=BgPorHeu7eFcf?d*x#9vxPwX{|K&{sg^}2`9VrVQF>92gMguxUU9!|#FIyi z`}0WdmBeWMI}Zzm1N&wpvUL7k~bX(3@zwa9plp~Cs;)1peEfdqf=`7hPWb=RZ-TCNisXItC*WDDwH_`4@N z2}xQMzeYGK)`<0HYqRca^|CHoT1c{yQTOZ#eP}REf5IF8T7lyq)@3>19dVq@2Wn}z zw-jaZX1wG6aY&XZ7baBeHX}B6l;y-rlj>#yJ{KN|6-(vEW`y`zJ2_F5P$g0snt1y{ z*CQ+W)ARK_?IT3C9u99VO}or1*$OmP~@PzrGPa*Kos zyr$%rQW<^_jqPj@TnM$_BMWVKf=CIs6oj^VyH+#RDt$Zu?ZpDGrcP!_3`$i81fpzK z7JM_6A&3g=;W$yd4Eu9UI1q6CZH`EcUs~5>4q-d79TvVsDTmkPmkEv_&b*ra-O%BY zK8x64Ejks-^5UX&7SG*A(5o;&dy{@XO@fbpp68}jRTQ+PN`Ry9nnsU1s9}^+Vie$a zKEhcm?FG`tF8*~jl*CX{Gp^hwx;ve2wao)57Qxk*b z#lvPEo(KUcF8Hus-ZXh~mwt`oH3=2l;eb8%d4KXzG4izJTY|xqcY|~v^nBh8A?&(( zjw8~i$QX-PpzpyP7N#1rc7-Enk+nIN_rGtK5HcFY{4NaJ@b=5tLrDrQ{ztDcgg&`m zs#LtOA6inm)$ya&E%aeA$3&-0*h#=~CI!jF_E;7`tD=o6maHc0+Poepoos&H)%ugp>Q`%>l zizZaXr_?JwAEA zY;bW)JcFxgfIY!ZSY&agluuNhqAvl;3rBnW7oqq>wXMArwGh7`ktPLRfq|Av>{o)f z#zgYEUgEyAuIm~N^S|e!9jXJ1<<_0^H0YO96Q*_3zK7G?{Wrxl5}&AzUgs zlguEj$i8X;SQb7xI3v9d&>zh1SM_l#LBo$+t+JawTWU}r)b6q}F$-D4xbvAPVoZs* zZ){d-E#5D;vW5@`?FM?nHyZfqY_`MRxPjnR6j-hUce$e^ zai%UH#iy$aa{G6YZ2FtEy80>*Z^}ThGLX4gk%*HDbEvL$t1`XUSR!;nt9+TAH~1M?t12a>`kmxo^7eE+ z%Aad8m37wbOuO6M)HGxOdGPE2+mqpAhlB!98hWL}Pm4x##!7^^$gi<%D zgvn}gYGQW%nELByj9^D7u+Q#7ZE?G^KZCD5{i^AHl}L*fncdGLyy42mp6p(lMjZx( zR7>+zAxtd&m0+i$d0@=17-fP5Dd^4;VO!sZtAs#`NAe#UlY+I}F{JWGW01M*knUrU z7sIXh@7uAc`O*VmquA4#Jho?SX8(&Dm9bcbwax(Aj(w9SLsp5zl6=(N%Q>ZGCjweqmhTcz+GVj7#Yrm88etino~xNkn7NM~52B5h3D0 zjy$A^$5#c?+ks%IAv{c_zNx_kaT*=*Bccd>Wu@g&l(J!Yh! zcW46RsiXzO$acGJOy1^WM3z-oTU%HNnrow>C7s>2Fas620SC?WCEPvzBcb@#* z=Hk?ATVk}RJ3dNRq`f5Y8L!wb+weR9eNB4Vs(|R*N~7`&DAos3^GzdxEqw4tYR}mh zL2Wxcxxi6W^61O85Kq32k!4rmt@^Ze>hs;B4O?`eF4;WqB3q+KI(lx8Hk=)$K>`Zw$Y7Mrgh; zW@xLTbB=a79mu`zNki=b6;I=UbvMYZ2(anIksD(}%J)!-!>NoaOF4!!er(aTJeM}D z_PEjU`2D{zM(*i?;|ROAe+LqGzRpE&R~@F(Ty|?p%BmURcR76ChXWZXNT(aW4EtB8 zoYpEH{!fOHOzW5Z)>`VqTbJGvOi5*%=C$vo+k4m*lCr0#ryS|y)79P>IZpE1`I&4Zw=Xr^tnhrO+KOXC@Hm9aN7S#qYq%o85T)6!0q1}MCV5u=>Yigo% znz*c42|mpqG~0gWF{!UphdLPzPggRJzkmDfdG?tYs)F9n`3Toi{!L_IRxB!wIsN_? z*Q?o1Ge<=VB(odOZ#tTMnzi8l0&Q+;azVpGL%{xF9sc9t@em52xPvJ?BOrA8tcNSW zzYs7L5QGl(IkgE0i`*!q$;Ceg@Pmc=P8XyUmUlJBc7C`F$5kc>e_(DfU7wnFHMSPU zE}J8R0e{N?pVdz(RWBJIg^9Kjw)U|d@ch8a9s5ibH~#t1SHEm(kHB1_8wRg@>Js1Y zYp_H5#Q3j!aZlTX)^$6X!zmwGdq!?JxV62cG7G6VniNyD2Fk*O-Ks9V(Ako|F#KJ- zCZZbTMmUO6&q30uRV})H`*OzoKgQ>$ono8BBrWyrp>l)fEZygQ@PC?q!THBU+=R$>S3}*xI^g8!w}= zwvy-Wm&&4gx0By4HLn*X`tMVMHxiZ0*SH^U+)m$bh|!NgLx652sXbc#E_514qdKUG zlH-J$^u<#UC4dtQMg%0V7vmE^Y-H!>c=PzY-dB>Zyrw`2ciWT)A-SyeqW*UM4Y#zoB7DXb!kk`|-O`|R&Lhxx0(N}Z`H zeYJM7#m2F!RZwcjOCQIZf{nWRR6@W0T8BrH#7;HzM@Hh17BI=@pm2b#g22y&*I4%1 zK4=CRn%T;Y>6N#ugBE(ZLt|$hA^@zxfmmdQ2!(8Viu7~{Ew)^7Xt-?$%TTCN*BX97 z(p(_%-=!9GC9v5CGMBxB*x|0pM!ww`)~UFXi{3BM%449exUJ)v)JbB_!`s+?e3T(U zjpu^rZ`yL9Igvtk<%YjCcM-FXZId}PJ9_bYE4c|SxmASaLn#wReRkW_EU4g+knw>4 z^|y=S_aRb(Gaqy}^~wKuTnloknONR`g(#Qv?a_UAG>>CXGq zx}WcegHA-(;X1lzlMl3TGW4Eg%_1z6LP)CkkS+p)N)e+ZgU5V9_r&Kb>3Z8qw%3OK z<)o+iMleN(o%`In*Tx>lT^A{P1${+DdHWV2pS&?qR>dy6|YGt>_KG* zgEeXE|F)9fhfS8S4cfChO=hlI%VgmX zvmQ8gZFqGXE!@p!YP=NZRg*^aj+t-%`TSi}&S zfaf)n=txA@idvAai;5&~9Yg(?)9gz+r}VY>93#Tw31cd7hL(|b^t#>iYnFd92Dz}f zm-;&jp|m%NaWu%k@^~fq_k+naRmdWhza$;Wr(o2YxPM9{ln^AaIJ#P0W0Bi^D0!ZQ z9KB=gC<&6it&p#r+~i43$IAi(i#Qo#dhQ_oIbf}xxi*@P7f9I*7-w|<;Sj1W9-f4Jdyy&4e#Dg?__ z??!K4y7Q4t<=ib)$KK^@usUv!KkeC7YSy>8{U9?X_)H*6FbwF~vQH0JtJaC&R!+^8 z14!e)$a-DCC!&*Wvycrp}&wB^hRhaXBM zzxf~kW3AHseIkdcZIh9GYBDwQMebBRbew~`s`=(G2!Ed@>hSy~D_b@w(Pi@7OR-Bm zN4D^_p~l5iFu938eRa70k9hhu_rU`Ds_NN+cFrqqb&>C9d`a&ExB>%OB4!2#Y59GS z+N5u=0V`gFC{79!aQwnWN`Io1*aT>%CKUi71Cq0Gpd%#|r2At75e6(&4dmMkKl8C0 zhkf!{TXv+ObOeR?AG))O$8$^srV+efPtVRzur~*yTy>g8Nimar71@O)t1RY*Hr2#u zz}E-U4)y8#di=W4GFH`H`{Z{OwTc@{%I}~g`#e#|I-{I^u3z`-VPHlCH4-v_du0*} zCLSA-^TuzD;kC@n0&EovdT=6I7OhQqZzTfL@Vqzk6 z58f~FaUxO8B=l;ZgM9YQ{o!v?Pvp;qYs6ODNFHwKvx8RsZMNNp1q;q8R5}(oY7Pub z+wWSfzD`H8*lN&VIhqUt?MSJCaH$68tar1+;FGMWv>MF{I2}Qv^tS|Tg2v-jBkhNg zQi*hiwB(o=SzrT0Gj>ndK4%SPXR8 z5LG=zx+1J`1bT*F*bp=)bGNCXJYUzRzpWF5i+hiMBM$j1&%QG%HbZMrK@lxU@9fH#nxyg1&s>uy?Q<8yoxm+%LwAIarv?Y;t5IfKw@BgoQtkja^tMiYtvV-XYky1;6!b?GF^9dO*;!z72`SIV*=J$v#u7xU6 ze6oc8bKIlyd`arE(eZOt|7;?F|~gZk8$zw3=&`rz@` zHC;k38_mX+i|cM5V&iX4;WNUbum#iqX{*l6OqiLOxjk8|(N(+J8V_WI-bzIl;zdQ@8Yey3-camoD zT`$`%6$^Kt;(np+QNh=0j2Ay7)inyk0u)B)p|x>t&cU!kqNL<7Y(c|xItpDzAEe(? ztFls)asIvEIL&c}81|sDP~#x$D7Ag5(Og*daAU`!CW|jBh;MVRcad1rE8$zqna~r7 zMl7-sF{8xLgl-R%l)kdD4Wh1Eo!G9o;8458s+TJ?X(j^DkbviZTeWqSNyF3mUyUo& z8Se-~!67o2I)Q;`>B9*jQ*!&i&7dk8!kX>|4h8{e?=FZi<|$|m57b@jT`!>@Y9|DIR6_*n-|weIg8$Z4)mux9 zrmELUw-dH{tl@IXD{*vC9MzjVMQn={z^YbZK*wHZ`eP8!*7wtAoFUZqjlaWG& zMKThgR11rjkDKb+HPW{Z_w7^#l_~~ZZZu`b9s6Bui|t(2?VcC z-cGDovER6~-O%Csr6hG#;_EH80_Sk!&{5(+fIuTS{H z4`R$qgJ@#2_YnyYaF9POIOA{xdY$NLdEaIQjmyx&WC-O5y7AyByI^M>pU@i@^jvb_ zt{`eL>vsIw-OIF5ZOkJU@Vl8V!2M60b?ti~D*W&BrdUio5ToYcG&s2@*i{t{H~y>B zRk0yL^Wwtk6)x)SEh`#3xEPp}{|mNgq+H|!7+4QJ-Ofgczb)c@%}g< ziZ)%6ly1rC)O&EP>6sSYjcfK17?dDM`O4v1tTp0p)FgZ^H+O={KMp^>_HfyDsVUtW(-7T9Ct>zy+I&)+kz5$qRH>a3c1F0E3EO8^$Y3g$po^y0J#c#cSj`zT zw1i8lmTE;N4f3);1{9{(^l*$3PE}XG*awzs%knD^_AL;e>;#(%Leo zScW5{P^UUqH+MyCZO>j)!{+!t0_XnxNfu;z*OQlFc8(qyusR%GudX za|mk`@5%L#N{2%2b{y%e8DBJYTW{)cLI%8&(f(W%rPP-yVT0g*b&+WW66oruKYp(1 zJ(g{}X>((NIYHJF=z)%@8+B5lY~c^&21he8&sng6AqwCK-P?I>{U18FVR3IhjC~R4 z>~Q4Z_(gyBVb+=PZs5&!_#dp z)G2IP=OmJrbK2KHHp2ly$@WBl{k)d5@e)>wVu~M}nFsh~6$MVOv5PQhbc2ZS8YJt0 ze1Epa53ndkqQxpG#SIvL@?ijn54^DeaoE&KU_7z9zePm7+L3cVGA<4?06^^={x%uT zC~gR^H1ekcY`>BaHhvTRF&xm~dRtz7*JQzs;P7p+nhu}EB)6)0K<67m2<#FVkVeK; zskgAUju9z}|Di6#&cI3#-A{&EvLVm3Bc7@ZI7W`fO2&@&rfS92;L;G>ZwS%fo*TFT zh3Gm)$%;~v1uU(UYj_M)u@_F#SP4>8d_W#^u-Yr%;~`lbFo8e;o$@ANQF*f+ng%y1 z962nlRrW3(J%==wS|GY`xSd%;OU=yqWMza;y5_hnHoH@#d>QGp2+@1d8uY!#d8GK9 zqCKVCK@uK`=SBB3$=(D87Kg~!ataCy;NTIt)*n%0Y77)}3Ag5X`;@rRVqk8Q5ZT4U zfOx1ihW~_f#?nkkm<3V5elQW%#uXmHnR|kRhS>tp0IU=xzdvc%!O3;YydZECGSGeJ zoFcT~86SVBr}2uZ+Z{0Bmy|}@m6=3wk~j7T!Xg1tkm6+wN!5pz3dtx(M-9x9tU^0V ztSz5;UiWT#Y8cFKogo7)%=+L26be%mCa2}f7G+jA|ZIZYv&7*Ni0m*)B>*?NmZQ9aLo-_QBI z!PQhIUbnmLA4eaBwjNqg6kBYhVF4Qa!+4%Bk_IYu!mLcZpMzBX8B+r>{VAn@2rA*| zxDsestn47zI2gr&^x*I?nk9ing-u0Tpg2>UpNUn%=fXpn!qBKP7=J?;aw*U^wH^a1 z1&VmK8yNp!w7|M-UnCt=gx*eTT~30D`f#SMDO&O=Fq`(s_fl+_?32a>7)iQ^!=jBL z<(Y{Favo{v(ueP6>>^Vaj-Jl7Fwl+!sL%ez?*nk2OmQ)DK9pCn|8}Hh-Z^JsW*oZbASXzg`^_nV&0(*9Z^jmY)iP9Q|1hw3 zeyaOEnmd6gzn&W&2uoV9n%UsqibHvU8rwe&L?PpUY_-a7F$O&q<;IJ`ZzlFuF*q`O zsTm$)NXPrM*0H{_B)|tRjN}+m`+)&h#qO_8@=i`VQM+gTwV5_JHIXt4Aj=iXRdb}7M{p~Q-*xjj(Po=G#Y-D{{+_!ES4Wo^Oj zZgvfw78vd0wx~uoa6q#HpEa+2#Z}EbagE&)j2a!Q-?8pk!JCh!#**5YLWpu5)yK9t zu_}UF(=0k;QxcyQx=;iiUOq(}QXKR;TYf&}%v~4B9>?Cqhk;P?f89P~uAUPc`MQ4Uxvqx?%U!ME~_wysq2-|8WPprn{G7z9S2*wxd z6QNdJC2syy=INqU^B(Q#0K~Aymc)wQ9R8DhxcrDkbZ5^{La}HfR_6)dKS=5&D3%Hehjk7*C_pcL0>paeIF4MtD8GkFVrnJ44X!v>xE?UJ zJKW6hC1#o4iuU!@dL#u+b*z}~c(vsd0>aOwDOwmYtXJw?v;$GFATkM-vXxreX}Q*% z_sQtsTkqhW^!?e=xcBLIbD;m1YQ|;d=7OW&YhynAQEAC%D~gs%(#BqH<5%fG04I7- zg<8samFA-B^UC15&=DOzemx(ZAw8aTi!T>W$p@>nf7KgbS$&;M*x5g?`(6G8$bCq| z-ao6X{2-q~+RicggEH(xY9cl{BR!{y^}m6~DA?W5`6#G!?C<++j0NM%Z0v-Ot{Z{e z{Vzo#C928D8-f_|>rK9;$g2i%Dlzhe!otFt8O0KnZ9S6Inl{r>x5%SUda=`I95zF@ z3WJBqIl_)FPbEb&d>_UK<=J%?jZZ&BMnzF1fcOA$>+m3j059@wUZ<^v3YPOF#n$62 zr#1hsjQ(hhL3A`@J>%~up_xq!TPPpk$ZncHx;{H~5H>S*&bW;YDs%dF^Cfx-Kq`>?ZIo10Pd+W+8UwJNG0hv4!@>S88&SM;k`+#nEa z_}bWbSdJ`dSlQ2_%ER*!WfI_P>S=ZSXb@c|A%mDJq`+`vZ>9=Fv)#yi?&@@xeeCNi zShtN+=npdX>8}M16yil9)foL5s};Y_j_=JON2z}@H>dpBmFrq4f_|!_eISr4I<#3b9!tz>VH`_I#MxPV8e!aiIzkBl)<0b=V^b zijOa~bN~jilCjVa6yqajluqq^lmTX~ATXSCD>qng@8_)N2GETt0%|gMK3H)Q#87__ zjTSG0mZQZh4tF7vRB2@USjWi|EfNA^_BF|&!_R!qxv%>lONcC-ut)ARgVz@r(T}+u z9QaIXo8LG$y)Idzd1o*H$OtfpI_BsJY_bfY!H~cibossXP7^f+Aynja_`QeC#x=s{ zu%TFAue9;u^+hylnD}WEC9&Y0v2*SDiQT8F9#=Iigp5jm+-I3>U)VEkQVai+>!8F5 zVhZ$MIBT0b{c_axCSP-(CVtK6P@{Us9_B^W-ZsV7Q>2FuOd5(|`L{d)jw9;k#=jLM zo1*tj{E+!3Pl2}&nP${h`IiXuDb9mOqsu_ifSw>WU3u%!ox3f+W8B|A?6(h*Mq9oG zIa*F+pfIjK8WwgwEIKx52qzW0Ww$tNh>vw^Ki&jWnIo8-oMP`uw^h%!&B50s-`vCm zx=_&kVi8HRC;bBR6WxrFkwuA!ghasSX*eZnqgq!@YuAa4Hb+jVXvTshTOvg>JjsKV zM|b1*10Lt+dvD!?bkZTfXMzclO+EO&=-GETsT4Ke0tsR`aPd|yXuCwZs#hZdNq~X! zOU7$bPAh13(yaF%a+b&9d4pfs_ZWQIGkiQc{RWf4?LC^wwP^Zg;kLhj8rxm_s|{MZ zYs|krZr|>q$Yon*Y!@t3!I1T3vt~_b)N>qT)npBI_h<9D+#ToKDFB^}(ONyqj6F_M8?&tJ za#~&5Yo`daDxtln^QET=KO9cg-g_W^Kqvga$e{qBQ|vY$W}q<{pX=G=-6^5IZwUrg zM(@wmGerzrgm8mgh!86Xpg1f78EWOBmdocdzg=+|6iWedej(xW-Fk9QPMHo>j}cEw zk=PQvj{fjwn1(lJ!I^0u=HakW7+ColF|W8XH_Mhv>?X zjQsa2_F+9tpra5i7M^CiH+mq?sq%kESZ!v;;Rs~V#Qz>)nG4owC-wjL1nc+F#(cK? zcYo@vF(BMQjsGlY6Hj$vkl$vvsnl=MbGx=56MZn4$L0meDo zY6WN3yv3yY$X#@u&g+)A?YOBNiTplo%>GZq$%?(~nQCFYT;PZ`<7Xjg2g`pmAte}x;!DL z2;KX|W$rB7Tcp(=H^G}sFecw(IE{z`Ep5&|^Oy=RxN-lYmz+RJ-pV~FJdkVb2Na9( zQm$$Yx}BeXGriuC_i>%XOJwZNHVv&C7fDr|5W7Fm?tJ*dQeQ&eS>H+o{o9z#v|PP`63)Hl zXYz-qbOYUkdS4eg-H9}erQdD&us3jG%yQ$m|2W2#jzypJi`4p2>-w|TI9>~?cNzzC zOohp5<$KsCTIS;|DeA!DPe0k|Ks=luXFgwtS%1Anbc;4@NL#~CgiP`KjRi#?^mdPY z5+Dq$A6ZDo)BeKtv_5!}@jT9l(W{oS@r(you(mmpdY0*X8!X0ebJ5n4E}j(A`ug=u z%zKCt1ypca181LCSVhJ|&Al54+28YciyCY4ak;}SU=AttfS}1Bq2tQpqX}n|02%4R z@q*m>+QfHE-VlW*511G@y!djjt{mfNL=?reHJTS5!muRyZQvl>VKJh@6s-4bOA%a^ zT?a=95a>}!sO}X5nm4kuv*Y}}ZWV@viYw!3h~Y6EOjp6pyc$_{V@lgux{3MD%Qk#u zQ~6IMI(COo{&(fOuhzRbE7cG{oPRf^$6p2%hz5}SHSi+~4x?}L2T_?vQ^vCKDxK2C z$s*Z|z-bhxvHN}?R@AxG51WYu7Qdm7@J95#x(usnEf|&h;$lb_uQZ1 z`qrgl!T=_TpzXkyLDQ&}ukB}EsG*3$aKv0P!-|KV+pr5FFTDK-2J6wX@2|;kF*2~p z?ET8?bU=jOaqAE}c8oxtzZ>La)Mk|~RWp1_hxqeo8R7B*LG!y z**NCz$u#85(oU>dym4247Yh}?{sGBUeF}>%$2SAu;p1!mGUyG`*@YfwSE0SeI+!;X6Go)#pybSTXtr4&P2 zPl7AT$6a13-Zm1_Fy@0Bh+% zTB7e3%i$!L55yX$^YQ6 z>7{Q^{`m0q00ShbhZZZjt_&lXHk~ScnDbOwdPR=ZWLvKkiRcnBZCos%ac+hC z$&dS+B=jbPdLX|qR>L||}EU`ucCWE%!O29gVdYW9@ zytKW}%?miBigV=bem#?%ilI*%g|8$vaF6wac$ zs%Od0%3Mv#8;<0<<r$)m|eKV#SP;WHtEvZ}ZO`8?00Up5Jy+ zL;J9&+j1O@x}f2PZj&<5-=H^mH+#fVUZTjOdblZn!m>c5DfMuuRK)vtN$P-OK8&g^xW_cvc6ESkT?qNa^jAm#+u+lZhE6JS0&0NrqKcK6|~Yq&C+zE*c9Bmfu-4=7wmN1M8Y#A@94o7_J> z#VIL2$NKXW{(@aEv^)?7)fOjCS>to|dw3bS++zNC_P0&FYgE4DyScN&pM>4E@6HYK zVr&g@5f?fLoHI#z z2l`F!Ge0@Zv66&M)P{irayuyKF>!J7GIHE)qoWPz)U%!gc02{nh@vW&ex;)D`Fl%A z&RmgM#Ir2Kf;j**m|!3#07E&57N4C&fsw{Q+oOW1R#;joB zqK1YB*XIRjU5|tarr+b3ejuBef&Z(jTWnDFs}<)Ij?rHEJaVNZp7_A}`nn{y`^K87 zvuJiy_qLC5F|o(OE}n3&Kj_~!EkJ?#lVZ%i;RUqNi1SmRh=K@$0@%lQBW~TxYdKMD zk&v0x?i>&xjhoi4!`u+$T2NeE+|ttW^Cyy$nCgkUv@s@}f^e#sg2jV(7is7H-Bron zBD4P2ne_lAMkf7T$5s$9NC*{HNLZ;+o)(W1Mp6MIeS04yysEELhXgNi{D7ruzfKLPmMhH?Ee^DFL z!;}T|M3$H`_Y&vuBUb@+Of2_7imZ}q{v$a7qGoD0QWn1A%=(hGdH_=sh-AWR)ysL2>%ofVUJ|eZu+;@S@C!l(;xN` z;dfZk3~Qi78UHk27%K-HCbVYQzpt;atBVy(hSKylMqeS@Q~rvsKZg-;JJKv(x=Cuk z!a-IN_6+uc16(%jFG#U%1o}Kclnkg1S`o872VS8fgjNUgv*o+S9F}0*ht_eOVtAw6pV4!Mtp09gYEk=^zKvExH#Ll-r|b_1Mn%mSy)&&E3cxRoWdtM ze7sw=%R2v(xfzSUj(gXsQEzA}lq-*<4t?xYWf6X{W!A2XF5Ei7?c(>iGMVOtwB2|7 z@)}!^MY%!&ru(cBm&N8;SXvt1q=%m4l6H_1x83fg>5FSnArH~o7mRh+Hx;&(v$J&D zyXPMa^s;F2wuz9J`5siaRar?#y_o#&;&HutIi1w^zDR=H=lo}NjR0^>qA>Ab1|bOd z{uADvuuA=O+YlosR>E-KpZET&wej&G!YO!GU*Rb#@Acom0+y_^b)8NR<=5&QG5C<$ z=|7JgLo7O7w#aTax;FEr*n{vqKgM^UMi;x5Z{g7`wrg#6B?EgPy*Kt;g&9p`)VdYLd26c!6N`o!P9Xs4l_oW!6Q^YJ!^&$pwoGfIfH z9bj>txd*2^VXO=z+-&(kPfzcvVna828%#aaE#UZ~cCYf{MUK_RW&P9J^1PZaj|S`G zUuZ}A_=p`ehHH zloxh&S0)E-N?_ojZ2*eHALJ`B4EwT#0Z@+_KvN64oSWZ95jLPSNrdzSqOpI!|JIVq zsNEZ+*vn>8v!H5`Z8A%zl@eibw~|LNL1X~z-&yUjw309uaeLN9YE^8g7-W~-L8RU9BYA#)WmSj(5HJx#2G-`>NnEI}(P?r0 zsXS&VmWYTTV@Xy0wcl;yX~NEL1Jkier&ECSv*W*tZjS3|QM?|eU^H^m)?Bvpcz%kX zh>8V_!(*CF7QG`oLs%4n4KG)~c%!i>U3Mo2#a=i4^+o&J?@l|!-9yAkvM>PY#X40h z;egtyPkxipMM**c1BJ)d^{YYCw76`FR6w5zhA^kjcf4#+m2Uf8in<<~?DhT{iNUWx zW`9M=+LjJ$NAH(ayN|6UKg!qdjtT8v_T0Kjb3h4bkB^UII859gFQ8hL!}99&E~|ss zii4?J=f+*nwf_G8@UsC_Rs(svcXfRqY$U`~CcQd0`PV=8cKwa3;J;C$X|geKB!V|3 zjtBGBHQqa3HE(b7erGut^PgRwT~$Bpv`w$G#pj)P*yzhD3fW380)|wdVL_7LpdcA^ zB&IW1ae%O*12SxNJ3a45RhzS1PskayKH5KuSf?r2`G zXDxEUYLh1B262r_1-JSMOm${Wc}crPOLeO}f-``bc* zF5qQ~_~S>l_ud|^z}vmw$vJU&z*z7R0v4>9Rf3#TYPOlv3||;pWb=j(%`IZ&ojk!q zy}Tg0wU3~p9Rg=P+uZkzll#ZlH(nnfnUO!os1`p@115s`aWw%9Fdzt$lj&c$A;)qU zNHgrk!M?js*IifFfGQ(ZIIMc?RDHeU>n+NVTpkZP#o}qMqMw7NuCA<=l@$o&Pc4qn zwxTsOm`>AtnA53ktH&dV#R*L~e6i8dYx1Vz`UnCCQBV|#Ffqwe<8iGb`d!wO&M(e4 zxx6qXvMHk#anaNmdCe~;>94gp@*u2>VD6l}Uk*v9vK)2rqe3wIUr(9bj5lKfvw|gMmw1wJojE~1`~ZLj?UrJ6 z&v!?Y{EANgNLzSjfC!*cOOF~nyD)DY7zUBttZqvE|8Tx)PWs=!QYR7-5|SPrT2SEl zD9_}*KXgL1M{+Wp=-Y{V43z1eImAHb=0$z=+BdcSBMx(U4hr^+L+a*5Pi->xKmtPI zT%!M*^j&LlK$#Y0E&Gk=`Bke7>}TzL+<)J#uGQuK!^&v&Zj@J@C0o#e5!Hi-r;AoK z3xNFBU&&tP*EcXkl2VzK-kA-ARBC|9?Lnj~5Z?z$mYI=EURKyt1LKPtsg)!{3AWIu z%xax3Zk$`|s-?S0?V8s|dA~zKWB~FBL-{}Ti#aVJ$)Z*oP;GY^4Ms$bw8s(G zN8mq{JfLzI)?^dPfTQ3t^ErQ;cfWuFcu-AHTZh2?{f3Y6AoO%rqs#Ae1te!?W^roO z=^^LhT4ar0Yd`*)c_vyU0}DdcueKYCCANj}5V%LnVsoyIhakgihhzXzgbh~+K7-Ms zA)v&F49%9M*;weB5>kiJdBh!X02DTG#tDN6k2Ur@Aq>Wq?yWCa?8|u;W?7p48ckz- z3)6r5v*Sv}j`807ezyVG{I^`cSgrs5SmS%@+Kw*C`m*@`qz``;!co>G@DOwD&5IOW zB#%&(pHI6;^B-Io3&}1xdZD_@hY8Pg0KaEg6g)%|#7y@>0taV}ArIe2QbP)0 z4$@ZYD(-nALW#ll|@8E z^zGcX*Q{yUO;q{};=zVJ`RmuVJ!p*1mpG>IGRDrraL0Mm{o!dLi7#~L^n$F~q z^6?0gdNtB}PQC!sg*V$=PM+GlV3UM?Ck6(RcdOsIH3Ar5o2wiV&CS3Xt&V$VZRLsE z@QThJRJZcCY(tfClvzEIFf?!pKgiQobF%a|uOFcoA^ac^nKvKS^r*1z-tURw$dwT5 zdTzk)k>x!^D7Vx56zZQPecQzi9pX53$Ldi1?1CX;#(i@R755E@4sfX z(;t6VwFe3nFbFP^BFjBFUN6|LJJ)V~Vf{_HpkjS5*_)scyZ8YDJxFLbbwACX5q0xo zWVU(itc(=VKr$4iDvf^C#vDrP2>4FUVl;dX3?v= z`$jyOvzT=E=NPaR#Ww0RKIU@ln z$wU9Spi*DE<>^)A9{f-j4;t*Es3d&fKI!`fFI5Lrp;4iu=)Ri;XZVG>KRC3ewH2QX z!2}`Du&G&`xc+hQc^%Fhy535d&l9UVS&QRTghRy%4SzxiCDnQWq`0Z7jXvG?f6 z`eJkZy=!@GAw0bT51s-{4-vk2?x3I$;Xkdy{>)pmX02GOAtYyLy0?IWJE>-;b#m`j zb#E;%KZUFBjgBnGsNLnfk(;9|0(VgJSDcq%BtGL>U~IgTTa_i9rk8$9(u8piQUI_;iXkBO0TlG~aAL)tpAb`&ME%2|YL>~J(nzhS zE?BIdT8qQ8wRW#2YkJ)}`Wc!)OM8>=-jvW%HNB=gHK#n2If`-Ugefi&28WKRL%zy| zpRlvxX{h>(JRS&}p+nx}J-0y~`sQ}^E$ zPfedcFoF@HM)!AF-*}FgTL`u%6FVw^{6FI`6&QU+gMu zAsRzz$tj{n9&jT4WR4k!2%xyQ7%d0tKzN~9n5A6OZ4E|twy$bxU8uWay}P%=FWG;2 zi;C0&|FkCiEm-7I2^XLNc6WDWDrRjWuu);-fXML$B2_qXs_`e{FC(U&o}8 zqb&g^*)VBi_5Ss!Bm|Ukn+}TBQ&c#}$;r)1d?6t=`>6$Vs#X{8PjFHvgpOIuzs#V4 zEA{4TeR7qH_sgsnQtB!{&>58$a z_oqd{7{oy8r|9gzqobqOWpRJoJb5B>`*(^?d|!+97;rx6+R6KoWSWyS6}bY4MsSkA zgMNzy6P^Cim8cxvchC+h2H%%6#P`Rm!E1iqHB=)-&2J{(D+>z9%%1y~$ZMyj7Q?nO;c1uDl%$YY*g@WT>{2DY??143d3nL*<)sVZLXD=@3ltV7H({le1^yQm*M?Srt+2DZ+#b*S z@E%h{YOS`Rj(7;D_s8(gV723;gWT2wKMA_~l|SDFObH+bm}k1%9gw=b5O?3ZD-eNT zo24&GKaV}}X3>Zk!hRc$G;-61ni$u#^#x}a=k}%;?xgKl_b7n82W!^qHc1PJvgyZMNh9!jW2{N9JYR2u>Gr7&Q5 z2#|kDH?JIjzo*|jIk+= zPDj;kpI6aTUY%K1>a=-H35`+?P~aXV$-|8YILw!;dYxF|B&Ui`Ux1GD2*M8(Q%|_8 z$I)x$-(JElw}g6#sD80_Pp9qpou;Je`FXejoj@aDhWc_O6ku}nqYphCpGr?2hy~3U zD0_q}8aeM{05}#>x$|D^$}Y%h?W{gZIYLs3>=Ul{Grh zLzRd~P9IK*#Hs2oYG01yPa59&W3tNK|!vew9f6&s8CAyyWikEc~tRaFc{3Tti{ za;_fMjTB-$JF)O1IVBYP=MQjMhjZ`GsG)Fp5W}KxLw9q9<7*_e<;XXBa2m2vmgl`N zGt-+Mlp4yC!HH^3Lk1q91`h@g>1!zczsPl7FFFEf&(fFqMb*r$&C9ui)&G#|eBHZy zZBTOkIWAk!_2DAQ1zWq>Hl`MYSbEkZN6!i52xOm8pxu!Zg@ycLm(Tf1r~0d|+BU7m zz&)QZzBkeVE)}ia_jGiP;Q$R-1c3mc`-1sftF&`H9%JG#XIx^FU%|c*AQG4o;14~J zm2Zg6tFE5hcX0n%d|fuzkCsP((QQJ>fFe*^*N@rk?oj?Y00Jh6ne3?J$?+pO*mSsU zn;1)DIiNX3F+s*{iF&&h^hP`Ga5Q+|fJ?V3M6r>L8$-LPzuzFp!P|JBkLUz7lRqJ;;+Q7bUc$}6e3BafUsJ50i+z^682`)u~LveQq z6fG1f#fm$WA}#Lj?(W5kwNRkAOK~ag#i4=X4tJk(zI)frkNnA6*_p}SGxI+0Bjm?4 zF9lMz1X2<{q-hYNk4caPZ#&UW3~AXEfiPl%?_wgTKwLs#5H>0jn1_q!M{#f7(>+>| zS&xyoZzqpbZx}tHf;=t;>T^Tg6W>L76(^Vtr5VJ|LO@xmEIUc%5iy| zt1()eTtNY){-DNd&zT{*YpA;Jk?$ajGRD>qPeiJGrG<_ep$h{kz%5rx|LA+pBNrS=RnJhF~O89Yu5J=-w~G6`kAnp*lq<0gTThCSN$~v51nPd;{W_ zdP#%9$AN{*e%?a4Ga~t&b~aZzM1FYOhBTxln&8}LN^n#z6lee*=ha9jrX5{E zZB$x;zq?1|_dg+Vv_4I8yqy2C=P#r)4<-~HR8{%NRUGL2Q#lMg@9d-MPDbk0N)Q~C znkTMi(>~Gq|8g-tV!@ZOl30kN01rRJYOC)?;&zod-K-ZL5+q6H+}&VDP3LPSIkoM3 zyQ;+o#BF&`L3upSJ$F*8AtyQ;GaGT}-F?P19UYxUbs*e$Dlc1zUx3f??|A}s`%}`~ z>f_{q-(6N@W>-W6U|Fa3_LI$@Yz!LK%a{x#gE$ZB5)WE*R3#br-Ks_@SZ$8^L~TOQ z<1_0ck|nPJoAK|zC8YWZkRq=Ab~CXgUp{Ar6X~EU^}pVq#_!FbDR6FFrJhO%U}Arc zOMqPXwXO_F*Afk+Ft``a?C}m)y1=-hR!4-Xiz)(wg8rh~QhI%h^k`+!(!m|DDmPe} zoCItZk;F-_M7w@$MqW&#f>5TrtwU-Hvy-kkK%bL;_+FeIU8VxdzcmO;2HzZ)g@!?O zSfB9fib0O7Lo^W^6AAk%(&hQ0$h+~gZlS7 zlZEg;>8zIR%mI;aBR4l$d~WN(7xn_{%0`Dl58=$C?LIIH@A#WCtdxi4u4h3F{h*JU z3_+l<3wV*oDqxA3VS;#-3>vO&7(j%bRG#zdbF?Y^MLvZHf>cI{X}(HLM(vJO8=M3I ztnOR%wxjg9pS&%+(fIK5s|8l)2N5BDRA^o`Kr#=<=lD4-Tg*~OPuR)!G1 zyco*?Ko5-Cgkz@+nn+QzZ(Wn+ClMV!Cx|cFGx5c9d_1 z#A*(2N0-zEb`{;f@2CfTt~7r-f#kD<)L%rsc3LwZng^Y*z2UV~w1OxkOJ|qAmTnT+ zyn}gEHN8zHMDK)jV~Qr7ob!ak(le@}N5=x@#e018zT!(Z>VEc9wwI*`!Ocz7SHIO& zaep<#7pl(4{U|V1(MJ$u+}oY_dnz$7G|CEnzG*4r;~(B--ZbLfloUcL(K%x34L%m{ zTLp^k)e`YlZ2R_bbY*NXB$fT0K0wVktd<0*`Ofu-r$0+4`KHbmIfG$L@cB)rBd3eZ zau0h2^sCvDYu}avciI5qE=9_&pwL+{6K=tb=nSOJ>`aQM73!zfx^~y$9e{yLwF*)KsQ`j+cnwoEq=ViZHPp@PF4k;*zyE8Ew zp))`J%sWOU+F>#kceYi0r$5vgx-5ZycZg%4i&oF$7hM>RUWwiL1b`YVFmQ5e#O7m; zbwav_qvKyAfPPN{_3!#Eh2tM+V*d5c=yAZ$Eh%E}cNk(lt@ZV9-cpr3Q5WgJ2#OjT zWDes1Qz~)(7&+cG_fu|=R&X^xDQH+D2?DJx_A7MPl^WKV1wn)9Nl6ILE0DL5``9icCuS*U{(ZP@T&#QHn9qKI zYgd3N{%2x*t6K$mHIWV*2FRuXZ*}uth)UvH&S!$k!c^C&g(&tBPq*k~g^j%O#u489 zN$VYHG1UB@k8|Rt?EXM@lI@2N?Qz%)s$!^}jNJC-mZGsZ{wTUs-3AG5h(rL`IWf zslbYj1%B+opE@IU_06A0c8ni|ZS(ed%}`d5mFswoLnUNYpE?s|EtH5;M74KpF5IXb z>k`|-gj=!R*hT0zd#<}-ii~al_>p&e+ZeYTWOwE(mr|Bqjk8TY>eK!8CxjI6QUlMYU%JKOx&;gsP{+KeLs<>o!TvBQz@+VCO`S zZ{yJTiting(>ZAyA9X6vp2@kH*A24yg5`3HoEp$y%`C;XjAasOd78QAb^!5ka;FY_ zMsy6is$hWDl@XgL?|CYH8nnmT2F#ceSw}6t1BCk|B4_gxw6L+If}4gk1lr#ew}Xq1 z{WJFVb3^%)c@3Q+&QEo%4sXP+h0m~sc3(V9WjH_*5*v_EzhZYOnh3T~?V2=QiB;N- zBM-c#XA}O26?yT@MP=5EG{qHZ%K9kl`z`%dcuLN7S31cf?!x(zlVU{X23u9q1i%6% z%W%obZs=4oin;kY=rd-*Rd<%BdoC3BiEEaE$1*!cm+wS(EE9QpZ82`hReTejy-QZ< ziF{Oi{ra_TJEy9vkm?x9;6?%SRH8($k2$ruYsqm;W$~-O`lC4~m$;5mv0fQUS!gYV zXI1Ni>}2Y>P?M9j%q44gKOd^C`CaHCFwTnD_D$v}$?gX)CBb9J)GY^Y8b02&Ahf zF1*AiIElE^>^rtFypex@qxrw9A)gJQgmVL;(%Yx2KQ1v^mez;PrIu;yt73nWZr9xd zBer(vi>dXdkHxg%VmS$4YqkEid0dpn@m?sjKK;m{_eq`dG9Hj0+Z^oXKD#w*zo@uB=AutQ#V~Bi2m&u*~9v@^HRLj zU9O5iocGvPAJts4Rm;!^i4(r6*RQAwzH>w0oW664BXjgkCD=-6rM{lY8 zIVOzKh2dC3FR$>FJoBIgLJ0l&C+j`$eE4-GC5m7?GF_r}_>S@UM{k)uFFYfeMBQO!JKw z>hX@9)Uu)9_#&z84K~9g6Qv1c);)Vgq>*}7YPXWnGFqK^f8l$5q3N^;iLA^bDu{(a zhbE!x)-do($ma74^aiIm(pc@Cbk~j(y0iCnq#)>Htm+RA4Ea{(eNq_0 zfNwfW1t&eimAGCXoiie_ZgkdK$QJ1SY@GA1F%ig|euWN~R(k268i+yhl8b;Y&qT~L zbP7NE6=_x@z@mD`K@)1jD_Lc~Jz|l5T)*JW_^3(1MqdDvGi4;44dwk9G~Su>{=doX zB&dNz!(i&%2@G@!aV^cYwScN7q_3~U=Gpm`p@iSgY>pg3h;dJOE6Ir+ygx9Cx8*l_ zZ!pDc88f<~gb0?t&J#+!+TshM@XFtq?$R(Bf!3P#lbz0$D5kqbI%(K*_|D#*wYhg7 z*={v^BQ^>KDc6g(n)cJvqsV&ueA12Ut+!TQ!`n(EsCQwc!P2yHG?3DH+LAbV;nH6V zl8a9S@WJGd+56LjBd!^W^pGS)oL~w|3L{SFeL#6e5clSz7!Ew(>r+ykY^y8v zL38?lcXKXAt~dEaFk@EdU(Ytl>#jp3V$N0<#Nz(uKnOHycvzo<1jL2TnoGLb&Jc)F z05gHI29X7j24tYXc#&Vh6DvXSood!rLQPSvspM+hV0zrp^ z!vj$A0m>pIunUgbML$ot9*OULGlP_eiNy#fDFDIDEmktk!5|b=hW<`}_y<8CBrHUo znXk2d8R#sStx>3T>#6BC135ZIzSoC6Mm`_67ja~Q)OnH^qD1@6ga%$R6OsG-EBkBJ z+ek|XfIcizY4T=ECSKTsjJ|PXrlIGB`6|WyW_A6OawV1EdQU zzV|r76PFjPK~`vN8HPrVsUqU@-vTr#rX+Ursx!4A9m#BOQB$w+y~#$Y-%VMQ54-^4 z|Ge=C6Bg2*?hKG@j?1f<2T!nZaI-}|*6!Q$1`Nntt&+HDR~hN1?=sM|w0M#qF9V8A z4VxQNUgCC-lVAJOR^3Bt(tj4~tjs2(ai=XP2e7>yF;Cjc5oTnpZ|*mle)qdiJw|UN1O!}9wi2Gt z)eyL+a0%E|x$hUJo!%Sr3avULx+fh}l@qc;zFC}winq8X6K=`(6x0E%q>c$F^2_Gs z`k5Kcl%>-%tT5DDRmz($MO7=wfVzg|L9~uZJylsomDu(>fNeqE^t^@sgF*zjlUGD* zG5&ACE=}+-K{B?KGjYEMygLrwH?2;uLehuV!5|PGvN@rs+pnWVUAO&_^2gO&{^zqD zCu1!}j}OOgdZN1FvP!9{F)g62aBk!O$QZ{GOy-S9Gr-k}QE>pTG@FTdM; zE4NHEkmJHsAHil%7%qjQqPlk7LyX|sBj9u-C`<NPuzAHluFFex@MWS|BHZQ@4WC$d6(O&@Nie7sy$aB#9a zJ}t%SK!5+fi9#lrbUB=Bf4n`kWb=HmM+2QZU-?}21?V`MUi_ZYzSN3~j?Qjg)OxFP zx^mp^-L}yku;k`#=qAMX6P;dP^uTS_Va@q;{{L$VIr!Zz75tnRWA64gGLjDcK?ena zz}}Y&R@3dps-DsR+h}pam^zRYuA}ZA4ET^o0-^M9E+{Nu;MnDOzTwEVC!(%& zym!K-)QYdsTHOEv95!dUc)4E3JacYvQzj#j8><&|b7=Cpm{L=4y~6&m__gxG!Y6AA z{LaKeGqaafqgAgwkDQ-9{TYy4R@a-EoWui%*gUR`u%X1nVXMaWg37F{Kgx8Ki1G4+xpd)6muRI6G}pVp5d}uBoW20l?luODnVC-9X*4 z!QOcVJ-z1v++1W2e}Sy?=CVqg-;4z$l{mQlK@`wS_D>9;F^MXT4rYw08dB?)+gM#y>mP^ik7e;rFNJ#f=NkmB2eMk=wgwkbhS= zX4+Do9`M^A&l8?&b7t)c^Kb*^X_AKRrY92@m;1PLCs*Fn5278uC)U`?gfA7m*-=XA zOEcrHfcoj5UD1hC>;~vKvt@U+q*5qbZ|&~x-bJs*5`$=y@hrZ(;k17J_RBW>BO_A7 zQZsmN>+L*($J?GI&=_DH&~h~9fp9=l+{2W4Ix?n(RTXY?nfNtmOmBA}oFlF8MT-jl<5zcW;s^wo&_5in(d|m>!@_}Ui`itj zxrToPHvN_kj5t;*c8*SmrgSeIZ~J=!2!w7}Z#t8EyXB6`hz6iBqDJxjQ~-Tcr|!Mx z&>+*DteCW5t|5jgc23)ao}gVKM;=*hmEJe!-LahF;^IIvDm+Tt_o(@gkB=oKsf}w7 zQk#f=r8JI!137T~mbAd<&)33|OuJ z4iGm8ubrNUIS>+)w4EGf`MBA8d!J9x@3dYHDQRu_`_!F0Akz^hv$>z;2;LnnUvCW- zU?iL!uyQVB+J3g#6og`^7c}P6KeaU)*}QLmxGIuYE}1i2Kn@2JW+bReihea;m^z>o z^*tT-`{8@}H^Gbvhn`mC)0VN7lZL`IX)%Zh8x`ZtCA%u#;8(vV-*jSBXx^K{JL51U zG1e@K|CZ%(cGXYF>&b%&5vG$RB_-l|0&M!Vt@TziFyyZw6G!t;ru5L*e;c{O{>Iz< z?tN1sbem=-N$?U;3u8d`1up>6Ai<$cIQ5m@8}suzLUL?KzoiXn1tsYSiBl9dMBHwF z4Gs=|PM}7Ic!Q$xK_jG1-oA1)@hEkU=;vMK;+A=x6C+Kfa76*sC^KLKVNTCG5m-v5@Ougm>1YtFbFfq`+&ln?TX4= zbd@Br9S#D?uw*e%qW&{rpm|saXU# zEdddF1|5~>olH!oTa0w&dlo!&8|zcWO2Csjzyyvg@Z23coU7idOW5Uq3(1gT;pgLP z^+1%kP*#%jByWe<>)~D86kjcl=!8X57<%Q8A<>`Ebrl3#OUMF)xWo z3En#;VblHgs(`bm%z8MLecJ--ncu^9r(@?XCcc#2!#`JnEDWAu5}A031<}pAGh3?k z8(VvK)0eF_o%N@LdOL1=;9$P<5JUPr$Zq)6`QpX&Ux%dzyM^w=4!_pC(ZTE9w}h=g z*Tta4lTw*?ZkH-bR^i7dr>NO#Lmn}}N?@3TP|nPx_^l8{tMA2Pl#2_utR2E26zXB< zu{k@+-12L>5iK7GABFs0nzX4e3%P_${ksO9jhsN(Kx1V?Z#ko^fs`u%4g68(JCOQ1 z>+tX}!rI*W`ZgtM(ED<^uMEG%_C2Q2GB4l!6Tp5DXlx+xgz)mP@DY-uW9o3DOOwDy zj&$fB-h`I{&%)<2NAC|?705=*txJ^?3>gd>*u~N;3I<|cmVH;Q_B9JPS3#v6H@EA< zqO5>DRv~1=c-{5zHR{WF5uXEho7Hx~Ri8Mz1Oh8g{;L&f#*jA!GV55oWBFv2HRUU9 zoDSy3>rvy94)*p8gHjN%6buC6fu(~7AvT3kcnOOBWH=#_b5%D#(WvE!qF*b!x_qaS zxbB0m<>Mc1bQJ(+%Nz+RbC=s+5g|>Stv6#{FUZ9_jy`QR9fm#Ym#>DAV=vofs9D1* za;swxa9DlMr;YmIK+u@q{RvZ56$1Y(G!(5}?0PiUK$NvXGk`26WZ>PRm;YqmW>tB) zu8uyuv*+Oft^M%@Q~+gaVVY@l!reeqb~M(2Fs6ZT^@q!@K$+Ja;|LuEO~<}yKxxJI zjDPEL^3jp?1D7n|ST+`C!DKvh5cvHAm$lcpnf?K7g}(N^gK;vWf?T{gQDdz)FC^%z zxji7USKE`52?C4-jK8|bLLnS4JAs4u7|{KkGop;}`U3R_B}bybT7)n`+|J1UQAHyk zFmXZXTmhtNg~n1ZCQ%hJ-geir=bPl9oCF#L+BtZ&?2-D{T9QWS()=KV<-@<$D8ZED zndy0z!f6QL-TLrBFkHu1YP$a=2y-PGwjfi|>t0C*>I`O{kMxaY&|7Th07qa58GdFn z{EG#NpWy`w9m~?PTA~V?oC-npy2zu!I~HH^!ccgNjHmek-h!Ea ziP%Bovg@~C`Wjn7V*WBnMPMVeYzic z)-nPmh^q=_=~Z311Sv_rL4p$I5fXqVA~C|(D$Pbtg8X3_8Av0&)ohP!m2pJi`HxF+ zH&_=sc+!70KI{Of+CMuhlKTh;T%{|=(X&AoB^#Zt-XLhFQYE)bThtei5hys-{SiUQ z1#Y%?c&ezJpw1V$DXZcPva7;BTT`D$7#pj^y@|DgM7pJf%M!=VF4#|c9H00{ntNHw z5XM4ePAqi~PD@s`y;0v#8kaQna@FPZpW6^W<<$G)B}DLOg-z)C{||dXj1=+Z6Z2l{ z(G@a8@PGhC)XrW;%%IHqfaC)i8GgCD5Qm;#qEWM2mItVudQ|qd@m6(5^p$uYi(5&a z(4oF?mc<`Tyi&!d1tzf#vJ{@db19)?i_jBE1NyVF68rwn6N~8!r2kzESu7L>{-#nn zSvTbKkuXcvE4cA`W#-X-Vk5X=S)1qgM4;XVEx+QQ9z(KRF6?DzI$Nm`_h~>J zzdT6zVU^hc`ftiT4l>+`{pDX;BHapu_uExn?3Z?)SO-=~bwuw@@#v}#c0m4&P3>9zg*AJ*nUK1u8! z7N_O?kH^M87uaEsn*p!?QXH3cl2nq&#%})m;f`!*c}d^#pp}vvq?=W`;?Qtno|X5( zBWVhO8oQ}w$Mfmn(dv26pFGQ~u%>(j@_%2%cSc}`^uK!)m=e0&PE@$#yOVds&ro<7 zzOd_}Wc;yoo#9Ae3GgXYr`8{yL~kgnN~50q z(p_{dd81*2(t31JvZK={@PHo?P^=yE+Ch~W9Vq}Bt_H;ncoi7I!deefz?>cXHu{aq z@!-1md!dcf{`;Ml^uyx3Pxwy7lVW|ao;vOCw-QH#+_=~1g>_W2dBD{0vf|2W2*+Ut z=)S4fxPKL16+zw@U((=a{~U?|hl2g%KS;l0SR|u=yq4#kMOxn{sW$|Pi3|Fj;s*z+ z(fY5Gf-?73n{30daSWQAIWP^0R9R$T45qSuB?|P;yRO679On_Z>AE^{c#iL+W%|EZ zosbPv$ohxNG|1(Jm|G$f(n789PIN*3V7XqV_7I7&f8qBdI#{S^7+>QC zlz>bNJRB0?5LsiJ!ag5c_T5R;PJE*(ArmCP{#Jp$mgz;052GRKtRF@|h`M2eT4q33 zfCLK1E)~qBF@7rLERR zzp+{dBS@g+vzn)Cio-?LPtfV@3{k%Ezg$Q+-qyZBc)BmEMazy@e&2$$=7@d{DU+mSj5*; zN}LWgU(TB#@t6>>xB+5Um6Dp2lxwo+&DxLDjlc|f`2jNmq&_$d4gBn1M&%+o|mCRL$5NLs;byc`$6!p#+JlaS3%iNS9dToU-Dy>B9C^cGGPAN zu`=(xCQeAa!};!eSw+QeyO+BuvA>O{tu%}a5EbCIKkd|>h6q+0iMsq2`d3syCIJGe z&AwNK5t0wmS1ByWfYrX!zN;F>tqkcEO7i4Ilo z18$~8c>*vB9rwa1FA@~9i#{DV0leZ0>Cq0$h@%vBJ?drvl`S^=Zr1*GXuqYkPW)-= z{2{w4+=?E6Y0Xi(xfYCJ5tteDg{GE{b@0zWoUm9UltI|`(okjWp#ymy7P;L&7y=l@ z(#UuTgn?vh=I8;5xhclF!#G2W6!?M04efH{g9sZw0N`QMy7g|Kozn8fH`)^M_m#K*2cxlf1hf+-G%zlg^~*x&6}ICLzYYy2=y%xMy!>ONgD^f7HhL zVfA|f7n5rK0ph6H^invU*k3NB3P zI4Tm>J@=Erh#6A z-OW+Ec$09h0<-&Be?5VGtgH@8MxB|w9U^rG3)4s8;O`_sDaQ(p!23prii#?Z7cbPm zP4ueGZ8*dUx<(gle5tCQH~iL=pKooQ|MV-N=AG!RG(0>#z3l-lF1u;*yYF=B@t-e- zI>`jjRI(*}F3&?Mv;;m2m#Am%!a5%x@)+9-NuQ2Ik6XHMfnok>FiXN{xB_DHJh} zUE1ZeQ-Rg+XX;s;v=wI2-Lf`)l?aRZR}`C-4!N^mj((pAX?aQ^L#qSDd5+ldsVst! z;lrgL9va$bqwNih&?aoaU^5NmNrUsC4HQR1Lh zu7;NZ73#swCg%2@xpF~;ulz&%ZRDAiitEKQ05W{6YFrbLT<~kOh$l;BHxQGM@ccW> z*=j!?#fk7a=_47>0RZ8x^1k7;x#MO2sX4c+BZG!zuXsdbOAZ_A*%?-YeHA){NnTEl zaEBWcqEU+h^ACZC7~_M30G#(NPcTV9@O+!q@z|G(U6s7OeKOrz_l<7KEIfio{sx1< zU~?|gV2ed~a_^gh$%Un>g)}3Xpd_G7UTSok`=z#a!`g_{H>7FX9dM%h%vTu}%G6CS z^V(p=?dliMnQAC*tAVYZ?YJF8VV#xPevp3{%h7zUN+B-7#iM1UaS_zGc0VfL5hs(L z!D-a)jp)We6($VdCQ0+Tn$afb%n@_n{W7rmEVOX{CNtM?g7|2o%5AlQFmkpHvDQKT zyXOAx(Zz;Om0~#iV#K(OEkhUmL}xb73O1owG&y2)SqB^I8O;$#|3kwLKbRP%MNI6%Ea-n;K8TO-04R zVtaIWmm^0E53hZ!y8XWJerWNwq=6Gsr^o|hF z1tF@~)z4*vRj)BCzI?Ga681Zo-{9KW-MXb^WR+=@ECpiWm&;!YRvQXTI=Op)SgG-A z^}d;E2gJPLs7zsKVYJ-@cP-GlE9dyTB^tU5+?lS^pb}TGW zK;cs=1SEdcg%3tuBYa^eOurm#JcWke(c$k6^cB3 z-DPgOnw;;}Qy{smD6nrkm`V^O^jjMs)jdcH8OdK68QCXh^tD6Bo#FqGP-hn`sc>`! zumY`&8g%GB5bRf7Q`%bN9sIeo&Wyv!p+D_9Q)LC*jnFjd~7$3eML>hH+Syv=%}DoY6#3-MeDj0W$av4loZBcVlC=#NsbY6PSNpvnymp_ff?ezMGD5FLl^Feh}&?@ z<>+YFWxX1`wa&J%`s3y1{xmRNpY)P`>2vCPMMYXPEfJfn?dI3K>Qcb4 z=uE6I<>rL#I?fk4zVI$D_lc=J6|WEv9n!twXk$|@!>9)0u+r#b3*Hb?S5za6@@;;q zsNIg?7j?bgN_%qL{IgLspz=ZsWWL`$*+48b!T?Trm<{JV`T35UwuCuhEH6Vb0EtWyaKbqFaCvz z4_6vJ?TsWo&L<$(Mt&C-u>&}IZH#v$8Ea(9gCp|KsQ-Mh$5fIGfaWv4#6xw6w87Ak z2{Hl*$BVTLz)?iv`fm_`KT(T%;*yQV1Eqq16L(5BtZp=!O4g>BVm!vt0CZC5_Z?D4x{^*;3l2cdJ*%qBzCD4r( z{XU2vpHU@3bs2W@QQ8zoagKFdG|NA|fnBve4hmb3KAyZbV0yU2ASERW0acf;l4GMf ztl7x-8hCvvDpSE^Zg@BQmn?%_l-%rF9FF24z2M}(D1ts*c zYVipQBF8HY5-AOp_V*V~GUumj8Rw#uDE|F;AxQ_rFWJH%lK!Dz{ZkFjLWCKN6&0z7 ziGfW)5Wj2&7n_9Uw*{)ks0K?XFtBO~@be>m1ub`hs7JpM~|0Re1 z!OaT-n4@y-TWhYDUh-D*@=L=ljdNOdl75$OV5BC#2O+XP=vGfQMtdZ<5rdO(V%?BtiDZewUZ81lK7l}u50h#oaaT_-9L+R)^qb* zBS!tljFoF7`E)}P;carNZ93364K~$`Mv#M;f{R$i3P4FKH49Tf}7Lh`R0c1?*z5)0H zB81JK_D_es4orL5?N+GL2G4%r(AvxqIzIX zB7v5FFW737hVQkAs;`-q(Dccurg2M?`5~02y>VyBEiWPDdaPm%*m+M|pvl5y*yqmQ zjf6Zt*VJ%g<_1ShWneMOwpcESB&L`r@6?XVd z$aHmIb<1@j*l zH72{v-^#v3jWXJS*WNY%ZY_gEhX<0)V=KSDKZ^5P)ir9dnBKX)^`xaDBX9QIrUmIY zeVR#2GSb#O12Uq!6rA0y9Y1`U+Ps?k^^1^`nFjju0e}eZUqB-Q|7J_L6!IO1{L=xT z?b5K~6m-Ec5^_TwHTHRpIP&qJLq^U+tgTH=$;Ah`Vz3|j!K%97g}?GIPSj>j`O5{a zO9??^WMrJH);)_d@*W(*mhDCs7x#O*8;D`2ME95@=ODQ*Xl}NjTV1vm;^O+nFCh^E z9~_cVgeA;`IF)L(HhX!E#$MK_0F5jG;*7GIiZA7BK`#V^r$o7g0)gOzAs2Iib`~#P zK7qnU>TYhmJ|Sq-(OOqqXlk1*BSiXRMMP>ZQ91A}HUl8rKxfNPp?aFF3e*Iu(lHkD zgF}cZ61~pi$g)$ZRRfCA{@Lh^oE z(CDb+6T&kZuQOxOAHu>i0RhoqKmtI}z@Kk$_1>8vVa5Z#OY~5%LSn=qp z#@YrmK*HOxV8_ktCqjZCEM<_F3Zxg*Ngt&CN-eO8npaUPoD{i-8<6x{Z`(O(*`le< zio%4?QeURgLr@ia!DJcnid=Nic(4hg1XA?(?&0`h?~ZktM8D=>-yo{`Y+IrPS9Q0Q+0SaUI^1qBoeFm&MD&bDCDuk=%*GP~39xUgG*E zuHRYK8n#O2m4pw#)h)nN4e##j>oe%$7~VfKOnI7?#?g53z2w#XbT$>rn%@^5{WkkE zK%{oVeFVwY<#~gbN`O^|A1j6l9(?hL@)66Sn~aG2sH!VaiVO}Zo4^5qmCf-%V@?RS z=RsdbdM^td&{`8I4YIv0Xun1kfs|qFL7xg^5aR^LF>oQH`A%>EgxOYX6sGIs} zJ%-`vMDp=yO?o}X-{~9 zrXhgQ7<8H)F13;UyAJ?Bc00t~A8tpvwiM#7@*Xz^Vq)Cls6>e@+*Cg9``w1&s`@_8 z0k>>u%>B?ov$#tN7Uo%Z2|3xGq{~*t&Sh-)_qa4dVi{$Av3alCzuN3}IqGgwKs!ye z!GUAw3<$HKy4i8mlc&Zi^1M2V9ICRo3Hi_;OC|2P*Q}gug>T+Ro zAQq({KmY6Byr+#eWFtS9`j5s}t3LHvT7)}yDULWe=Q$$hvQD;EAg9kuSHG(>GY=3{ zjecbvrF=&EZJrOElTpkD)Q_iV56C`7ltLeBkvN`Wo12>vSy5-L$jEK-&nVy99%~xM zGXbUdis%bLFQ43_fNzOlzr*4ld0cADa4`ZgY*Z}ojbcZ1EdWFD2Pw)(y?c>XR zdOcj$vAbaRHPL3}_QLI0U`1Ypxa-&aybvcF=;Pq2p%lncxvVpV^0S31cM#&;_~jFW zz{@uaRDUm{BBgpm_O3Tgmd>l+NY?}xjYT0OF+qXKO3JE=nuHEX-x{w6EBE*|D4M{? z(8|cxU=GJ;(X!q~c$`C<*gXuX@L7eZy z(Nzr%Udz&3x?VlQgj+6!5TcV|H?ZQ;?*MA-xQtCNffHrR-fZ35u+l#2cV{vt4Ng`x zN$aHfY4CK$Yze1t?|bx~3)p>-^l+Q2ZoANY-XVTFx<1i+bh`-r&xb~ScLUW>B!B4d z?k1%Nvr>T%^M7}i9Qp_inr33m*~3eU|EO6*6+m46ZpW2N7v1flyi6VT4p&6PZI45& zfE^g>_qq82)yBrgTg+F6eix#$H+}dbSAV|LRF+G8+?tqIb9jhd1Y9ldIvyI=;$EZ7 zRvL2i^Lv~R*Tnz2z7_&0uoNe2N!NK$?V2%o*A-P4GY@i>m6f4Gz<{3;CKiS|51Ba_ zQ<|nzhKp#(f>lc;HiqK%lwfN!Gjc<8|ch{XmQ1FLDoB@ieVzUy7r>K1I0@vM_DJQ9i2jFD0j2vpvjWzQ1G7Fi`2 zm!S`!smXT)NDTpSV`uAYMM+4^j&N9sh0NBE5of>hDZq=U@xwx5JJJu2!+s!w_YFZT zL(L~Yu^k+L2JUhD8ika`#HJvw8Rz)#PD=zihNIk0M1dL9(Foob@ta8APAx5!<-WDc zbLXrjS4r1i@51)!#sy^LzFD6nezD6s{nYkNr?Rx-%)w7mjO=YNm3&r4uEM9g^ZSBr z-%+;S(FDahHrhZ!Z2a&HMyCD&qY^RCdne zW?oDMN_IC+BO!v&)}4m2`|j*fe$ndH1x^YnCM^|SRH>~K&rhzJc0XKhdK(8~`#T*; zAXh9lg#)`MN6YOJAD2_87fZ1T#C`o#T($O8mV52&3}uh4fJ^XfDUeZbIs5Topy0NaJk3m{+Qu>%AtC@3s2yYrJI zLl0~uoqB7D4aAhq^jaIqE17|?p0x;4uK zx>Hq{-8LR3;4mdAxwhN9GX1JEI1%5YFrI;+b{cF=Bt~pz*7Xw(7(t?=u=1tW7~ZWt zNWxBdl1R+37vb?5k=@FppXXIwTsVTKm#w! zeEFUrJi9e8J3D*M$m6C;+b~-C8OU9>GfgjC$g=N7XGVcg!|J#b5a5eb;g+NGB4fE$ zdPW!K=fehATizXWu}U{eyI~tZ}PE9MRqX44-`I5F`Z8nRy|mtk$%BN`34!ULrwt?}CE0@dN#hB~Br7NEnTM{3wr-olAg~bSWlzz+9tsb@dx2VS;sX&m1eLA{yXpPH?{5W<3iPCA{AAHYglTjNNc3IF zmHwkYfBtmD9!jl54M3+~NH)e)1BPZoK+LWi!<<0#(fhqAErmKtUsPyVn0zQ|f2gNC zDw^~LbC9!8E{qFYWQ=b-WbQxNimEHLX8Pt;TsWDJGFau7aB`9%KrUBgs#uz*32o>9+J$U-%2WCOY@jjtAjJ%!Cvf>DiHG zeXO%V@jz(t{{7Ef!#Ci!eB|vj^Ua?|cL4j1MO}{hFT@nMn#e(*eQ`pzXg+zgG*IXMR2oO`o3cAOy)p#An_f5L zIe!)^_{OJ?kuAp#=y3=9YySYw*A<(Q4ak*!#ZcQs!px`YjM^> zVeG7l+Fyq`SCcHeuLESGUR8WMkmzMDTZ+>2cN0YFbB~!+7q-QMG`PdW34f zJ_QCO5T~)_#evZ?)PkBChy6vI8+$%LM-?l$d{j1n@(G6i@DQXlTRHAlv}Ftv^4NIi zgs3JlqP`9@Bn)aiRw8fUN$Wte=H;hv*s+IqwEIuX>5PVICuhHTbj{=qtdG1)&A}7?CcCVbaVZ6TyHtW&~-fC_Nk4V zqCg|<2+$rxF*X6lFrA3;vq(IJAGch!ifpQR?X9MLx6}`RdWLt$on08D zDiusFW+H%+p1-JTM^vzSrI}86c6!CI<>QR1-^n4miJ!yn+T-SCL#hC%(X@SjB0ece z&q(iXs$D!l!QH&f;ln4|SKil48tSfy<))?MD_5LnNLhn2GGTE!kzcKL$5wnhI=rl* z^N?8X3Ie8aWjS}Ni`f-X+fnQFqnsUELA?lc9mjX|qK9$O#augBgPpzkjEp6-Rl0Xk zxPB;vSiyA2;yHldvmal5J=Ue5e*4??H2!+gtJY+zP~;$FN&*6@pP1={m;`JE8Km_7 zP?=rTyqyV|huBQY3(ojR<=D$}TpYUE@?RO#yQ{uuDU|#(fWV^fgup7$WKp-xWxTps z$Nb=ErKCdesMaP|*h_*U0#RO3xu#2^AtWI(FUj`-Q`GOf#h3K-^nY^H+P>C#>h)c& zco`b=lrqYbKR^CIY<*=|RNouzfFLbM2uLH6(kLJ?l+xV|($d{Egn%?CATbIEGNg2O zOGrz1cMc3NbC3T1_qkv0e4CkP&Y82%KKtG8de^(w+g$ax*b$6?XyM%^cW{rjgM(Xc zRn6vaa_9w|kk}8qr@MP{u4W&6cOuQqPZ4u*>4m%*4R*A%LvsP|e)u*9$etkXbcTXz zd)(f?XQzHN+MN9QpWX;mh6cn@FidXKO%cBsn-C7M!_1+iitdW1Lw56)Q=u^Iu1lRzel>!P7Pd#?H zHvQ0zN4h@xvp@RDIQiPC6MkIG*u}&#n;Zr6OW6s8X#tDX==4X1PF(0a~Kx z^V%5%eY4Z{qaJ0w17o-=lf3x>xS;lNeacDS%NzRwfXLtHg2u-Lcf7wnz0+gHfo zMK;gigXEKe?ftuXU(aeq?bm*0^GLqZQID*;Xi9NE+#JAkDmE$V-ig!hXRes(Bz=q zES>d*AO)I|EpDwJ-%$Rj>YYN76R(c5s^8gBibLCU{Q#4%H>Cr*yZNJY3%Drx$LhSg zYt4$>6$~X)_mt%i{u%Lwm%@N$rZ9g$EgNRpxOkvTv{lag$gIH`(g_=#T`)uQ7yk0_ z=p~|3htX+p2^fdd86W`T%TXsxkBgpL-ebNO3E4vWq)xpeRwPFw#>K?0^~ag_vSBz} z|I2&KC|~$MeS8rVK=gPM#{m#!{%#_uc_?q*hxP1x@RwRKB8(_j_`#xnTh$Y5t%0v* zB9}zC534l=dE|8ff%ga{5Nr5P`w@v3ST(1j1|;dToN?XM z*3i*iZ=2C*3vtS58{v}na}X@^tu>fpa}FkL$W=+7FGy73(3J# zef5yum;bX+e0y?Uf440N`<-=aEODqYHKrN*06pJgg~Y~pMRL{HxcrQjLSrQ+uDvTC z6CYRSo-G1lTJQC2k-y7*!cwrat>s<2uB_zRmnjachY0Ikafth*`Q|*0igjwag z{c8I}Yw`0e2Q^^#sjolV-Y(G{``kI;s7=|1$BA)$EYE2ER9LwP==HGf(V@hC*l3ED zS5d)3irevk>BV13JsazLJ1X*rvANE}gU{O#tG`TxPj_oF4>3Tj%8BZ$vp)m(=ec!| zyYM#0V>;a(^d}Q4vZ)w@fu`0$rrIkH53kd89Y3}^qIVfc~5EI13D=T4= zJD$i-!vDF8Xqf%L_H;?qp%%6n|JWe_gj7|X%O^+yjNv2bhh!7c?@EUcn0)>J${W7;B#sxJn34&8 z%y%!0X((9IFzM1oN()kN@ys)XslC`688D|{Ce5tXX$Gi|zCj6;S+;<>(!8@OhBCRq zW|(Ap=cZ><5aQk&<1F~@{@g6QeY2>vtZcTnL1+{#@W=gPr|$6Z9zSyRO3Oh{v!zsn z4=Un*G`}tzbb(BG+gH$fy;l#Sq|2cvb*SHe4|~u4ihO$upTHBSy*HP(WR}K-T7a=e z?P>1|DW$&N--59!@IEqDJm+sY9}&@cP@J6|x<{FSGbSqT!UcUzqg(JYU9x1`@&fcR zLv&X2kXUtThIfMV`5`f39^a|hmXRNb))NKP1d28o>?~;m9DM<%qmn=U;J^SSC2h-u zS7D>yZHfGUw{PZ6kxy|rq;vy~l|4PDcJpNWA`$QRYuoOxGTTsx(B21<80S}q zhBoopq@W~;aLUplaJ*nKnH{tDg)4vh_nWor3Bv1v{H$%Lvn6<+tdo%8E+7`vc}ko!h&&9Cg17E>A# z$?>Z9v&4dL@Ns3sQh?O-fwaYX$B$^PhtJHvNXVdnYhSE$+1C~EM-bAA zMU2)xxMc53d3V?;{y=`_|(l`T%*+ z@&%^_gO+ut?X-E0S4NKYM7ZI3N$K+`dB4qw>~%SpwcGL>Br{f`WZyDN4|}nU{{hU3 zbx)R&h_s}n#~{LQL!;b7=e{PlBY5Sl7co-uQcv4kvqUT0?MOgG&mbK|e#|0$>@jzD z0mUDVTkmKdl83HJoo#M~oD)O{&BTm;@x{s0>>KT_C4ZCqHhpvF`#H>BAc3$~mh*T| z+YlhrKt)3dBowZJ{%#(A{yd5D`KWj-?CZ{&EvWX9pM|M$5{Dbm*M(i*#Y~lBgs$r9 zx|*S zKsulbqh-l;uTm~04n_B)q6}^d#zYmexVZF#9B1Jx{IYnKWfhq@Hf%v%tf?@rV2X24jo37DdNCw8@ zRR~^Hh4S8xWbY>z#bU+rG!c8;(S)L(tU{=#Els!ZX|`Ka4wgjSQmHYeCx%T9WKq%| zUfyf_dpcJv3zE24YS4VcsfNwrhVb@MSkQWrh@#1nPf#CDd{blnbe5hfGXfPSllRPS zb$JsNDAl%27Aa*uuIFu|A^m(F+URz}E?Q2{y}^Q@#Kr8ulK1w8-(rCg!`!BefYXAm zYyt~H!=^m-_|K_j@OX>1?k-QR?;G8+X^Chwn|w4HNztU7d~on;GG-O7^YW2U@a9%6 zO%p*MdARkp5w20u(VoAtjAeqF`H;|g9so`#w`!2O$fw+d}FN&rld=x4YwOA;xRT6WXR zfBu`QGp`ma##iuegcL*CrkDLM7HHt#JyUX=A$^ghv#g`?0}y@G0w0@!pJw2hwlEonDbkF}n}>l({n2<3Xd@pzs-KXx z!9Moplr4JOM^e-DiM7?!(ImeL?42J8O2O%_?-U1n(_PGfR4iIOzXSAOH?gZg_ zk{ASb#F7XCND1u@b~YRTGiPEAbBPspiVd`oMk%vTE4!duNh)ri2R~^&KZVmPpnLM$ zs2t-w^83IRszpr(5^u=hx5T1qSoX4uiHY63=^;eC4T8O852idAfoEBYmIn_Ee$RHyZmIKUEQ)%zYidI* zAW2n4HWzrgsh+6HRV0%bjzAZ@Z9DuEA?^yMnRQE;486ocmk|BoL94zcNKCN!(`*9E zV#rqSdvkL@WvsAXk2+L4pgzd!Rc;s7EZ-02>dMZ`)rb4v)yVhIlSlZ*J(OSRn131q z8%GqU((c&2MRjC+%PLUO?9`9=VmDPtZ>}3feMX~P83Q4WhfrTqUqVbt;Nl3v0$fZ?Ow@YtT|yky zbVB;C*K@Ox2}rVhdIrJmpm9Qoke-d1&!grAyc?-pwaYM5c_tmQNs4nZu@KU)u0<&+ z@$2Y$B|IDTGtJo@;AP)_ciEj=|F)l&{FHxYex(m^iy`2X5hQLJ(fNK1o_9X^!5xS< zZxlZg%ek;SB`HYpa9m9M10noF0ur-iqZ8-Y_uYOknfvnHdazev_~MRc z6LhJAkrRE$7+ID*FM71c&QSk<=DT@T;E}`=ki)KmI;j2Wd+d5B> z#n2z-U$h~|=HU$Wyui@P$u(EByBRS2OJIJ)HHDbBPvfrOUYST}0YJleX>yTuk4>l4KkzUV;9a$B`1VELajQ z_;MI!+rBI+Sy6K6sUt>KZY%yuIm8%TsS-p4SC6Cfa>Ks{Rp*II2E1!$a06gtL^nU! zP_#~Oj)OX1P9KBpQ_VKF@wAqk?rtPGblJVQjD40nLs#Zw+6!On(Fwe5vfd*P49bf4 zMPa?^$5wpGpd)^WIO`nmfxe`V^)>t9$xeTZGDBoeA2V$)1wU=1F4Fu>j8(^8&&Zui zr!v>v&-}H!xHpPvq%SxXIufovaE$6Q8)!V{PA=W)A<|8WyYyTUW_wm${)dXt3L6We?l3i09r=2FZ3*woNdy6E?+Buo>~x=_P9OkPzUI zH8@VJ`oK z07XaX8XB^M=h|XR=<23z`wF)ec>1|6dFyhUw4^TR=)hJ&YjD3S$%dJURQDj`c)fWu zn1MQk#gLqwx~^v3JC!JFKJxQMmTCSf)ivhq!SheipJlr7tm;tc^ki5;m2JmOXJ5#Q zUuB5}>Wc<1#X=g~t%CwKBC%$0WKCz`PcLgwx2XHzQC~|W2lw%Qy=N0rXYuTLLY5G6 zD(;u=g4$A+ui~`xpKHLaVY zSR^1OV|z5y^Pu?IOu06vJ`akV``F3hJ)84xgJ-e$laH7!GI|Vq!QMA+F^q=O;HL|m zpvaJu!C&PLvz5YQ^T@GG6n~NeWSm>!O>0vCWqTt*0>}6LnIiThN9Uisy&Ha@;tx`J zq(Z+^SE%>XL+5FKwD2%dK$q%@`OcV2v(~Z1(kxd-D~e+S3A>J$fq{ajAaBaUm7)x> z#nqF7EX;^pEF4%lM_xsOL5+!L^t}l**ea(xs8gsa%xp9HoY zULBhGhd{()hO)$*XPehjWFL7|znT&d2=qI+tE|a9JI4h`T1-=P`+Lzn?Ch}jR0dm| zMp-+%qSsf;&RR2mq^F;-2Z^M-Ox+3wAE!Qd-N_3>m80OO?RiDPt5&4n>4jYfH=8ze zoHxjJ3;xUsk-F`GS-ULRo9@piuM4W=G+^PiM>mV*B?(WJHqrAZ(Y9fySed*8RYv*7_`Ue9hY0V5}%+W@&tnY_Zv z=5%q%$B#vp+W~sH{QhnuMfK{y&u#RmP=yY)cTtyyhD|XzW~=r5ae^mW?$@InTEQ%G zw`T)d;I3yrC2}|8&=cuZQ4N=hMG!r%hR3+H7iIbu2&6?H^kVws3DKQ_K(D9(vn_g}VTR2CyF_aAU_yYNN(~ec`N9i>fs%f(cQo8`#uGKS%`gNV znLx(g6IqGi-$fn6GSnKf&>>QfUmc;fy|-)~fI55-2s!2UQ}qKN3zLg$}fBUQ~zLn z53WmQmuw${9DKqCW59i$ZR$>d?$&xncJY?aLN<9dUCc6lx~A`CGj zddgdob1_!Gu*3vurrv4Oq?x(^;mc`37bGWatnIin`s3TD%i{iym=KA2vy-gNx2NCbnx47nsT}0qnR7VFq=<0yK?PdnWPrL>ACDkDj05uGE zgWZUg1Qr5I}><5cyl)Rw2#e03$zQr^A>;Iyxjo3{{@&Btvm+=*Wtr6 z%k*+PAM!GmOVL?fS&2=WeOxuC2(TIqC%V-WRymARjm=p|qn)@G0XH^LW=F^7%cG{_ z<<^Nbusb3*LNQWQkm8afG}05j>IH+vP|H`cgi^yw%jo!dC9@0jiGGrhpNj7B>1HXz z=uht!LeG~=U+TH6-a4&+hYgj$;2ucH@-b#m$|5!izy(Y8$*FS~7=H}aGBu?7=($Z! zwlq_Nc()i45%KP_nT1zpcXxMyn?2~a;2BA>ij}-`f-)+x&A!^d`=OrXI%j14UP=rR z9`*AlAhXDAdP!Vc3Q)f$v7{bAXHJd@OU0x6qT!O9OOK<7E0N5Ij$1UE%6J;xFO3lr zMw;Y@jRunG5EsV4QG|d(Il429=+d-0pR%+|ce9WaMIOUsPe4!V*>KUY?IH4kp5f!* z&sjrz9jwRU-$9fXZ+qD{H#gT?(ZA+JK6@LVAAdg~c0Ew2kin#gjX6g9kbZC55@R6I z<~tqDcd!J{1)b*~!Mh`zekO7&KDFbQuN>xiBVHwM`)GuD9#DrO%B(kEJgYfBaF^kz#EQ2nZ zsAGT0qy4A^yV&VyV@Y8|6lW|;veUPRV$ox_^OEjimP>ZTR6LX{P-^cVb)dy|I(MRi z9M;7G4{k~|?(aW*c>y|?!Y#-FnGS;=kD}M`Bv*+h)d`7oykph&31Q+MD0^%mY#*@; zp3oh{w=6e%59(7TuGGD)g;_Jl^4nhO;e@@vw5&ap($eln+sx9GpB-4T^os)TU}8&j z1z%#=sD1zX87$4;7Lp#t)IwmutCJ5zx7Gxl>kP(qHC{~+c2qxFxLg?Bev_9I2gjmN zzQIT0TWb9_*NHjpUtM2ZRxsn1Kj2@;Pk7#c*dmW3Mp$7+<4Q(h%E&x3p#c%#DJ+)$ z3e6BIf!9edQt;aO-iB+cYg>#H`|EU$Y%F!6^ zQ*#z41WOwtYru3+%n)(ckr6hSC)`FJzOo!Rmc%*a9DB3G{40Tw;c5Vf_&E-(q(leS zQdFLmHl`3ME@}&)3bER8OM0hh#!7n^^U)|PH9>zPU5_ZtLcKh0~;7cKI zuM*dK4!V|>Q3c>2!YvLMh|Hwj3kx;-v_UjfmiSe{E%xhxnd+^}yTBIZOKrfR)7EZR zOJX*u1Dr~^b_+NZMfLb;V=JYOv}xsLd9`)8n7$P2)eO;Nz3bg?(>wp8fhAeSOpPXa zr*#KLo)eZH@}ias-ZH`PLogNg2o3kKpFo~p4>P15hMxqo(`TI0cl(4cFwG^5L10H&6r!M)Ef0or3lz-%SA_{|^Hi{fKX&)$G~sUdcO3ff`N?JPkz6G(0~ zF$zY=f_;I(Ez_P+S(w4kDv>o&+8c>Xky43;@+A=2BaTCt=farF2oVKwyqXyV_i?-= z<_j#K)7I|L2Utr!fheS)i3q^uRyBpDOr&|sVMJ)R_^7%|JA%mzpBfx@-y&BpXRMW= znLGQa;kLWVT-e!8T48v0vt1Q({up_)!9u5P+;;ZS^ssGl>iqPZMZ@-EA}L=HFNgV7 zS4Sq|_YV7C^eN&9@;~FUdKR|SD87+<@zm{aMoU6n(N^J64gcaue;k7q# zWn6OK>z)DZM##e>OW0$LPbQV}2_Lo61$-4Q2HjpAK1ejKlk{UYGv&2l?))R{6LyP{ z;Gn0wax3YVVAId3s{pU)FgvlYhO+dgV_7i5K$z|!Gs}P4f>Fo^SdY4IiSuYOxVBYg zV%F?>fPVes=u1xz5BeY3s^W5QUiPopoa&1`Z5&7sbdwxOX{!C?#3t;2mkAKNeLw09 z%{H>ZrBoUda36p~Iue{B*46u2mTCikk%)L@i6FHVHP;7U7MsrA?tQ>|q%}Fo;jUpLot+_upRctcbBzSprD+rsXRKSJgxmXXSU*4UMnl!jv{l2sxYj&wD7rfuA z43jM;7Z^x(l>*dO=;x@8HRyiAH?q#{Xy}F^V}pyD%uw0VH%5S_Qm6d z*+0fY>q(AQneX@6pMy0tG~71lC$L^3W>Q|qv)K;s-%5faOYa>V?(Xv7npt!s z+(uPSQ-I2)jJWRSWhQMhqQ{o`KW{+~8{WK`IFp)mD}E9}gd>dm*<4j!+cJiu7)DI0 z;tIS3CiQfCVn{`BR_7w83#W$$@I;c<)@&tBEF3wzu!No)_!rx)Bl@~lE7A_IAX1}% z-L8SUdJTMPX0SXB3Jz}F)p-_nhl7B8p$m_Kz*f3~aR`?ioML&wtYiHEN|_j)4%*n< zZ25_L8P$3}5{g7_B#v5#ox))3L6yfU{)$=WBn!r~lfr=`$prf|Q`#mt28_u&w)zwg zrl#f421I#&i{;4LwNs^9SQD{fOYUxmiVadUV|k>erI+G!zL0;6x+fGjsrMQCTq+U- zkYN2TH^!V>U(c049(~=Q^bU)VR%V1YWEE0CLWp=3cT;qo^I(6qjs9(15n*v1;LFkm z{}u1@!9w=beFiJCoNVD2u+8YpU*^>#PU77<;v)eiyDG>hyv@6XAAeXo^zI!*${u0o zU_HSynz?MrKfV<^)jA#WmFZ@W_lx%ni(F@FP3Bf|(Q8>cieIX4a~ZIh;V_`slc4^z zz}yjb7LU!Ik(0RL4WO^x_cw|FS z!MknzmqoFELE-|6H22PI$wNhq#S!;4MoEho_a^%LW&2*<-oeP(RUgrButDj)qHV2i z-6y$f$m*MqX}+XvtT{iu?i(Zc&%rla2IGXr#qXC|Y(}rL50&elo)wVE&U!ywOg_Hd zQC>@WqxfWm3+PJ(d!8avRK-6ft9g39?DT!(yTCTfwNEiO6fw~wNyBtYyi}wp2wk}J zaHccJ%+x6ZWGRucgv(-CT7qL~^)9QO{35lN7c6_3hBGB5H37&h__7%5n(skxGIF8k z7m!7#DRifC1y|R!C=XaCZ1l6IZwe-*NgqgtvDg`!;Wehl26RM}W)4gWbZBkQf0~sZwV+^J{7< z!E9w|dbmYC<4=2k~5tEQNkxp@PCz|;k(uDG;HspN>m&pvWH zv2nEA9o~$O4+CHU@$bXWThh?LxlB1WS=x)4x!4yI*^WEQz&(%U)g&(UIXHc;_&K0$ z2MOnY2bfZXp>xDDMGH(YJkCe2V8;=`e(Em?5hc+10jp%^O-*t_l3Cuivv{xg@JPx9 zGA5h*qovAgPgn&&L~uq36G*=t-Sp7+>FPl8f*Rvly5+Zx8aqhBkft3p1*!z**dZ65 zrJ$Xz`0n#&E5*<87s5s5rExut=O1y8_frFZ9(8s~{p4g!p(wt%xY!^k1Mg7%ot!~L zI&9qB(68(ga5ur-gnoHd0*Dg+JDremto;2eR?-tYIet!gau=CKQ)jgrvFk#}CkkOJ zlS(-O?`4)^OJqmPg_$wh56ImRu{}@f_QI0MY}!}3ZEZ~N{jk!IfAc>!gUM%&KU4qs zWzW0oB!PNy!nfsEUz(y*+|IHhj}Me|aiw0BfzA5c1ApJ!^7POz91cl3{=RrNXt^2; z3}i3G`O3xtAhQOiAf@|Bp?KAyE&>5`{I4R$-$?m}yZTlbC03d0LFA>JcE@?k`0gIp z-0T0l1zVwCe%n?P75*Yk+<1jj2$n!6{`Reh?#L=fqC|QP{s1$$@-_e~SjW~(f>0vj z=*t&yMLy3KC&3}rR|k32ze89juM9y?rJh|a(@?vjR&P$hIs;b>-flCVk&m5CWZwD@ zp6tf`9Y+AI8JJQR*+2@p^>yqrKaf$=IID`jNnWb`MPbL4ytE_o8|HA*8j;obT8hsl zXUIyHy1_>LmvC?b&wG~KmEmt+pAcoO7E&QnFRCEXXck{r^FS*LODQ^!fdw2G1SO#@ zH6Ne*j?<%1ghkH%>Qf2B*NOj5?E7^AvRf4~{`@NJ0~1v@EnP;*(?^YEj8vg?-xbRq z#iiPGp|TtqvXFIrk*N<`Se_k+j%3M3iY2%BT{Q^XV|R^|9X^cH250--(dH~3XZ(085X}F ziaJ(X#fFFXP?Z5ud&(g>`LHG~zOZY-}ytI2%Z|lIY)F_r}@N z;mJezPlI%b>>9t0C0x8+^gJ)yy8G|{Si3znN9W;UlMvSOyrz~0Qe@h6dJMw7Fo)Rol8#j~j6lLD=HF0oxDSpJs((40bKWjU|6{YeExMn$`Sw;ai2#Rj1dk zF(&>wx<*FLsJNNjG24^=s65N(R|9by+jVs>k(b=T*8ma0Z*Q{5u4?OU3mV+!+GU`l znmSJNQ9ep;RpdnPQxwJlM=0C%3;V&)zCln}_)4e@ws-@&*EcS$sEG9FN=1TrKP8wF zx{fv^pYDS!5#3WGu`1S8-pB1yM9C|zShTa^82{!Ae?tSsa&6nr@nJB$Jyg(S8P$uN zSqj>g5Tz~r3L_jlQKHwmT>DvofCaHDP1j|&7+ju{Uy0V)c-vb@E6c_#xXgZz%T$~u zf1-M9>nI$wk+RpgmA@6{(@{x=Iz(O5-BfLPdwJ!Ecx?=R+J{8#H==+nqNsAOH>q5O zP3oqn3V0vvzP9dNZrt?QtrO5+4|;bVnZX}(D^{M@a@ajKTD(sc$n~e*?tFxk%6gbp zB_Gm9s5>KQTkoQ1r@!0l$&ZIOH}6a%cBbuUjK``~YcU^5uy3QxVF|t*9^)R#y47Lb zFf8W&?{f8BgM+~#491N?GBTvciqt|b654>h2RML+K0A&i{FnjXRSS)^|NLfS;GEvL z#`6c!|6D|0d1JLq&!kw4}f8#g3lG+P9IwM;lUJ$Zk(4K{5-@M>Fy ztUSJ3xuexvK{q?AOLGc3nq3t9RgSs@G|e|Y=1tg)=7$@3;*Vl^J`xVnzkXaKK7}PJDPtV<$XF2uv^8sHYR&GEPV!nRsVBlF= zxe?6S;pSk+)$RN0+Nq zUMedmln08E;zp;bO_>$CZV@x>k&=^>Z#P`N6a-->B_ygcL>L;0w(#t|+7RVBc-Tk2 z)x<4+JLx_uT8xcjlt#qK%E$=m+kIs9f*$cRCjRMv(?xuNE-0pnJ{s89nl2Nr96mVi zUB>~*q?V5J^E{wjsL;g`U&!deoH)aCt~P!ag;+|;gHGw{bPK~ac<%H>8J_BB4`%S! z%Y+bNy5lo@@d~X+%@V|56SPwE^M^%_h{=J>Pag#bRaZ97%3_1s2U>lSJD+ynPKaT> zxN`n?f0>(Hp&&q@_5Kzv?Yhz)>hS#cm@Sc?F1m!;%qN<8MVxxy!?)giU*wp8MA36~ zB!j6Z(2XLQk=#j$6$8U9mFeJQJOZ!LsqQ=RJ$g4n;|;$x$>R?16%o0|&3Z4MsO~L9 zD3LVbI@CMF-2tSMkWl1ugr&~faFB>1>B901SuCXG&wY%6vO@EX-@ z^70gTHMvcuI_j=b#_YE)w!BobTz{F7BVJuy{lW;Tr>-aU^mu>2!KKRUep*cIesW=< z*|rC^78NNlwmk2B^m~Wj> zWG?0aeC|v(e=K_x2O;;q-v-~zlh!vBoi2JWmll<2#l#WU|2(A`45QgsM@*7z@`ZuLpIlfN*j^J*bZ;C+ia;BuAwC#Ruscz9<_@D{%mOSJk!<@L;>NcQ|bu<#)^~b00GimHq3$XlqOO_zJmzu zc7wg+F`AmFeeZG5{jYLypYqDe%2BqHjH(0xKWck@)^R&U_DP+UCggnRVrhiN{A7N2eIlG1-?YAuX5M5E88mx0EmZr9SwCBQO{11SP$dNE~oF+&z= zAILDl`V#2CL3dPGY_fqm>JU{1I*+^S)AhcDrZ4ZVOLEPiSst;$F8gben&kf-la`p| z6&gj<{VhEaLH-4(pHkK4_5nV?%IEPhXqO(z5EkFg3ab_10ez#_?zgFECyakA(U11k zOy}`C6=w$eZ)zt(APjqQ;q9Rz^Q><(G!t^=Rszzd>nGm-ZPU_kFDDjS*k{eFA8~Ea z#Uej}l!nS||F-~4K1c`QN8c0P9{iT&%Mfhfz4ni*x5;^VmLdBxUTTg=~GAQ{(y4G2{5W~bQKPI$6A>bsHSMfxYEBmKz626OoslJNIHLrel}K*?qxpK_Hg zEGRbf=QmWkgJ#X&uY%q)`=GZ!SoNjhM~V%#OpQ_+V6s;Ad7I4~+de*BNQI7-CzI^b zc|<0iU&^-FSjBz{?2+c;HAsC84;_oAtZqa%B%t{88~fPgNSDX>&7R~kfO-RGB(z-5 zV6b$1fqoZdN?u!jM&VHY;s2fwa6hVTMbUe_5-NsK3lQ|*x$AM1g$jwO|85e5%Lv!p zWyo}+p(S4UQdrc$pQck0VPud+F08vhJB4Q&OKtpkzs3HboKeW|Y6!j%=x`kr>7g3t zkYJf||J1d>hT6?0E& z!{5Ne|6I8p!^{>}p|;<;_V$=UnoE0l7B5p=m6;_of@04Ga=IasooHue7N>xv~|w?uu9A`Kg3*{CD-2O6{~E zNgNy41cPe2YBA@gK2D=4nL4je#5VO_=;*qiPZnv?(6r!UOS-Fkr*}O!@!8#+&fI>T z`JQ-4f#Er8C__F7)6+OTO6#5>^lq%TLq>p=4ky*^Ea+;%UF4PcQ)ckUa5~H!HJ(u*VVmg+_nFb+$ z$|kK*R@HrSME04M-j%oA)f5-ctDEAqHe%<_?IsDPK(zJ5MmFlOg+xePR-!8t@zCvF z??y=Gi-W!;5&uh3D-S5)Bx(&z?N;Yy|wH46X%2L^jU z<3ruC{-Do>*?`Nf7SD!mCX2fo#&>@2teyi2hd48V@?nG6TOj&(HG=;#Zq11YnV{G| zj%j2;!Ooh2%HT~Fs=58S#^C#XQ*K|6%Z1$=`_G(rU34h^&u#%inZFbUYmX6KnOM|9 z^H86&B4b0cky~isg9v0v1`G%3+;jN-J4FIi;kiwSX>SAo6pS zuDyw$6TE8lz{%DNOL;6)uIB~Ah~u`~;b_&RmY{pT!D!J|y9ts^#eZp0bNp2; ziR|Ch$VcV+G%hpe#bWyT_4d?d95MTXkxuR9X`juLH}iAOfj65oH7|#yzB!82I{v8$ zy1KACNj9asUt4H!r*-xE4$vqt4r2eLtQN{j0Fj>V-AQUx2+(i!oe*Yt)%@?8 zlTK$@Spu^B0ZT?VvtnvVg}qk!QCr*U&}G4IJrq^b`^jev;(K^lmwXxo49V$Qgr;fO z!j%bO6nee;_QrWNL^n85c;k}5wOG#jiuk!r0m1eSZyG4hZoLyWGt$InPKc$W)x#~j;6?Jdj_z5jH zsA}Bm_O_a@bzZJU?3c$^*LWYYas+xVe+<5!F!jGB16BDS4Gml!rEutqBi250GixHJ z_Br$M&m_coN!4)CZ(L*mnrpyW(IUADw4}Urm?P)3(|uG4D!y1$Jg;MLwlSekwzi4&89#U}FE*NW{OVOU{g6bp{f#($xu%W6ToZT1V;TNXXBqU&3;8dI~& z(m|S+GpB^9W}vZ!roA`c%itRYx+FE$M%$%>f;m{KkJR`w$-ZsdqDNB|OOb*SBTcEY zw%{N~u~T5shLEh3`x8;(k%Z5wsR8iAjPt(MqVgit#m90O)x6?hEx)^5;MKYqjicw( zOF{PcU{OB<#-8)-VFJ744~Hi7j2f<5@3M^n%zaJ2>J0azf`YbA zJ;!_Y1z)qWDX*rswzjgi^pQKdA<=7S$B8My>kYUwQzxK>Nx4 z!`{xg&Dv3j`m+|Fzm!H4dpmAjy^o)3j-jb>{MURoYN!#Aw;CB`ZNHjaVq*Lznee~F z70*8hK5!Z>9exL4WMq{8c@4uAYIVGz(ez@I!P4;$`O2!cm1Kt5K8^M-p1q8{#W1TL zznt}ZDd80zf>!-fkosmL&m`L8irLTK0whre;aV*5wL0$Aud@DIW@#&Is%oJTweRLH z+&^R`ik+93)j;%Hc49<=bulC}P^$5~jeI`|98lsH)HU0dSF7?)O`L8rB~FEq7S zyT2Uls>W;)yymp3Qk0Ry96VsuOw+_kK72rADPi>?hm<-G+eyggPCj>|@KLa5eI;y4 zRt~#F?&8Xy8i)M_=4}F}&Ak%_(Q`4MllzMYu-EmJF28-BzC!#342Duq+I2v0GstcE zXq}xLcJibWq$CfYqNmmuw0`G!VhX>uz_WRxrK`(ABEH#i-mE5I8jV9+Oo#6kjV;V} zBr`crvwy7k!lX@{Io>@`Vosr_4ba7pO{~h#8kuCdj$c1sP=7d$rIisW4N6Jwv1m{Wg~z+GZcolq;k zK6&?3Pc*{!<7eY?hv%#v-jf!}R~H_An(0k0yO)DH(--UEma3}W!JU~vjxF2giRP*{ zt0_Gc{|VkKTe4ezS-gAS3Tmex2=IS6!TxW_E!sdU0AYqrdf=!qU^v>lGnVh}yzEe) z($Xk1YB*aM@nn#v7Co&|HD8{&s*U&Lm=V{}7WZr`~EPhC!@@$ zvDtrqIzJ6o)Ux?nE1kU2qqI|kmn1L{NZ)P$s3EjUbPWv*o&dEfAt?ssiynWjr62yy2fI1qmb5d8 z%z(Jzh)L;O<9QJ>-@mwkaYK#Cz+U!UPu9l8F z3s(lxOaq5iFX%)9%W}VGX$81pK4NVVlvsTYsVY=Wo~di8c9=QJ1qcZMKA@vL6@r@z zF|H_4z~B+pKogcKBTmBYQtVylr}Mv{L+yIAeH%IYQ;PV~1r3~>9xvMvVh7+n#Y?~j zMMI-8FpY^sTd>@?#iVELjL{!vp+_^1(I( z2xCLR+oJBcv%TZ889zozehj@6M2;27IHYb!zA%o!7>D?AWe?fkDJ8mLdj#p(pDxo~ z!!jgz8$~AbJCPZUOi0*spN8;+HJQA#?QZ?!<_tIVwf~f+u26U=Llb*{ArG6>7ny?1 zCk#04g*=dv2OY_fhlU>*T*5wgWH@$9A9t{jcf`%1U$@9X$nc>H2N~L*Br~2E zbAe=A4)BJIco$nZd~{U0xUqk#mS9{3GP38qT!**aOr>EhmzR_rxaW#jJwyL$dlkEk zx`M{HqEtNJc+!1{knRvkR6)ai`b9Do<8`!`kE*rB!O{1Nkz+}JZqZT0TFYIz4~qY< zyF>8*+uiYR{}QeZV48Fbv>_dNWe%o1!6&)lgwJ~pgsWx=K8dz+QR5mn1Sofy4T?Uu zb$t$kFt}~uKcI2Ze=Zg~@wJEB&3+w&CIV|wnuk==nhd`Y_bZV$qj$ffbcWl=0mkKe z*nKywzk!iM>wcw%g8~;b;2opE&v(8|#Jh(_xUvhw-@gx7R(@;fVe#&XAzQT#a`x6` zC7o+_RD^-YelKjLD3UhgyioX(39Pu4d_}NQ^o1#kn!w8WzEb4BjJF^bnCTQRw_r*# z=yVy0HxblDh|op~hIV82NNQ+ZhStUH75$ z13_OEx?cCQr#xzEB-#?=YqOl_wZruNqA)o#6Lc~%zj%-1`4`&2r%kgAmX=<;#X>HT zsA7xY($*?^uDneCL0}Nf%7{OT!S^>;^G#R&O!_sHO5-8DTwi~LUOxkwE(t!JWF2$8 zbmS30u8=OMwz6I;9A>YO*^OYBbV_%tARtJGNW&nFNC`41(j7B{bSMqdUD6>U-ICJX zjdbS_-x_?L_uczAzP*pbKXJ{9d)@1b^ZZ>=mDv%VsY5vNiS&Uvpc@u_9H5~V|9uH1 z5A;=TBo@wBjKU`@sPs6oe~ePXhW@0hYzc9&vQLa?N~?)LHTyQ#M|*{N{+l`3SOJSh zAl)IhUB1j=sVs!>rEhTqD7nv*IpG*r<U|93Tl( z45cf#OjA&!MBrWc{0=|pLfwLmvN+*U_~9(#x!7m+iGz$g!(8;9mtmAA@5UfZe-g;D z>H9y)ure{C5OMp|s1N>d`wPPfy}8fs<6o=fMrK;b|H$%^;Af-blSt*OfaV6&i*QT- z`$gDe5>b@&lLxaquuhe1vEzlImlNoA7Bqc?bj-=%g(kG8c-)tnEK>)PUv&uCHbRo0 zZl&QLhjYAOyBTF%6Ma@D%X{q(s(d$^L%^T<`!?~u_$~IBZ2+fvZr_~YX-RaB#Snqe zi%A~ui(kDL^W^CF7ut6&xaSFaK=yYueSth6XJtX$gnlQwAy0X|A8#)w9a1ZPAO+2s zt;py}w9d=PN;$?HkaJjI#q?7Nxm$F#7?sJKO3p{{*oxE$YJ=;SFw86Wmm*a7`)}5H z@7I*2YTS#f2H7e{IkV)3V|r8m=J>Yrr48K-M)2j3AdA<+YR>E&(Mt*5g_)RH$UEdY z@$Yeivc6_Gn2q^4HT0T3SRSc~wbR0s>wYO$`!e%qjr|SLjBSqZg2>;czRoCdiOC;? zF)Mwv=G;CWH!t69wNm-b>`OFJIqCYYlbQWNmkT|_oP!63zvb7Z%(!cAcI>?NP}TPJ z5F{}z$}AZ-|AWl4HCud*QGRuX2u*bc(xlzw0b*aUG$AJ4+_v7XXFXa1%)CII?S-!- zVzD!b{PvDc&U54w?R_g(v(HZ#rAKEpl3}Y~HVifKdpWJCUbhqDRpvV`KB49K_ zFi`9MZa!BK!Sdo}Q7Xta`$~-O$PKE;gB&Z?# zEGN2d!$(0S4z>JIxH_C>bV1*ylNLa5?l>MpcRkvVJnQv9PZMrX8*0f;+hH&^swxrs z7Ks=eygjmvko%bVA$fli|H$h!1`uKg81iVk*(y(y{kIcq7N;B>Mp&8qR&@;@h2K|( z+@}K3b&V0;m$Js-JitvLN&X;c%8(b{UoTu)jCpW_lPlihq#P9MyE&VV9bcS}{ACNS zgh{pLtc`Spti(pc!8Jbs_;FGc`%hhEwL98%@8kpT@_XyJUgUVVB0?(`dM)m>nfHK_G)uAaik zqo(-!TIf#|wgp^40XA9Kzr)@9=-h6?8`?iti@(Xub5oRg-^{ zDg<6~R+7gWC5v3l*VPyYeaaI=zG?A+6YXjsxx99X8uy{rnPOFA=&gMea|geLE%(fI zFZ|8rdfHuKo#=W#0h_U~XfxilMP%#eid?-mQ2JwOABoS5__Cl^dRRrM#2iw~mJtlA zIQA};Jn#8@UcBH3l{J$&)11<}DXsF0ZbiyqXPX3>?-^@jw{BqzOVogd=Xw-S z!?kGqi4i-fj~vuMsJbni!AiV2*gC=-mY8hnX$mOc$k00jOH~+H|MY)<3o4R9OHeDZ zV5fWZs}PZ3qw<_;^sP4fl$6Y>w-*Puxf=?HHaFVG`)ivsPXL))7BuCRH?s9#b_bl* z{@gRZyP8neElCg+ho6RA-dzStc8E=A=Kc7jIdx_TzwWKWPu>NKU5=I%<49awP8b`P z1gwbFddy9ag?})z_w+=RGt=vi3$Sox?tGfBH}%}i&HOMT`X&hsbzEKl6GB?Y09*c^ zH(q4;yt0f}c6w;u?&4r6F){8W@g*CXEVhJw$gzt=1LU}Xl|SU6*9$-<)EVaW*2u9} zzC5b#CO9u`x=MS|>tZu9n!=`xnro)wy>j=;s}6pVH%Z~Nw?AqdhgPmDc4vCVOXJ1G zaTXBEg(RI%wRXJTO{_mxhLBU>B**3c=I_MU)g8<2I*qU1n`|^)xe^`;S!8XJ@L1is zwC#BE_{kIQxr2+oOiKgLk)bT(vGE@yt*uMeb*a-eX4Pm{Cxh}EO${6w^~1x%6?oeg zq+~y;7eqW(3#V(zGt{TTtf>Wy(!?8r>Q{y2!;>XEjW~Crm9r)P;FY4&h+V+05CjZT zS_zg5jk*t@Ru(#s ztKy{F*B%29Pj5vWRa$l%=psrBT=!;$U6%`uWPvHH=Zz=B^taa=xkmG~)}5)l)C0+4 zWPEfHFkyt~07e2C7&#^d#g*z! zPt2T-v(wk4@AW_RueA1@iu;4Csx*#_S2)kCH+Y`kUP|7L&rE=skcb_B{)Gw+uxmB3 zH827Kn0dn<2Ag-UbGF)yL;mGjIDB_6v{disuj3HIt?rfZTPJ;#tHiBss-!n{#Pa}^ zPLmX;Ceq&#A~>NNbkUReceiG4eBP<9kR6+qEi!C8x92FNr@$bMUbsJUaP~6X?>Gyf z5H3JH*RRWMx2(1#Z;L~T7iQ3-2Wi0!cS&eWp@KpAMJodMPnRfsF15r!<<+1s0 zrFfkujOKk@PH!AXaDmI5O>yz}{Gyt%=_&Sh7OHK_!qTDwu2QdkU+~tnh+y2VxeKlK z1o*d2_4wGR_P9WmQ*QEgLXuF4q~}rBST6-tVv^+L5%t>OQOCnpDuUNcQxsOVwzePk z!&qOc-S(+cPb7=GuO7uJpMHIk26LxTGrwH+H!TJDoO8QRs>Z5fpI1+dc#7e0>Rsm1 z&Rb|`zs8Cr#KyRk6!sKW2>Q!@LylbS76tLL+#qRiS&nkt-AvPsK~yH)AT;kXi-6## zfDn#o>}hc*y}?FuX@aoX8`Dwi{9@Ga1yNVt9B&zY_!NdIkGIZF*oXm`tC2%ns{5;~Du-vp|64z4?djbsi*TBb6J5XzE@fsI(Hg#Th`1m;U z_1GgH!%nWn2##E!(>yQ$Uz(cZk*MUB#sD4tVvjP;cA^HgW09pwqCiw;xU|Kh9%|y2u%zwwO+fTRPv^iix?;df@%p_~>2|@-o z$7M-_*v*SZm1u@A{!?;FDFes0NU+3KtOCR~vs&l@X!WY+o6UHA#rar5WiF2qZ(bAo z_|&|bz(ZzMi)?zLF%#dXcm`|gJ(u2&X&k8DQ!POcJV4xfM9Gr0kNQU!w6t9w5t8k9 z4GnA+aw9mmNGjdR-DCtjCR`O|Bl-LKibdPhqgE->=u3(|o)ZzHMvl;LQ-pa3#<4R= zx?lVq>e;fO&sd!5PHiOhhpvuRDC)jUbb2UFvcK5k+vVl>z!ktPaM%(tQid z8x)Y#pgBJ~$NuEnwIXIXdWW(i3lEfC zy|lDWG*Uq&>Dl&m4L{wHpP`22tEaO*$?BA7aHi^OLf2t-#C(XkT(!NXTO&im9>S2`}Nlj zF}Q^13XN1C0PSFF9x9EUyJ*WBymg@0q>X}1k@`q^Gt8thAgwo1qA-gfT$`RkzhsXy zRKNOc_~p^shKm^8@I?!WF@p3AYHhWzSM9KC<~)|VamsQjr(A0B+Zt`igJFKc5rFeT zTy6F@!RSfez*xoOc~8t&&fJ7T@C*d2UDnuS&d&Zc(}jmn zf8>4}_#Y~UZ~OPpRuJP&Fn>#mHbA>*Fkx%Pq2KcM8@BC=ILZfb7s%Ly)1(637rW=+ zQ7Sl0ekrn6|M2U6iMU~HQFV1;M$Icirb95zorTL%G*3gVF)_4L4Q?>y<(mO0-Y{7T z8e6r0g)Q!EMz5Y)^E{cUs>{!+It6z0{DJ}^Qo;`+pi)^-nF6q(r1WQU~&n4x~H7()RZE7mz58@hATQ zV_1S>GqDJTkXDdSxV)I8w9oh(9M|;IN4$erxQv6%Rx0)!Yos?L)J;a%IQ_>an&*u3?zVsPH+K{imr-os~hWJE&*b`qBr0u1lOoj{!|G8oO{}zP|u9HVo z{$CZ(kF%t7KoWdt#^zBPexMv-M@ZyKoLD2=t&@ZXaN)oru7|PPN&v=YQV4$|7sCPSt%vFA?eCg&x>3aTx)}NDf`gn(*n|#35 zCg1aJF=HOa@l_)ivXfF59hD{Vl6ci)2ej%fU`Y5}yup1?$P&40o8`tR3ZY=h-PMF& zG~QJ=HLiX1Swh2K^jdHPi=eYAHl9x0;VAoXJT~F}{ZH=odn`K%giD1Iy&7X`icPG` zkw3`{Fs(2}a8-_8c(5tUqvJdL6jxdI6hx-T&j_l3<39@Vd!Ap@U4WDN5#`7PKRqty zKhm*wZ*AXl6L4JfO3@h$VWAb-&tciR! zkN7FY1xUR24~P9gryqv3?r_PO*t$O=){m5T>iha_c<)U~7%x=Xa34vR6cc6vEdJx;4QXmYVchAGXAE)Q z-UFu*{~ZFj?_PCa)Vc28E*~>Pq0r4y3Z$&4OEFYey{FrA31BR7=x_dy#Q1dg5-Bmp zIm5>3r;Ye3_@b9>k`M#Mh4Fm^iPi|acaZt0c*|yLOB;!EP!}enffdIwSB2;oU;Av| z&v>1<^4ia7e?nO0XX`iMw;J1j96J-ONMcHt!_OMO9&8n1H@_fUZU6>QU0aZ(gNROi z&lk-630RquaFKV-;^U+9XE_`0Be|+GcfYLwLctWKW9Q!%JN;lX8zX6HPlFZXyQ9FS zo88hpwUk5&$9~(p%X3!habhv=5p4b#I=JWo7E`+fzXBF2asEJ}!zdbvR=|hiMGK`C z)JOgxK88WMLN&sE1qIbX>+MnGSG;I%*`-PxE4yFvT)0P(sbt=MWVoiP7_O{#q!L}s zSkjN^zUP_vE92-eDiYUsSH48{t-*Obg6oyfm}ZcsK22Y4D;V! z*ey0wm+r&O?yPK^9Meln!$&E1O>|T{JQKI9t~i@28h3Zb3IK4*&1|J-ViMJsh?qm$ zdjmn6r~1Z#K(sjLay1gEIgVDl?Pqtf)^BiH{~<%K1{n7e%mNDwwh>F~^&&(*0~t>Y z1b4aScq=)k$HyIZCd)&TVrBYA2E^+C1sX6h=T3>G!}Unen$x|}=_c*=g1)fVrO2oW zCI=&mh8LDE0i$89CW?~IyWUDB{Bm2m25gY-IO=|H^PG;afT<2@NrEdb$|L1%X*v@^ z9?4DHo9sfxMtlt%#*;N8^JjmSz&%zr?u1P9x9uZ9kEOs);^HLKk)q@x8p{5A^t`3T zw*@ot>m3?khkQtf`|d7Vh2^8L0rEzu{;{VIzp!dUB>a;Vj20tx%P*YO`c-3=Zicl$T5Nmn1gt^ZMt=&ysrk8fNTHn+Du zcTXq2Ju%=bgFY&~L>k0Z@*fdmu=F+}8f;2Oxe&=Em2E zrn~!p#&>&lm^XMPK4dgIy_0pe3yn~Ok8LmG2T7gAZsh9MY)0=$MJxx0OGpS<{q7U@ zJTuWdO`83}U@m8f>mXZOx7WUW2sFT%F#K7Qgi=K9ueWk+lWd;T2|L8ReQa1u$igx` zGGt_|ul@S<T+~|BklOO~yAq1(OxGdOM z+1AWWC6SgN++}AlWb~yiO-ZoCp_(H%D^3%i0?@0hp>M{pntv+nHpxONEf7_VnX0Th zTLccaz5U?{;IeG31ULPHKzslpb~xe~Cp5;!*Uz^ip-0j9Bb7}tF#3@OAMu@qA5HyN z$E%Tt$8*20xOB276|p7+e@IkWsEy)vToF4vxE0Eo#BqvR_G{bMTfm4O>JQYt<4vIqG0dju6=>1JTLbazKAxZ!0Y_Xo*T z*`E>pyl7~T-!m#b1E(1Y6f@lYK6TM}ZH!NUSyWQM)N)BznFZ4oTVfVF3CT?O5%!t# zm3oIi=@c06t=7qURg!_xlrR86ntBN7^b|4JbGoF7`Yhg_e8kGp+J>V|MsIT^mbpwT ze5DV4|H|hW`cD34@79lr=R8acU1dtnWYg0(CmPHII5686OS=Y3@gt!P_g^;*pl8oO z^|i4aa4aBw$t%In!nCAfVuZUw7-J*2MXPNCK3<+4?a96X(wj8l?HNZ%6zO@lsY1wG zzayaFn8h1KIPx#GsW+Di_HrOoiQ3u(riknh7*{NNXg@5Zw>RI+tf4hb3D)ZQEI2wg zzsG4<1#1If(o_XD02`~Xgm^>F%Rk^%C}Tr;i7eAI>)%BWUHe%$OY8txh3?$-r}Ec5 zeJ}l*`%*XjGcT1?4rtWvHV9U&Z1pl!KxYjV6e`*3G6zCICfJhq_df)<|Aa1cdwY8W zNUIHd0L$xqE`EakUkpNTewj?+q#cD?^yxLhZm!)c3E&?2ISqAb4}rnM*C*o@uz}L= z#jaDFUqrr@XtQX?xDQpx3&q)d*xoX(aail=IF=B*$WRzwv3r~Wl&lN|He?J85Z{$2 zTy{Mw*q{FSTCG!TSozORnMh2r>Mj8^m_0sDl%O^tTDzlhwcml5T)z)ZEl?2baS5-M zJa8rM+OsAcu(I7oCSyI*Tnpz25+moAh1w_{-U`?0lkkY$=o?X!syn2;jQ$#prlm2 z8)IHkaJ>~vJvLPt8y833pS%FwtWa2H z(WmJV@Q>;$=tT=+P20IgZSZWn=QL@i+D6UURZ!?T$RG4Fp$q#=Q4~k#LRQ9ZkwgqB zrOZzjvF{6YwVtbgwoy_*bDA*fyOKv85!_1li}0@f&~}|v{IsXgR)Gt)yt=s+Fpha6 zexUhn-brG|78xGJa|W+J)vLGJ^B znRH%X$jagb3IwY&XWR~F#C(GHdcUmfP)QS`P^M=czMk5Xz!ppk>?L^i3uz za9Z-yeB$cb+0riKBI2+_7xg^*Jt6j_ zCf=W(mN|DN+0}dk^Kc>%4n;`cFUiRt$SfGB5y(>CVK*O4e(YzS;tNG*NMUH-ltxIg z{qBk!N_Wc#vJ8!OBaa~|Ii3lQDR$zV(#;qB`~wnoPTiSt0g2#w&uX5f&oYg}QdDkg z>rwb(P`~5o2dwf?CXI`l(ilJybr6Yn!yak>=Hk9-n^45oi_<(QN8?CBB&`@_dvnTT zsu)3{^o^j_p!}>41+{D+VcT#@UPv)c3X)c6dCBFEUvrhe!Hy!8{>i|1mS5~|^&s-Y z`TNo0XB80N>GAXYKlELc47~>{!3w?w(y}0d=<1YBb_NOIY_*6jewVh~*?|hB2+MZw za6-OrnU}UE?Ui`FwDbbMn*x9Fl@}DeO24;i_Q_J64{aO_ITGU2Kg*Q-U0Ff!06g$} zRNtQr^^BADmSDE`hEOU&mYKGm($x~Y{49rnO>qEc&ttVL{GZn|;Ss>S;cex0b(7a7 z1{E-eHw6kpSbgwmqadyi%`Gs`1%8hA;jb*2UNGL^$ax$!YX*`1%G0||ukq`+rRPeZ zT}AF!&jf%7qnw?O{)(nkVUWYjhPyhX^P=zxBQecM=YG$E9{qv8Y9v+s*Qm*It16Dh_ zlazl4Gh^#Vvq2YhL5GeuUzI^o%}UA4>#i*7@U5;68LoOCncX!5`_+|~D?AQ)R)7^Y zEC5bZga!iXWk^<$fhgaK3v6^$0=+$w_RsQhV(s@`vdMqpe-QUaizyICqn03EiEy^1l*M}(wyuS&J)oy)t{VhU;$r-uR|Rz z@ZN2%Xnt8={k_&Bb~KXs1liPHH+^PaB)ZHjHTWeI&`#%IQ;*<&iPgrWa>m;$?*JkC zWIz~bux`KAhS`aT8Dh5|VTLV#;a|c`VjSsMIqG`p4TT|c<;rwlJpwIodb|qDE126e zSqfxmR9@_gwji%i(~nDpN}7{4hnV*StAMQ4LY};`44o)3mtge13P9*E6k?Mgsz- z9GeG$hhj+PE8h2#yr<41lbyD$rn_6$>-eK2-R{T)mbhG9A5+eySi`RQPy0@PgfOrE z`L=TET+RFblWTMMfDk)i-uIkh*6o_psND=6{yKt_+2Y>Q%dI`7DXAAj_%6J4HK5vh zUa#T<6bL}i#^YAxCB7GaKTQ^+7qdJgu15zYB+7g;{7lc)qI0&*Da zsLC(o=clP3=W7&q2YEtm?v-gbQxAm(;K$unp%{@0V7SJjObTGzbKQB2I9m;l`<=Te z=C+ZNG2XDxyt8o@yPhkNqpQu#JTqp|rf+ zC1V^P9UaBjtmzmkJ8C*xzcf?M3$zp{HJJ&}7HSlSEw=|BP3MN=r*)?WR_^07@-(Gq zXRr1Jo9XgEXPgQTyCSo8OuC0L+Kj)g`+e1)YP};!IYHIRRVg9Fj(oRxN3MhWNugT* zAFuh1G~OVE5$wGW)n{ZT0=rd`m+u_PlK&h@!o@L{YhtHO?-PNP#L_q?Z54os9EckY zW(KcD6NMuQUt+M}nxb^=ZmP0q7zt2W7k(&u+ElYYcC#i#Cko$_dAVZkoeR8i>vdn9 zwTl$WqY3f*qc;+Rce3#A(9F0_=9Jx~Ns1>a7Cl=5r!iC&9=pc?n=7^54AtsI~*d z)fIaGuO`EFX@(9!A^yyqTB^U%&Ky!Kh>di}^t}w#)_V`Uu|t+lcegj$K;=WKme^9x z1^{@weK&vA@rQl9G@;H;(b)_$8nwqnm!uqkt=x|Kba0nyS5;iOmpnBTs0Mf{3nVX? zE#`PRN#9J^6*gvA4D4Y@H)EA48m|i%j78v$<>f@_Q;Q{ry3ytSjb{S2dNhT0d2(+5O`Bw$Ts1rEyte$ebcCK(ufC z!?LBQrt+hx2arGj!qv0vjfqjgUwyCK`^T&nSf8)X(3pHQjU`xs6C`Z@jd%?Islos} z=HDv9YSKeLfY-+dYbAO%LWRv{KL0?BAiaf>TuMzVR?3@Av)f9jL^Pudpfw5Fa_KjC zXak^OPb)J`O=W9M1Ew4UWuaJWPHbUdBG&*ncOew#VIZAx_3!XbF75(NQU@8EsrAln zDk=|>tCKxQ>ZFd&(S^u@GM$9ikA|DD2+f(%D3$KT`h=5inlYC`B%&kvvuU!^d-m zhTY~&q}U4+9h%*;FKBcOOyk?J5>?n8D#1t$Dn zn09Q;l5<435$Vi4gkHF;oxWM<^5`ttn-7d|&bk>bkx}@+HiCn8H>=c*N*j+Z3O3c} z@8;AY7#l<{7~U%iXG^ron2^2GX3VGmBp2W1+N~l}pj7l<0QdhxqM5fF>P$gS{$L@& z!r@(PYHI3r<}xOf8M(pIvvT`y_4&Q=vSWJQpW7c{{|3*>Ro}LsQGhBbH9ic%saBs;oIzI zm96kWS5sv0dhDz6cF^99Hj{B1l*u(E-qMnHLdUe{!P6mJ*#h%s`FoKhao3E};!K9c z)u`N079*=~P5oCZ4sk=NBKx5axn=-@lLiS;iB~wMcYGxN8{$hP-jXN3!@_&An4A+Z z^z_-PSOW9=ADRi@xz6j}U%0+H03!sEmGM5(1CWNdoF0zCT_CSTgtNkz@j#ykV;>X; z@DxD#d=Ve!l(MHnp7s=McKW8@9&pbqzcg0Lqt>54YkP7d9*s~qfWEkhbZRrv-bTA` zr3KCbwoDZ~5wF#M9f-bcAx<{Xf-PxE@D1iOH_8_SWfwOGANShmYxWQ8W6=_cKRvjQ z0MZf_pP^ist_G$pBi=rbmPHnXkb8`|hAPOxZ0nr2h(05pfY;43l`b7qJ`AvtMb_WP zl9!zE#O{wzOvvw@20(rB?-vA)6k9dyLWLPJni@8UdADdf&{4+hzPN81khg~Q2ow!s zc>jfNzv;Z4PWkr!AkPT0ILq&wfk5n5BLoER5!f-|i8(K2ML6PkWLYw0v)dfc9VN)$ zfr_NZ?mqXcwq;!;q=0J9`74%U{fEEN=IqB09&98A>Z$K9P{HnH1vs7u+@L9mw`k06 zkqFw)Hyq4A%aaHf5D%Xb@N8cpZI1Drw#3>f8e22A*xJj4DfN!HBauLMSTN0~+ zKsxvjW?KmrJTCgVh_8hlXT_fqnTKDNzpEu*N%JrttPbEm!1yUa0NmU!37Kq8sJy?J z1|u%SA1a`P`Sl3e&ez+F?IetN8PB{)8c35koI6eGF{QU*OM?IPhJD!*pL*rJlyK7} zd!ii+Sb0SV(c&2B#N6h*gS0*@ij5Oon6ja>&I5$)MQ1v-eROZw!L8?%ZNY2`3-Sk_ z5tL&7MCD~kk;6h2;-P}=Qtkgt zWEeoP0_ZR@DYc@+Yo|*v_@!U5KeR7vmBPs<)LY{6CH4Nbibh!G6DyP1h(;)jRIYJ6 ztO4moHjpr3nv|b@4va1wO+7rj#Cwxb*1(n~QA>`Q!|tl$m=u7C!qq{KEt z-P}=sY8+5`k(2+A%1h4wr8zb;0V^s#otNZ=9!k%315jLElvV>1Jz_ayMMnR9ZF?yD zC|&C_0M3@D)?#k@Dr7&gptr?8L4{F`2-~vrXGKcg&lJ%!RmYX$&hZj=_x-}adH1~f z+T4O%o0Z?Hz)3%^`ti_oXgSB)x*Fl>fi%l~|FP3P)RwYa{N6xlZ>D-548JJVvt2mK z3H3nKhQVCAUT^Nzzm%;`q#}U1aZv8VoR(%cT&?%&p@`8A99}@Qcygi%HWG7O4oys~ zU#8>{JMTN$97zF2r_b~#DfOz+Ue+JGSm{s2K}#EtHUJn7#5E1{Q~o65@4RDK<{Di&x)nPiKyV@e%-XG~nVR_A=ZS*NbUD-t zzyj-Fdz0Oo?11%3Lc}XK_fSp$gJOJr?)nVW9zHefk54>A_(Z=VyP~$kKD0-$MD(8= zX)7sF;_~R1o#kbezp0|%^P#cvx0;3q`i|Q!kZ}2ou>O+7f{?h}^<&7u^VyGy)h5y* zGVbn|qDndozwCCldBSx@pT|x-KfPq6kF@+@vK{bz158unbomZy)(epj_eF9_2!~-B z#-Cy?>@-k`H~|WKwlDupFd56of21oFYyubtLmi2?FX(ZA?SXS>Xy|=;8;6*z^MA_Q z?ap;Tc^j`(mi-M>KCj_pPf{j@jkqWuN@xDKr3R;8;iqVj7nFY=xprVv6BFyYGiATri;0Z`F z5z&DE(!OF%)ECX6u*}g00^$woXtLk6=pJxYOA=lwCkqvsv6`h?C4 z=}*3$Z?xqx6med<>Ss^g304LgiUrTD=nZ*~eO>YM+MOrCEGrbY;}SK;oK6b`+)Y#( zdtX;}#(HRH(ssbLcvWO2FWrYS3v~+vXTLm@BoAdJ!bLCsNgd4x* znJ?m=x^e=)1?Iq#7wc(1daX+Wj5L9iXvOVLj-@Rx_6^nha8LATHWE*j9WJ$T(bwP2 zFUk*BqG~$_e0qQtHTo}@v53wj;Uy~P8r|OAydFt{&y_cPx|wzc2;RD5GuFGhPg9x4 z?u*d#+HXV}4uWTPi|J{+@-hqLx>5YJfStP>NF|}}&fp?~WP+}85(of25mgr4nbq0K zSr+OoEHx}Vl}&$a;!Rl)Q#&_)a=0O%I$2}A67c6Bs{0}Z&?W|Y8Q<`Pr<}z8w3#!~ zBDA)&cGP;b>>ZZ!VkG*Y`ag=9*kA5h%fQqbmdv(9>iEFcgA{6fpVF1GT@EI5=TzjS z97kl;JR~nQs51&?FZynsOMhUO{5_L7S)W^>dI9y`cK*j2o&37kBn^v;QI%MF@ggR5 za#aA|jH#2tR0Gh(fcUb66>U%aUSv0NMXp2_GDL_lwZ9QCTi=Ht7c$e=8ymi+bY+sL z$>t33AytybiNL#|*D%}PqrK869ykU@Rf-IkYy2-d9Pgx~_Tj^a);7-)%9X%iFzof= zf8%h|SRl(*EJuxCoT}Jv#m%lOzrvCr*9PWYI2L{*mLjCcR?kr9eQQR`!CSDXvi%HC zN!%h&L^u#5NrIMMiY*!f=*WiZ6#ygAi)EXqPgGf>zN$Wvs3}s%TI2v>VW|gvy`_Hx z%snxM-b)Jkt5g8OgWg(No%m-t#VR^`!Z_hUf(OfOVU*<wr z0hIpkFs6AurX7L9@Mt*O;1g zIdhXxhRhv58SQ(p$%Rl!F3qWL6>6Hy#|GHHN^jdI84d4>m&f+O-}3mq0+Pi@GB6*U zIpBCl${qM*5J4Gvf&N|(Do0&G;w~faNzRnWeVO1(g6xB@YJp!&I^n1XzG%niXE^ZH z!ZQ3(AKfyGH>3n7Pw;Q8D4FhKp7kXF#oUeViuxcTl~}>U%C?_&t$inOpXSjLS#~)6 zW&A8ZM1qD}ZNPOZu^ANPWci)`zs{R(Rau{uuQKzMJUcLxm5DmMU+i!8x{UZy;8Zt{ zD*oPI_80d@tE10N#Ug-K-tZ+$a!s()W4OBWUAftv87+yIAb-oRiP7{~51yW|d=cL* zGQ5UrqII61^!K^(ZxuG*$~*lIPOjuj6S+cx?$*1uXW<9^dao7k^NM3a%BV&)*>b&o zzcZ9aS|8XqEFNs!5Y(;^bM8lg)5}>z^KW$mdaoUK$ipS zUxlhIpt1GfQFkzPI4xu z0Z>Bb``OcA?`Go4PM%$3Z^~QTpIIMTj-QH?IEe~{HhqD5Kmb+JG`l5+iQjIeucL!c zytp@^n6`0n0IiC~Wr)$WAB+(O11Td1=o6yQ$-Wo#~7GHdw1XxaeHR@bt100*uKbD*>w? z7zW%<(PE%u#Q5OCKJzta?Q#E;HCcS@;;g1OA0^3ISwsglqQPf1y~PH&pFW)jMAD>c?32|%EC`NLtnFA59VNN)pY5wggLI6D+>!`(l5Tl4~V{{8tOv9Cq zQe5Y^m5AyxDxOdaeJkR5nmRu}?Qq@2K2}v7m2q7>Q*CWn{qF39!)w1w*EmfAAm07N z|7U39e9!dYR|^d&6LH=|{#CB?MHEBPRz8UE{lE(FRUtVpcwXqA|7j*n9K3hEw_FW0 zp>)j5R^}T#ze(20G^p>;Sz35 zl9319Pv#oU5aYxA>}vIw{{!zE2|bqXuOQrm6%CEV73_17BbfG*R}V z-Zv+oY4-hs)F6@Aw^!R;Uv4qqc;m%A)x;ssWH=94tSi}>G{PB6ny>-5ZEj+XCfyG7 zAi{TVYVBY}$1Yq`6rmj+2qZWfg!SKMhtqRj0$qk z;(FCyH+mk{9r<01Kk56o#k;ShZqkKz;M0j=(UKYugxzY#0#<8=J{TWe+RbDL{9;Tp z&#$)5?9_QIP!f?}oMYI{*fll6-n~6z^ya6P>uY(d?Lc}jbwm9QQdJ^)Nq50W1;EU6 z+r699eVdxqsWvfNVn2Eu40T9)GXsdECmZJsxC#nO##~b+V14@laL0XR+#2lGZLWPe z);qy%NG#vME`*``g>vY+R6oQ86e7g>9Q&G+-H(SEBliIqD2E=);c9T_jaSrjbsKP z3=LIF^Px1@IbenJEprH7pU%n5t%IPX{B zOGQC#T*O;i@0&W$ER-C`o;rDvEHrhRTp2~d?>~uOo}KqP%Uq&jW(ym9fCErji;?Yp zHpl=dNBPPmn^W4tnh~-fqxKly?)hkqy!a~QzP5JxebJ7kV8s}&zQ^Ai#0x9#{&%p; z%|t(K9>+6Wboa}waUv=~TS~fKdQ;U!qXOcb@EXroqshk$o@<$&d_5n3L+iuO?IaV- z_HI*g0g+P!Ufn=?>2dhU(|@}rJtaGd{jmN1*=e)G62l#xiJnb#m1^)WQ<;4Ytet{cK+w%bzjMe#P^-#OPgP%5oV;h3E8FBrd-6v|Dt z9uZa7w^V2I*h}c_u^ITP2=N)u^a$7AF-vi;$6WWF|7z;G%}w<6S0zHCj*kHL3XOuw z(V8)y9T)yzUMD|gw!8zuQw2%=Y*D5{)B|Z51}tc4`O!IZyM(X|ppSApci0@-b4gx4 zsTujQZ(n`CBrNzAXFZZ|B-tr~nv8M&@E$eD^uH`E5jH*T!xW2HFA-DH^N1;o_V#nO z)JqKV)9ZTuE?}SGRaOaI_J9}M%sPs3N=IhX9}DA>W~y zGM1bF{ypC=yX@FDvl9Tn0XMm?t)=nD?7Muxcu2$eKv9qew!d$(S68fNITaG?XhANz zH8RJwYn~SGSzS|^(&8?STc&=%eo@_dz;UyIsBhcyi<78zI6HYKulaIs90%pz>8SFi zMBN6P>Opz}I)Z);_5UT6o_c2GaxDTd(biTQ4t6TRF-1<@;H%NOkr8zRpt5pYT#VF7 zGe8cy9Mp?bKLuPZCSSa+m=AnHVZI>=TI@{`b&WE7rj6>>uI}!#DJ8|}7y2;s?y62V zx*-J!#%vV_D4N;nocMjW*D%9Lh-5Qxm{`%_deU4iuY6lXx(BO6^~hjXV{7wI>|Kn} zBT<3TXlVYR(z`b5@Dh){*UqB=qMA{HG4|;07=BY;d=my1rWTBOX7PlXb$s@^yc{Ct zDw=fv4kqxf%wzQ(*MAtv-ThCy9{^2bI&v>y`o8&xXsb4$4u>6<54ew5^y^o{c5UJ3 z2mrlO>vdU>2a0C^d9Fq z%POLhw2%{8kFx_A>7;}7F-ZGj{ZLw(4|LN_AZv+L?vdAo#+Jp+U z7ulI68i0Nvq>O-9Yl%dPrRcG+BeAx&4}r5b*-qs{4PCLo_Pm`k1CXh6G*_U+rNGuX zh)Jmjmkb!T04m5%kS)!U4^I_?#&u2oxGg_b28KQ!%8RG&ht5omWufusoIsh5`7rPm z^hbL6Kt5)Cw$ITEWLxTD8cMve)b?C`W~*|yM&IELA-5)e;TrER+vJNtCmZO>+txS2 z@2E>@U)ZJRl@3sfJ?-l#fQbQf<;e0{&7m_>h1xt4O^6!Exq#5~qWOBW&7)ZOP59kd z8gV%D`}YQ^GZpJ#`1M`!e687OEgnA(&RYd}&lNx^`|)EUbh#?rcD51%eho{Vm~cMs znc``>u2^st&ty-XZ*ZC0qQtwM~7TN=!zGP7sHj@OcnNoZNjsN!&SGc?o!JX8PS zrK%D6_68rHj& zr>V?;%->Xc!5M-Rxa6l)U9_O{v*wrE8+ohaP1~@Tg(4MumRBVMkw^vnY=sTrYV4g&qF^*&v0x*!nj`|KqIvZ~-hK_PXf9~&C+B6!yekg^F%czZE%{gtw0v8=z$tGnJutkL1}&CC}co*)js22zw!!K{<332 ztILW$_0E4Yx@t?i9i0sg$o(BIam6vTP-!>h0|CZk%#3@SEuZdCF9BmMsK%|iQp6kb z%TIsu0reI@8E+s!47>G*Jbhb#eRx0hC$m}}AV1NtnO=-o^m*<1Z}bk(BTlY5>gyGZ z#S+{D{jrJ)>|);fqj{4RWbfoI+AO^O37Ppl!X7^N@uOj##~8Qgc_U&Eu{QPxv^)9p z{YJm_Mi_i-*3D{c`Z-dfpXEB=s1M~%7?JXr?we{W4bAZPJ$Na~50;Sp{v7ZAso_0s zR=<)aGyN@xn$DB@!s+iI05zWhbn-M;f1@)3S1r(qe7wjWNN3k=t394K7PC@oTYK1W zb6b(~(7?cE&7aqHe)h^XypZ^^J9SS_Rl5poVJ%K{=MCPrF-Aw9yIW}cf;Pe$C zf^1f%FQU^FThD=;61a7>anfP&M^8h|mMKmNj`~Oa<~I`-&q2TJFR2-P0{=G?em8HlOvnFW`Ioj(w)1+8 z4xc#=p(G7rQo!}`VJ1&8O-4DVslG_Rr|DNZ&HWtE1NLzi_-;3b zk_F8BeO->5?|ItahS;q(eiy8XdqyU>IFUd@6IG|A0$D_iVAE{AIir=+K-4-D^Mj_t zX{l6v9pmE@14&qJQ2JbFy~387)v>t2(KJCGHz@p zzP@96{j;u&e;LtF0AyGr_(X0+o4e5SEb2T`WUU77nMJ+&y>e~@CuM4uO!6V`D$%P- zx#f?w8sex*Qs@FvadIAieJmBbK#=7wt{HpzUpj5jQKkPWgLiKO2oGnX&a8ABbpz}&@#GkgQJC*!71YD8Ku~~-M9fEG<*k%bemlLR%jG)u0z_rYH}a5_YDceLlhWL%bg*fx6t<S#~m%EE3gI)ndf5%)}9L=M-dot3N z=&PlwnnN=tevIn#xu0#EMnpu2IDN3PViUbSlSlAdcqXTok7R%7SzTVj)W*%!#T+i5 z@>~up9JlQ@sB1grftjSY6Um-qPYYIubLkKyRj)zQ21&DTAM7aQ^%a!rtot z&#@P_AnPK~3q^Hlmbsb5!`{0`lrVMe(vOChod=wMXVLjf9!?oF+Y6tM`~F%$|UA&@A22p+UnFcHr}6)V)w42L!lJG z=tV4*Mai;;%Fk%{KJa_(+S+9OpH@FMV(4PiF;7C7GmhV_dappI5#!;=N-Vx zQVlJ$r-hCY{|235dCwLD4%SvTrs*)-(EZ(6TunT7WGY>duw+Z4a{ckH!i3qo;~nv` z1gh%?&{6nh4uO{AL7nZbq4q+S&`cyI09Td{@9i^=L~b*wCfewWn+2z-_I+WeGk--( zgVSO{)@RkVHQ+JYN;A4|vU*P%BA9K&PKS6ROBxn`@5Jy~UCDDMojm%)(lZeDMO}>o zVmv$$+9}Say)|ht$$##9@&88ZW!HpHC;wP+&0b0tkK+85N?h-BioCjy;G(~~PhivF zcROkl)P*HWt1ntwI5G=ZU14!XUkE8i-v?|l;}NMlrk(_+_se5jVE?7%$g)==h#s#8 zBkKZQ{2LPCZ$0}3`hJiRr54;|EF^MYPS|=k+)C$R@8rZcsZI$)cqW*;HE1($?0r2u z)Z)u)XiO$0^?9)On_qzjzuv=lhy4C6i0$F&+*`uT-+mO{u1^}6d?J1jritMPtfY0EH<>Y2uP-dpo_Kp%W%h&7bnS8Var0~9 z&W@8IEH+{%?Z~CEsPS2C(Lv!EcHL0K3*yRBaimO?L%B`cV}WCwYf~2PI6OQw@Qu4~ zlqz5P{g-RsQGroS2nLY(o2to2h%SVpAWBXxNn>*-(@(6S_S1RbVWVE((>!bs*A+)C zK{PB^&*8$Qw2_gK5{PD&_2dDC+}E;H(fjcEqre3n_|C%QI+R{FtAm1k zma}EltTK6l9NNb?eMac-$OM`$1O@r5=55aX6oI#U9l6%chqYG7)l*Yb^EF@6CT0OE zmTvQIl8i$(n#SyPr{fm4$332HJdyR2vGsb$^_RKdC*w;I5rVC5*R=_yxTxeak#Ui}}=Yof3@uaCETH~s63 zFH^9XhPY_~63yOdEV@hQx^2bLI`P3s4j4?nTusD~HmU$sAiuY&44D8xAQUsF>ua2Y2wYg=%;3Et407>-fE@OaPv&UJcToYy?kz`yXb&ijBfYF=E%QKXEsZd8kD@TLC7sE&_)&}%EfOm2;XlVQyTwFkd z^g&ZI=8M;hG&4xw?t{^NKIT-;#ahY&C;fSTqj26p%kUXF-Q#}uxQ8U$p> zF063ZAOlBS5#ptYwv;_;#Q$D5@x>Dprt)oF&&{~pQMRKm^`}A0O~SGL=mQ;Iy`;Hx z`zFSnX|D+5veqWTV{o(m(`d8RNnVX>4RB!emAh@uFMnt-wvpcdy4?DBERmF=Qmju+ zv79SHP!}S@^pM+8Ya*zccq;=-&w7r5UW6o%k$Yw=M(?l?`Z_X#S^kdi&;8b!Xe#u2 zG#yp!VI`sU2Bm;90RVx#JD$-qq#5^mquILl)gb=yh2Uy5o=(6!1{yG`*hyH$<+IyU z^#lK4$(*&SFjCQA{t`46sk&!@Oh-a$%K!h1ErnESji@8KOkyyysAX+dXFC+IOh7qn z!S2>fPcf+hgszzTzn^_i$Gnd0dagKoqGvi)vcS;lQ!LbAlW0+;fy9wyP^I$R1jbpc zE*lWn2{A@VP@pYb*OCXr;w7w?Vu(TahTfJC-Vi zNLG($_xqUWfN9%DX#v}0PWxB!?(Br1>j8rMIrCY)TnWFidFYbjnEOD}6>CkV`=ssM zrHc5Wc<5D7_E-8b<$L6x#V?*+O=JCck4%s)A>fCbPilZ2i49;cm3)!mTb%mXG~}E-wneR{>himGk7odMa@r-6xQn*yFUNdr0ZlEc$?2G$rnd&)TMAl z`Uw@1QIDxpMs6y<5-Xy-oNxWR@SY1Fc8jq8uEE{o1Wah_+P5CQNhvz;TfxQ1j#@lJ zDe~EM1%2cdPkPj)Yiw@QxQRJx0mg<~N_dwB7hu2X!te z{M+=N2{Dmgp?-XKf($YaIyR3>DlH%$$&?=$^ez~^MZ!^>d!Pv8BHs?QImL2Dd06+~ zr@jIGSky*6>X;-1NZvgkP)l|aeJJdUq9p8L;XRBTdjDJ>h+DrsBV22FxF;UHP=SGAvMOtqm&NYO~$|WfdCcdRTki88^ug_&RF7%3v{hidCFEqYlry zyo7MLOq_5d=5>33R;W23s5wAA7){@Y*0#~fV!6ga)zeN%B>s$l3%&-^G|Ae64?t4} zT}?AjG$UiliRVJmFZBO}jz9w;ej&tA@yP_OAW#VkW#{xp$P!moV5)+DkC40jihkf= zYY~R~gxuV2&v3Yz8i-l8Y=_D@P&kfQ{H`(>^A#|_p{->t*tVFhGC}->U2m9bSUk`o zevnf8X(A8>bDN<5le#I>q{sCbeAzn_Mg2Hr0WjIAe;+xSWhxJsZlPTa~Cwq@E zJaI2uxkHoL4=F3DfV3Dd1iqs5_Pm<&UE0o<2+OTRO+$ z+!2-<%02@u$OMXUl2G)L6AS0?wk#7`DJ>mssJEljt^ zbF1^(O6nFCJ+wInEvO*W{l5pOr~Ce(a+Jhun4|G^fTQN}_5S|;^387K^4;ZpCbp6( zY!q|upfg0JH+5Q3)Rof@1jzGTZc~C_Mc~@zM0Y-AZu7>INqGVqRc9P~fy7E}_3$jO z*dIU!AoQ6W%cL(KzGD?*!PaPd^QB-?;4N`@dRV5YTB$QyLLvM0qpQC8^UPyZZ0 zTol4fJ8!ce?I`>&l`7`#%9PQ6Z2MARwOE``6oqpoqn~+r6zR@^W&at2i5AW=fmo{ zo&|@FZ{I3HblvZ_o_^;oDN3LFHRri~!g$PAyRdMdI)l(ZydCtq>9uF*t1GPDe-Gc- zWQX5G!hgMx*Q&8xZh+ltI`of})I8pcTC|M4(I2NGdkX>f{+T#4n-6@>1J!Xtlmw0^ z1Gz;aP?VI}YP0S5skdgs`A=7V&s$Oo-t=GYvumC~nzjRo5EC_sF3Q`e8ZEw2y@oo+ zOS|*!(QCUT&X${p+4gKru|V?Pg=zLttGOF@9S;d7>HAC%O6^7$R_T6W&SoWN-9?>L z)az!a&7M?%2=4L1{dkyLjbD?~5`5=eRS)HsuPh^9-?;REtnKfEVzgHP#*&j%MX)@^ ztXa;dwX6#ENUfTp) zJ>C~^zB`^BtFqlZ^WDehP8T}8<@9>oH|Nu8y>FhGF1dSRp0UC{?{x6-x9h>LOt1Uf zQ3Tr+zJXdgoL~CBV??;|dyU0whqc*Kugfr2^%DMV5Suc^kva@@7$;rHdWfO#PS8rAKJ`C zMl^zhkig?4&}iNgp#ELsVK#9!+#5mmkl=Nj09#5Ey=^z=-ZL-ZSuQRpvP#k1USTyj zjmb?-b(-*#5Ul0n;js{O-B9BHI->`Ng*W);i7@+XtTLZok!$I?8}ugRwcJdVE`3J0 zn)}UoT^yX5yz?;7tup@fIpdnI6*jZn@_1aJw%02PJFPGoE%Ox%BVym#Z@drFdssun zUj?YbCBpFB`Ejqh*7I_IR55j~#b|WEij5wG5plJ8l9>*F*!J*f7JXZ8~vj zek}5GyFXmUewv3^(Y5K2Oj%)J+G~fsW7yHZ7s4qfK;wjwANgPI)bmZs1Vw*9f?KsS zNwf*BYt&hc@Os`#{2B}VQ1NAR`JolQZw0@yg3m0&h7r9oxE^ac3jxYAMzi7^*(=Wuff$X0>yB#C^_je*_^6lVs9A{suY)-Jkv` z;IY;hB9YFs{87O6A#9|@GhM(NBQqz|+S5&T74+Ebf!w)!*VE(dFdV2rsz@f{d^JR5 zQEPcTUz;w(rg=AXDCnfxi4ZFcLrW^;N08b#_AVW_r!3}g7D8w#(}nE{bAKLoKBnWU z|8zGs(0=I;7yuT=gd`vCr#GK;XLd?s>sVUeKQi`~R$qQut|xnYvsR%+#Dy}Z9~d!?vgr^=abjKAS8KOA?w)_ z2UUO#5uTDVn(>T(qdC?(welRw#{oqC{H?V4&JUsJfpDla(&o%YaDYT~EASN_)%VC5_R*{CLh-~G*_jOu zck`u94~q%cTS~NavT}^(d#VK`-}$uE5v2Xwek087;o^5a9J&@IX=57Sf<2Doh!8Q{ z3Y7rgIz^}A2X;y%A5}?dX{n;C1z%|yO)m%S!oqKaN042iAr@hjb>fw(0BPT4{ItX9 z&ye1APu}4eqZ`RW|CUU(VwUJ|^{3Q_MKkFGYEY;^%-)#?Bt|79*>%$rDdh>6nuwHa zz}U84(gA4)M;avA{$Y2Kwd29-ZxzQyfwAQ0WP>AvIG#vFU69*KmtEbQ`H{VYJ92KY z)fQqhl4Tv1>UoBy46q;(YjX6x1TE>9qWMp9*k~jpLbZeWv8JFV6!prISBLjyn|pCs z>Zo1e`uhC%RvKuVhLPDyubc(|__(2Grg%XkEC5O0g%)JWr6x!EPFecMSAtwAA$)8c zDqol`KGV>saA6eOMW;6*DXo#67C(2moaztLagu$5?6nNT(JImHcjocID-eB^65yP` zsl9abyCg6$aEkTWwd2y>!EP#Fo{!|U*Trh?);nCqV9n^~WLMEqO`6SLbZ#qy$6lxT z*K3CDz1YC<4Yu>~XLjvCGh(hgl%VT7gkNND6(vl>s@X>5h{iorkVL#i7AhI?aZ(dM z`q_h*)M(y`oP3U9`M(0>g-uiFyIWOa)D(G!W#{#$OiQl&KgAh-foVwc!y~2MZSff;4YrsCCpunf>WPNm_(GM*=x5)#ELit86-QJ9@!6D3~!-*n2p z%;grT`%j9=b{BwsVX&q4*jDTQ0Xy}1bRaG2Z~#l&uQmwp_tddpvr~3L(w#_#AH6l= z?}LK4!{Rv_Z8tG8!o8`$D4$MW60%8lU=OTN^|X(We@CyghZ%bDCiQO&Y%%NqFgD*G zUdpr>{khy^+dYn-o{r(GZ3SP^m)bKMZgD$9YFCANXTG@7_U3OF02TP8W!hV=a z8R6X7bNP0Ay7heeUNa;wXk5?3Wh?hkK~uDDY+%7>C{H>z&e zE%{fY`+Sx?DeB+t^->!%29k zOXcgS`l6N-eW`6uPt&o3WCX5hl*f7W`+Q9w!^v{_L-kfYAtuSv5^I@lK-ojv8nrOQTWq#oIYm5@V0^*eQKN&Ac{smb4X}a)sp7* zL$c`OglNzmd99Y`IU@L4RFeNTso%@8|l6`JKi45A^H?6>HLmrf%i_Y zES7#(eV?SPb8D=u45OB}y|^JtlDC|xd8#m9iz4nJyy(=>fb5)eyHsPjP~)K-b5s*CVkt>q4p0e9C#Ohjw(U-bb$=IFg~4F)-)BT`Pp-+y z$$^Gg;c46Tm&VXKHmu zZSA$+wMd=wX;wxA8pgD{e~D@t4HcfR1Xv`@n@p^8&T_e;{L?*!=%1hBVGdKB9O-?3 zG;(j$^$n4sjP=bgGrK+AI~a}^P5mluP^6B9MNlsM#otiwT=fg#y^Ki1^SZiU-c80M zG0|MGqf0q)fC6aqgCIOi#=h!?KB!a-BHm=(XC`H8U{$~dvbQS$^UY>aN0vIllX72V z44qke7CkBNiUx&Tg}ojyiY2y$bSU zb#=A4;#OfRMcKg0ioS!+<9b@~Uo-?OCg84zp3;o)@4JYOR_DF$o;f?rC7O1!`= zM?>gu1Hu6qdOBlRNx^DBnMCOr01e|nKxeqSOMy&<8Wk2B1Nxb&0S!|h=W94qEa*uh zkSPep(j|TZ4BXm3@P&%eVHbufxnX?Wf!OznL6~ACGWDm{3^9ufK`d(A-nJU0d`y#k zO6b)U?r0o>fY&N>e697X?pZI}NIY;KKcrm75rC9K?eS4i=}~@X^%T?A78M#PPl*#- zSCtQBMC&|FSDs~1>nETfAS*Cda_$yU{tuVR>5@;llHN_C$p~oH)~GjIcL$P`v>UtpaYTJhiv4y0^5Z(l(t+88MHu$Q7GFnLa5e%<5J zo)w?H1pJrO8GU9J0P1fA2Y|A0Yg5Fp333tbKTU%8Xs2hKwIkBJ zo@xh8T=-LEAKLOIztM(l3xw0uwJs8$nkNsz)QCAkKj#aD?m3sd0um6=A4mSC9cb)| z9-;EpAkNWh@cDKQWKD4kpKfL;6lVBejC|R#r{N+SF3fpEkB3fW@0)%{0KvhpF)YUs z&-ea>GDG`clDkVyaE6)XNfUj7F}});r>M_h2m2qiR~=sGoNs7P?%gZg zeBb4-^Y6cP=J&}|F#_5qlvnfCwSG`O=`Jv02|nxQJvsNHo?PP^lI=(p;R8qqqizGsa?k(*1|Qr0Gg_ zg`K_G;uPNCUvb|!lBpnHNxPbP;lBJ9Z-^*f5~cMP_wAuZ4LpRa{5)!`W>wmfk>hu+ zTS`809JsJdy;~73wnX!Hwc($TaBUKCW705spOpInjACY4jsD<(Ju^>D3Hfd(kFVeA z4inF(Han9jZ~m5>GGHCyJszPpd2*w|7kP%!2vi$n(FouP>`aPbLXVRR%@+DWm(COX z=}VIB!v}q)B5Q5i#jkYxWS15OZ=pOq9p7V`uCsy{zp|n zsk13=d#{8OtN_woQ9<+aePsy2MDunqP$n2%JW468NCRMy5>pE(Vvz~<3yB*xq(KUy zX`79!QFFNXEm5z>!&Gt$t)VvRwi5ajK>h=zdo%05A54xP*C`_QT*wL?`RTREtb!(rVpujL{;we@gC#FjW2xd+I$S$% z8Q82PE!P8iMt|xme+kEl3(emQPEA9>URi9cv0AL4cxCEfU>fn6IxE(-gKO@wQNN3Q~W}}Xz6`d`O?qdFCw0fv@EJvAr4cew^o*+pTP8$dZl{DB_F*@oP z-~9)Z$?i=hiP=`SFS*-k`s0DwvezZ_$BUx0#X-lBkv|kE4Sp|s;2y$7p+xuZ$5VM|c3>E14VJk)FXH_tBJS-V=JG-5psi_e;mkdXoIPiqqH?m(#>|$ZzxT zSeqmj6x7Y;eIl363kp5oua_JuyiL=*Dutg}`AHyX&%v3$0B9yQ*o26~6CDkYqpJXb z#)M7vLlXd81E6U>G3$PWHH)eZa!C-7aS2|^1^|Jm8$f-+HxO$OI?6R{ z>VG-=OCjeMzei3e|Jd5jANSqp$kBUm12 z{IG`~!)`UjR$uZ*Hy(~kN;>bctCkvWzlp;87hBq|m+sXdZo2NDhJQ~zopPW+LGVPc zj_)tz>;!{nid`&6m&hwkd5r^ArEwllyF&0q3^9&vTC^3PAjLR&-3!L1rGrm8DPdfn z!G<+ne_Axx?*m8T?g4FAz za|V73e=xU#pN#xaaMN>MJ?J;Ozt)@4m~ztfv}s@HvDQ#&R_{v$)Mf) zfqGz!SSj})2v#N)_pg~L_s8T&jb6xADeR1sXhe_e)vVU$r~5-va*J9Znv3`gdS1?W z^frJ5GaE>o*&D>QyZi@156?r3OifVaE$w*E%?#^ynR(v@gRDNYDF;FHKPi#9^)-osbRQX51i3DkxxeeA6C zqkJOZUFXDq!PzMBCMgdfk7AkiFryVm=~;|A@L#%!H@T{WH) zQIV7WDRXk501!Ea1j}?-hDk0qzAWk+m;g*)7jkD&6!=alYwdU*zB+8K{!*J=UyAE5 zwADo%A{aszTF7Kyv$%8(J4W?$K2~CgBLQhB^Vm{|8OW5s+}_9X-xbNEctOiX2gs1e zpHS|PCDz9UkgT_jK5acey0>d>YWn%sr9@Yk2|e+_d*djvRtBT!}TPSY+p7|ef3UIFsb&*0oNT3XvADse1-BEO~< ziuo0qVx6h&%~>}$?wY+h64qGLQfISBBpqoSco`k!C~8n9HF_ki+{6jsk_k;a@N;Yb zcQ(nh3YnnNE7e=Mida2%a#IHhW1iV2ZtX`$%VE{e*J@pp3s*YLloY9dKulwc!uJNQ zQr);UORH}~x0#}g1>CM&Uz4W@IT>k93?_v)BD774-wbXI<&%2@ z+1tWil7<}7&wA0~RVF(h6j&mVe?VsO&pKkS z($is|FZZBs*GI5TPWa)kRU?U{g_5uPjuy)Y+b`8Q^aPz?p%@~9_LD@vTP)lZDhhLp zIMI**Io2fZ4trrqQ?pI=pJtZs7OV@ii>hlyH`Qe5sW_=3&(b|Y3w$%)T_X5%s2JQ}iBfH=Hsyaw z0u+ZD^&)~%vz66i!!6FShxOh?LP+x6k6|r6SA%P%*Ao+SW6R6Oej<+t&e*3vevmux z^iM~ODAB4QtpKoJSz9Av3ujR6y(;!x=`Px@d1di^ieD@4sHV0VA;KuJ3z@U^ei!R2 z@!{jK)7U4C`uFPESR;SCjqB(E@uk|eh&bh2e`oXNHAcBF8lu0QfkdSAv6M&=6jfG6 z)R`xKs&5EnwMi^d4)${%{_b^4c~_bSvtMTHEa8>5!EXK-+Zm&a1R_wM;DZLBi2=G#^`ow*dey+=3@;K-_@0b}DKon0WEj zsz|6n+^Jzp=W|ZDJKW04Zhfzs%}u0HMwS)`+5?h-aVJ0RbW>Pj5N$E&7oqUt_PA^_ zV_+xEE~EV)HiDk>ShOdDQ&=V%Zx3+3yp*3cjHsjM;Xg6yN!j#dC@CQNQ8Gh}J^b7(<~` zN?v+G6w8xUbEC#u$5rVM39z4@cHC8u-H-%=@}*dz&^El8h3_^sOao)a zxBt2r&Y>s418-8KY4_!gW8KAj)`n8GuPg`{isU8V#fc$}<|TTIdmVAjn9Vh3hxGjF z@Oq2nK1d@*X&t8Vf`u#zF-QXQ{ra7EC|~v?$He&f!+fnLPwWT=aaSgDgFGJlTQ9ZK z&{K${K=MDBAq^V@{&0Hm+bqpRYIPO8+CByxje%YsMH^T5109d*890F%{kl?CRSkTo zTlU9`I8P*OG)D=$FNt&%->)7Z+?BC(pCK;dpxFi?=8R&}E{83YTgG(6M0+VLW5MzDiQV zmV2vj_4jiIvC;4Hnk8wJZyv|C`8f$b&_LdjIrNliu^NT{21ye97)lAIRqb|&Q1eKx z8vNyKw`HSF0Ewoo<+7tu0ZA(3DG^{ii7b8_HOl$QHJ*T7Ngnm@qtkhFm!;frw`N|i zZ7J#M{;&mSE_`F{g&B>i^GM+Qo4^#dl03Wa8UL>U?6J?(5MrRgwi~44w8z8kc&$2? zUO+BXp+&Ste$KG>;5F#r9mEy%Lf-?Np44=qcsrCkkRu;W#j&1`YE;NN@H#8jnwVbu zM{YQenMdC{1wF+cr_Gmtw=J=TfHVOb*ST>@*y>p&?~?&f#CehL8|yW?NY=a>z#0jc z@Az#U1pfVlqS79}SMuPF#&$-YA=_ z+1x*OTftkDf-)bQ5Tv0KggB|3#>-v7UVMS)aufLasZNc>-%CT^4<$0uR?|pf7>7?e zE!a@3Ej5p;Z3#caV;aS1k1-jTld-6(A1jBLg5dGp_<88p-_X?Q^UEq7; zg*KhYiu_fhu;A~&D~-0eyfN~ z>>cHDCF;pUa2keJ7T=7uiYXy!djIe0cM-NrfH&ds^~hkE)H|hBCdK?HB>e_OfuwWT zbq83*074K=8o|=1@Y6*XP0q4C>y{&7@1PEuAOAQ{z+hm~y_x4p$TQy#Opecvf&UO9 z)vjzPfTfE}I_5ouzII#{WFSFv#>i{h5DmtRDOuCCzIhjO8CYsfJky4^qee!S~5TegBRO*^Tm4Kx}3g z(B*XRcgaW%GG+5Nt){5?bP4%9+J8Gawj>0@oBC)&rHNKF;hQNUClh3dU3-HWo~-N7E? z-c(Wc1b|cqu9S-vitQ?p(fA&gD68oaLspI{cMk&dx-Do~0=xs4K7{^4RR|_y@~#j+ zB24{DmkvCod#OX<4SxT0+|Po2W?b!i^VHb1;6tV8BhFhQQM<>p?$#Sl(MI>fy%xnt z7MGCcU%ei;^vFGrQwNDS58O`!ruL_KCMUUWJ#-Ec7D0_R*$M>W`qql-y^AFp@kU`p zoY)U{HPc%y;e*QK+p`T%N%dSlDe5RdUHKhgMcnD+v*}Oiw$~8N{^c6W?t>&*5bVF2 zo~hW_jtoFj0&k&04e1>VU&PRdJ-fJGeB5YW z?iHG;a#%NC_IligZ^QQw6~%DD8k%SrimT71ngwGddHwXbNM*_1ed)o+L`}w8SA-dE zz1>9esL0~X*kYs6{?;D~5d0o~pKw)NxL2U6=jrkgi7~?yTa?a&U@Sf^<&KIzu5=_! zK|?ZEi$4kBz%-4SPr-kqj@j1GV!W)Q+&nsI-F{mloRSrQ#F5ajJXBckuivnL%T zUpWMwD>vUq6ju4(ya7AJ9PJ)kwRqA}m3mN}Z|T8qeO>H=?l_E@pKmTvKX>?EA?J0y z=yQavQolG>X*d@efVqX3S3)Z+Y<8OeZl&F?^0KpE!=ZC)_&aBci04il5BHCi45 z$mdN*(^9ERLvuaK{Gh=6!U=CG48^|T;`HeJlF#*y!+xf1Kp|=-B|an5iXV54%dWdq z2~KA9ypX6I`>Y?jzmP*|;@_lpIMg_4H@j~JAu7|o05ipuhl=W-euUy3@#?n8VrtHx zvh*gj^R_J z%Zn}h-HO;`s_hUKEj~LAT!E*=Eq;;8pPow;Sh#ar+#H4nYL0|a3fty4HoEci+I{>; zB?}l}g}nG^ZO!$4ctoq|=B#?(oBt*UDJ~P{BhSE{oq2kGWCYAZvc>9;4svz?gPk0n?+#s zzcLvZsRl%Q%Z3Y(DW~fO`w$2j%FqWc9M1o|k4T$qG@qGb9Y&;QM5Q(E=bP$|M@KD- zzf%&G_168>pmP3IA$?{B4cD+1pkA3p??~xL%DV#fgcDx_bPu-ylDsED9NKkN&7XSp zU)vQmdv$Y%ea0_O{A~L1q=<90X|{@ff7|j^t@FWNVe`Z`$GvAwQKcKA5zlJ1gyt?9 zH+JQ-P(ir7^Qm=R*>UMW(~}n;I1YLSOv=UeFCIn9U(7`_reCC-&{};Iw zffJh`?+mQ;RZKT=3KgfPu zF+V!V9Nn61zIdd3r0dJUCck$-*;@RmRXbvp@07g1^U?FzFEVK26HQ*chN&x9?sioX z6$z3hpFJto)&cnUU}Izuj+S_AC7E5=m?McR@%#3?!pv_&*5v?zP?{h zYhEH(5~56G%UKnWQsWPYqInW;kzPF?AG;ZBs<1d-rR?t+t`Wlx7%nTlDV$ z((jY3D1kr))hDQ^n33p^v1OA#=%cftejLIjS1|z}@E%|gpi|1#`U^w~mTpBtdkiMP z0KLiV1ORLd?o~{FSR2WXc6TBvVGEyimJYaTlpDTrAjG-Av4QGLOg~NfVtxOP$WM)) zL^uU!U~3m8x089c3nkl|oIxlvkumX4&Y#1LIK6i<8*k5vEa%Ew@Gm62JOJtY`$h>zP{B?>JX;u}G-M+=|B!Snqr#hp z6hOFgv?J+_hwlQfd+&8^64mS{H-x<16a%^vU<@WxtiU(HI7bYT!5qXdAK z+w*oJe|g-_72!o2T_MI?`K*qLLok0yg6J=szJ9Oe_`o$vP4wd~yva{$ew|Y0Vl)yB zLXMj-Jbm7T=<{*vx*hU_{a3_sQIIKy#7U{l6|r!K^>yO%31B2WJ=}DW(M2wi)h$RP^tP19sE$wE4R=UXYtbmnb@1<&j5bv2tjxF z6hfcZu{=AQR$CPe@(}hwMt&@m>$;^4Ub8tgUKx9F=P|e)!qSe>^S6k4!EOLOlL2~D z$pIjsH;{0=Xod@0bgw0%9DoMqxYw!~T>&ODHrcGx!MRcKY?bBZp4pbiO*>_vT|MO2 z%#5_8ca$$RQt)dUnj3(IFaVSuEJ-d-9vIsnVhk>cPi@n+v>bg;4U``iI?=Gd(Co*M#3JJ7J+uamhqL2ur-4F7$wb|Fdf z;*t~+G6BHn%ev2h42<2Y0f+%~4fXC+K*&l2!DZtB%Vl^Xb+yv)+TG z>Uy>1)z03?6n@Mu!t&Hc-Py`@6kqp?B}x(aT2;dyyMh8K-%s=WqTSt(^Q~U+>Ucqs z$?h0I1W~QRK!+y!HJoUROgfMI4hy zvK91hrvn$8>k^`*ozCS}r@MBQH~ImLCi@NWR`4MO5Y<~PdIh! zDRh?+_POVy!h+c)$;?i*Mdt7p#ZsPHjQT0_ndZOOd^wkS)ER_BDeUrIj{fmvcln;2 z>;JWMm2pjWZG0d|NvD8FNQ#7X*Jz{%DBYqcNFy*B28?D(cev5r4I>Dq?~d!7bHzV6+GDw>l5dz^3Hg>;(!BeN72v1mj@Nw433ZzUsPc0o0qnV`owik5 z`U>*U0I!WQZEqix6yA(&G;voTznu3mx)VAfh-qu3haX#7L_>*mGO^Z*$8*$5;Tyk9b77(~z&-Xj15en|%Z%DzU62YeiaL z(wnfL6B|dmT<>;pi4r!KZzRes# z1=BAxPbdnGjxDFlHbgLL6l1C`Y-B$4?DqyfqBv(&$XX7HKn986AfnH8iWii}P>P!vU1b=IZGs3}C1@l* zFno@bR5zl_dWr{L_l$~vp`i)#gkF1cgrqdVw1P;HA@{OdAuJk}>5n3**Q&(q0K`&= zbSIG{vF|+e9wdA|{JKO%P>dq-u)ZgQ8~iK5c;KaIs2#-Z9ViW9n4lP6%d7atP6mrj z%#jVYjyQNRrp-t!&WXqik1OlnKG}ZFjz%9vVfgLN(EdIhCau?j`(t^>vcIn^?%9KIxml?VP@JxD&h!= z7vW1GJ&1}jdS6OYPOcvUS62o)_~cOkoKGF?gmq46)ED->)-PM6Ox}V zS>Tcu_MLxg$i#FqRYc37A}h#q_9vQ47tV~z$=JJ}cM%_c=NRi?nAZ$u2crL6YsAHS zUTzhgK1p>#)}bV*!7STruztuSd!G|m+P~j(5kv5NHsz3AlR^cnoiS?9qCbA4I! zyVTO_zxxdfct)QcKzTs=rb(W-fp_bK^Gmw?RvtTmRsE@**Gx)?HKxcSBN=;d@!#(M z-7*iX34NJ&KN_F$Zl7jDKZ)!gl~(Y*-D5_h4XAR0w=pMSJY+ovZ2)v*4+VQ?G4n{lStHJ-!w9}4&Kjoi;5a@X_) zoq(GLjvo&(gIB}!ywVTICD8Mc=#IXL*!F*SBQUnS6&fDwB&i*bWrN-{-d2?Kl|jyo zX~>F+x{QK+dM0>DM_|$p1%lh9JQWi2x?@q0r}5N<>+>e<2|eltjON2_1Y*bDSk|y3 z`Heohdb~7I`!BIbJZDpCQ`yr|JHGE^#)m1Vr^cH=n7M@MU9ugx+%G>O$*mfA*l+ob4yis=XuU&klIR>hJU25E(0dYXD zWzI@~Ps$Cg*72x?98mJe8#6XGW^@|n0WrRN`}SLLvFi}IDcsFw?{eAsWwZjCX8zi7 zwG0se=siC@RTI21S0MudWD@}afg_Kmz6g;mCz3xc8NF5jr643Uw3At_Tbgk5m!$H% z?+jEnwyFE_j&i=oxpO85eo;AL9tKuNjk>H z62km@*LQztAfv0f$K8%Bq)-fN#!^YYg1{|L*GXle#`5AEzEnECU8)kztkdOLPoXsR zQ295k>sm}Iernmrdvc7ag2c=uVV;#VuSX}x#a0_}GO-|_xLzwu%RpzYul<-ZJPEzw z*tC4k;&i1*BylVNEO!yT_Hgg}o2fE^Vef4C-A-*QL_nX`0v0kHWY%vVNhzQ=hx}@? z-aF@o5XFKOB+>? zhpV)@t^mO|C&%(&q5eV_6izAYVu~xB6;U5q`R@Ju(6J<4jwkzYbTeWY?9g`nmp$j^ z=5U6Tu<#5Z^$_ymzpulSk5g~Nm5|SxAX%X_!d>(8IZiNo(zhrQ7QB~PsB8nr-0ij7 z`8d1WD~C}hg&dWMs>n(27f2de6e|vt7i7ZSH0zs z&$*pTQeAb+NHnd4zwvlS?UFA%yMV$^g;eh6mj3A-JZ~dGa+jS`GOgDtd@<>J_FESS zLRsp%10A@<3^f+4hYy|V7likn>lfPEq+MOlZZ1y>tkV8v|Aar@;_S)Xh_WybkT)KP zuaNXLq|K=qSV#2T_gPaGbUd}m{Y(Owh^$HvaH!w%Fz@=<|1kJ)V0a9TK}10fE`z1m zNWC$FA$Sb1iP>Imv!%W7-~k#Xe5}lhQiXODzi?uB4{4?CFr%ve-NJ11W?RJ|YlT>Y)~W~)q(oVT7C zkCO8ZPc-h3R^FCk5!1ZDY_+aE-|-atplR7}sF5EKrxi6eW@edxF7@>F0UqOvRG|#m z$3;6a=#2*lf&=~vEGH-D>RJu_W41p3`=Q}I@?qQJn9cqIavbW!n|4}^5N>)3VP@t( zQC&De!hQwVYd?^vHqvl%@m^Nd*dTWYsTgINy&SR~P}?ICY=4M!HY#Yw>ZPmt!_s%X zeEnnlEur*OOh=;+xOiM1Cr{*p+xe~K*K7nQ3bX@@un%_6M`rC9Uq>q(4kS+wFNUJi zM@Hgzb|wWBq@W8-P6DLK=(r=4qnI9-X^qutKrJv7;6R|)32U35@6En=QA_98|KHII zL?|&h{Y0qf&DBk%>eg_P?OP3v<>2Sj!>O&j8;!KAZGZ0E&m|k}k0SB(0UKA}K2_j? z8=>4Elsyi9APYm)y?=IM61Hw=u*JRl0eP6UGOV)VF?n^iJ0;cVv^Xd{TN$c|_Us`1KlO%-o+Y{!A*$>HyB*Gw{| zGUccp{9&nrq!xg}yM>Nzl-IT#YY?zj@~x`&*boLmgdb4(Heam%(AA`(LPJW~VbUk5 ziEf2x+cC;Q_ww`eQ7DvQajm3;2X-4?qx%b!raC0@7a+aaeX$Af&=>)dZs&w*{FAxA zn~tRl)&9yIC|!g)di#ijG+P_l&ARAv8{=Iha?hxzj4L8)>q`JJC~ zx3~@dq*1z0e69!38PHE_@ONu>LH8AQ8sNI*y1j20{`wHe+0OgYfsA>N%Z1?eo{yv_ zP4$(Q<%xb^WoZobOE}1{El_*3-2XT1{?YpcH04e{jJF044 z$2IzsM^B-`C$p6{pQ9`%C;4n9miOr_;8E%^P0LZ(h$4MhXU+pg+ywqZbruY^%5c<9 z(5{AtcT%@kHfq%`iq#Y=Ge4i@Ykx4y?YH-4?E#pe`=gQ7k^Qa8&d|HP;kzP>2?wsP zg-B%rLc-Sy^nU|`j?#MV4xD$tmYcDP$rIn|vtBvO_0eVabccB**@6k;ZJ2LY7xRnW6Jfx`FicdrlJ zo`}_z<`o-%o8eXnV%fkvH5+ZV{Ag?-B)Yi}AcFs-HidMuSLB59;?#tUSPyG6ID%48 zP>=;>@hq8or_ps+&ve-jWacoMEq`J=$%?YbPN1f)8=qTPoF)w4s>CVR^qw8-uG}RB zj2C*Z7`uQtHLY`8o#h@5dhG_w7FW%g7K{~A04ubTGrqLL%tWEyxy%7-^fz1G&8#G0 zJiWT+hIr9k7q>v9UVDlE+kVHfF!6+wU=kOR^655QrcEwuC)`*b3L%T-eA6Vo8?{qh zH{ZwP&sa+@Y&zo%$7VDyc4}K+>vw-GHCnpp(LrYRZPj%^;5<-xkW*Triud{~V*dr@ z;X;$TE-4YwTuB((yTiRJk7ZWq=j|l<$_x9~mi!i`2ImD!jG;H%?FUQkgS#2NXY<^* zYy0kev%jioRE!KwpE}P?&K@xD0u{CE%Oo{?0(KDys<5zsb8JwkL}Ub@TW1*GE+8T( z8@#ZbGv}@z&#LWgMn3O45^<*1FMi{?dW<+¬@>0hl{H+i8|{NRrW)M4O4J+|Y-3 z0&}`&GDe*W%J)S?77k-J9>XUJ8T|>$_g-!XPMIfL+zLk%Oj|mf=));-$DRM)ZS)td zdOr+ZAVrIZiC`4h&Q-$j%89IrTcL$~cwK8Hf?7YrDsW{s1&6+shAr=Ji#x7VuF`G^ zb7lTwxqW~wUuOk828)=OSmzH7MvfsTz;J1CY;0_F^wZ|)*V=ip&4F}9FvhYS*P`!* zTmN9;;EN2}tCz50K-x{_1ESV?zoOqZ+UMwUbjt7pWr^U$qkKyN*TQtKgHy^L?zDSU zPVwzZmYms}I>(Pt2RuGc_(vvmHooCBvD(s7?v%4@EbY!;0}CRNSwyeE7`=}Q9)7+) zI6qHoom-GfNR#sA(vjxKjqA_H^73kEY~(hr9~DYXYBXaYcCLz8)76#>gl1Or(IklX zn0$CODeYSedz#O{L6FJ@VEcAIp$pl)UJzl`b|TWTUjeeR%e#wKQ&SzEqqT0G0(J5^ z__L?v?5U6bp)|xt0pJY;!6<~!{8q`%j)Pb$^V87HpkWF@YF*5Dk>(9rSAD3G%r>`% z5Hhz5ozt|!vH#19Nc>8J+q(Yz5QmUK19Su}2<-xYYYeTskOZLU@1nk)s8>F0(WEu= z*EcjY1#p`k2T!>77m{M(I>Y2=TYkYiWt#4U*`KSmh3BVXO!>CVnyC9qp7}yl^ znG6FDtoz^5t%Xu?daaIiD+9P7A|B_}M`m4%Dmb}~?t4Y0$QGBgzILWiG!6_aGCBsn zWks{lfAd*Oa846fPAa`_h)$BOLYzR8NBiFM>0}%T2KPhX%k273`D%TK8U;Qq40;~= z4n|M+_7}@MJ40UWK4@uVDxBy`7#}eBlr2A zrUvSq-=~{u;I2|BODF$ln4C7`pUt*K;+}TQcA}+z1I8ecr&lp<4t%aR@ah?ftAKzE zSIT2L7AFs-Le>wBKJUkXEO*AK+k(P;7w@iJurAOd3v9(8Vd+&86RTa)P+1&zWEI{HZ z**e3r0xBBsSTTnGnLQoc6{;yRnXNPXq-yBEw)mX*S)!IcO?15utJLA1GVjUn-;N%H z@0t50Fy-U|k@h_9?PG<5#$*_3C~&9TgL;>SlC0IeSw0afvPbynQO@FME9KX4Hf*!4 zh18J8#{0FRqM{(_B!i6CP2bFi7NQ#Ev?xY3rF3~P`F7sDOiEB%H42rBO$^(4U+DZx zjp)y)t2HAdqUEmLg(XqKOV8M~sJ4Lnw|uXcQje^8SUC_Mzy?4U4UEZd@U%hqH^cXE zZ-QxW$?3B5C&BosIch9I03>h7XiMu{;3qpLEK`(lBwUj!el&tF@f|GL$YQSbJeD{7XX^ZzIQ_LA|1&Mfh?c zd%ETJ$~0B%#9Gly%1PX8!HzwpE|;Z4IPJCE!!r3?^!&G<`m6Km{1HNW1#Txn}#k6;A8#1qSNYc(#0) z`X8$~y7S%G!MaLVxww$JkngiVVMgjMse@=}Xs{R5KbpV4n)Yq>JlW({PeYd;)l!;q z)?EP7nTCRbX#%vrNvwwv#>QZ3ZIu+ltISaal!cbyZWV8pz=zkhwa zm*%jRTQKazt<1&J#HZ-w>o+>ird?rwh zw>f+v{;}S(va!&xt)pC|(pJBGYI<_WSa|uQ=}U^x!P35744buc{cL3AI-ndsU$}EQ zyO1hU4Hfj+)$_`vcOpFnF}?-r9u0?OiDdGoBbg+%5WS)|Pi&4dRvIzQdgGX9?Q3Gv zs%CnBmEK>KhHW3--yJ?*qKfDSTnT}ctgGk$D7ZH-Qw?hFg1-zgFbbhbos@M;$Sb(+ zklx5ft6-d=_F$SQx(sjm-=&# zskVV|Yyi@QIQqpt9|iJ69gJGP1X60Yi`iI-m`CzVCZ3!EPBFmA`?1!QqsdE8QV&Ae z9ADvmd0Z%z%=!mamR(xvFvcZ~ngcoxzIWcs7oD2h9ZSTcQa7Daa91W`=J?+8^Yh-Q zw|~!A-fqWjf-KD5yY_Ij2bq2r(gWNFmK%KR7HC9dVHJ`PL`@QHH)#n~lIP*$(#91O><%jGH}QhVemGZH=bz*D5TDWl9Id#ju#pPB*l$mh>F~V96f!f!-VdvI9W`xsZg7&xGD+MXi_vGMU38rRl>@`Se~lw z(6=Yl+l}6r?*t3teU=X#L@spVBVfSB0CX`-4GqQk`Exly(E{FoIwmJ3tSVOf9ME#u>-)2yh)EEYcuG{V@dRxne|AB zA^1MN=y?@9A+Z8+ak#n(sebaxFxEQ~+_0EuBw_T}xYGxh{GB@;7tZLK|A{F8-N*8B z-dLp#Cz0g~`DdJ-Jp<`mz%r!Ao|~VKknl}^nK%AlFxAL!aDYqP>rB`; zpd1=X6$uOsjBk!@DZi?}JlWC^$J0Ctz@z!9rU!gdKu5+ieJd3&pzY!oI9P>+6V(e8 zm*4+g9!GW9FelisVN+8{39(6V>TXuqaThHXh-4$=>aeEPBpP{01RwI(;S7w$IN2Oj zozUfTCTkXbJ?kvWz=+gEH8&g2Mw-~qU`XH9Nz+gRMa7hue)|j7jJ&O-41_WyqLi{F zTri0Zo5@#k(NK^Dlok8Ub(9Wc>m1~^^vpiLr)On7Fdf&PP8>Mi$E zh2Acof2|c7+zl|gxw!@t&Z8nDA{(X0M0s($q&JVO!N&_Z?+Ia~%fuE@*rIwQ1F$wq zVumut9hJwiE^JJzsAo*!?N#>!1CEv=HDp+5{le$lVb4K1Qep{;F?jwkB{ITSXdvlq zFewKk14^wc9MF&ec1P-FKMUA1mH59iK|uJEp8oq@CaML63nHN_4^X`0$sRpxdc%`7 z+#T?X>Agl#RG3J{yZG;7&6;lXZrS0P8=^7pMPGSPh=N&LB4CDmISh`PArHDzfzX6o zYti+H8gsg7oI8Webf-~l>Pq}>Pz|nuCfUAG6~f99#4$J;RmZktiA(+u1psG#fKE&L z97ICG$ifa�YA|kuD1Qp&+&|DOQHs-0$I+Db3?yA|eG*E~V&074}BDr=MQiX31vX z9s}b}!gmRpv^TGEG0buEM=<{nW<^?zy`C)~r%oPJG}26eo4#kun`E6mrqbgiL7l=l z$;KHC68=+E|N6}JBJJVC-@eYe-6jq$I&`PpmKVZf)ys!B@c)ZFU_$bo$b8jk3QHi1 zS3Q>cxLwBkP$l5{$C}7PW(VU@(#Oxw&FsTBbxtL*x*!78Zs`Ab?yYJ!R>%5vV)nOW xJexEeC$>pH(-ZR8QZq?^E*dZPYuYV#zw?#k1T9tX>8XH#pR&9> ~/.bashrc -echo "export CXX=clang++-16" >> ~/.bashrc +echo "export CC=clang-18" >> ~/.bashrc +echo "export CXX=clang++-18" >> ~/.bashrc source ~/.bashrc \ No newline at end of file From 0ed44a4289f2c725aa18565083c21d847a94029c Mon Sep 17 00:00:00 2001 From: Wenzheng Liu Date: Wed, 31 Jul 2024 23:45:26 +0800 Subject: [PATCH 35/61] [GLUTEN-6590] Support compact mergetree file on s3 (#6591) --- ...tenClickHouseMergeTreeWriteOnS3Suite.scala | 47 ++++++++++++-- cpp-ch/local-engine/Common/CHUtil.cpp | 4 +- .../CompactObjectStorageDiskTransaction.cpp | 1 + .../Disks/ObjectStorages/GlutenDiskS3.cpp | 62 +++++++++++++++++++ .../Disks/ObjectStorages/GlutenDiskS3.h | 57 +++++++++++++++++ .../Disks/registerGlutenDisks.cpp | 12 +++- .../Mergetree/SparkMergeTreeWriter.cpp | 40 ++++-------- 7 files changed, 185 insertions(+), 38 deletions(-) create mode 100644 cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskS3.cpp create mode 100644 cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskS3.h diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteOnS3Suite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteOnS3Suite.scala index 30f443265caeb..c95b788583229 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteOnS3Suite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteOnS3Suite.scala @@ -30,6 +30,8 @@ import io.minio.messages.DeleteObject import java.io.File import java.util +import scala.concurrent.duration.DurationInt + // Some sqls' line length exceeds 100 // scalastyle:off line.size.limit @@ -43,6 +45,12 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite override protected val tpchQueries: String = rootPath + "queries/tpch-queries-ch" override protected val queriesResults: String = rootPath + "mergetree-queries-output" + private val client = MinioClient + .builder() + .endpoint(MINIO_ENDPOINT) + .credentials(S3_ACCESS_KEY, S3_SECRET_KEY) + .build() + override protected def createTPCHNotNullTables(): Unit = { createNotNullTPCHTablesInParquet(tablesPath) } @@ -60,11 +68,6 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite override protected def beforeEach(): Unit = { super.beforeEach() - val client = MinioClient - .builder() - .endpoint(MINIO_ENDPOINT) - .credentials(S3_ACCESS_KEY, S3_SECRET_KEY) - .build() if (client.bucketExists(BucketExistsArgs.builder().bucket(BUCKET_NAME).build())) { val results = client.listObjects(ListObjectsArgs.builder().bucket(BUCKET_NAME).recursive(true).build()) @@ -168,9 +171,42 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite assertResult(1)(addFiles.size) assertResult(600572)(addFiles.head.rows) } + eventually(timeout(10.seconds), interval(2.seconds)) { + verifyS3CompactFileExist("lineitem_mergetree_s3") + } spark.sql("drop table lineitem_mergetree_s3") // clean up } + private def verifyS3CompactFileExist(table: String): Unit = { + val args = ListObjectsArgs + .builder() + .bucket(BUCKET_NAME) + .recursive(true) + .prefix(table) + .build() + var objectCount: Int = 0 + var metadataGlutenExist: Boolean = false + var metadataBinExist: Boolean = false + var dataBinExist: Boolean = false + client + .listObjects(args) + .forEach( + obj => { + objectCount += 1 + if (obj.get().objectName().contains("metadata.gluten")) { + metadataGlutenExist = true + } else if (obj.get().objectName().contains("meta.bin")) { + metadataBinExist = true + } else if (obj.get().objectName().contains("data.bin")) { + dataBinExist = true + } + }) + assertResult(5)(objectCount) + assert(metadataGlutenExist) + assert(metadataBinExist) + assert(dataBinExist) + } + test("test mergetree write with orderby keys / primary keys") { spark.sql(s""" |DROP TABLE IF EXISTS lineitem_mergetree_orderbykey_s3; @@ -635,6 +671,7 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite withSQLConf( "spark.databricks.delta.optimize.minFileSize" -> "200000000", + "spark.gluten.sql.columnar.backend.ch.runtime_settings.mergetree.insert_without_local_storage" -> "true", "spark.gluten.sql.columnar.backend.ch.runtime_settings.mergetree.merge_after_insert" -> "true" ) { spark.sql(s""" diff --git a/cpp-ch/local-engine/Common/CHUtil.cpp b/cpp-ch/local-engine/Common/CHUtil.cpp index b6867f656a07e..e642a67a5ff3d 100644 --- a/cpp-ch/local-engine/Common/CHUtil.cpp +++ b/cpp-ch/local-engine/Common/CHUtil.cpp @@ -569,7 +569,7 @@ std::vector BackendInitializerUtil::wrapDiskPathConfig( if (path_prefix.empty() && path_suffix.empty()) return changed_paths; Poco::Util::AbstractConfiguration::Keys disks; - std::unordered_set disk_types = {"s3", "hdfs_gluten", "cache"}; + std::unordered_set disk_types = {"s3_gluten", "hdfs_gluten", "cache"}; config.keys("storage_configuration.disks", disks); std::ranges::for_each( @@ -590,7 +590,7 @@ std::vector BackendInitializerUtil::wrapDiskPathConfig( changed_paths.emplace_back(final_path); } } - else if (disk_type == "s3" || disk_type == "hdfs_gluten") + else if (disk_type == "s3_gluten" || disk_type == "hdfs_gluten") { String metadata_path = config.getString(disk_prefix + ".metadata_path", ""); if (!metadata_path.empty()) diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.cpp b/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.cpp index 7a3ba4bed2449..66c4470101a19 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.cpp +++ b/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.cpp @@ -94,6 +94,7 @@ void CompactObjectStorageDiskTransaction::commit() whole_meta.addObject(key, 0, offset); metadata_tx->writeStringToFile(local_path, whole_meta.serializeToString()); out.sync(); + out.finalize(); }; merge_files(files | std::ranges::views::filter([](auto file) { return !isMetaDataFile(file.first); }), *data_write_buffer, data_key, data_path); diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskS3.cpp b/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskS3.cpp new file mode 100644 index 0000000000000..4a73c5a4998b7 --- /dev/null +++ b/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskS3.cpp @@ -0,0 +1,62 @@ +/* + * 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. + */ + +#pragma once + + +#include "GlutenDiskS3.h" +#include +#include +#include "CompactObjectStorageDiskTransaction.h" + +#if USE_AWS_S3 +namespace local_engine +{ + + DB::DiskTransactionPtr GlutenDiskS3::createTransaction() + { + return std::make_shared(*this, SerializedPlanParser::global_context->getTempDataOnDisk()->getVolume()->getDisk()); + } + + std::unique_ptr GlutenDiskS3::readFile( + const String & path, + const ReadSettings & settings, + std::optional read_hint, + std::optional file_size) const + { + ReadSettings copy_settings = settings; + // Threadpool read is not supported for s3 compact version currently + copy_settings.remote_fs_method = RemoteFSReadMethod::read; + return DiskObjectStorage::readFile(path, copy_settings, read_hint, file_size); + } + + DiskObjectStoragePtr GlutenDiskS3::createDiskObjectStorage() + { + const auto config_prefix = "storage_configuration.disks." + name; + return std::make_shared( + getName(), + object_key_prefix, + getMetadataStorage(), + getObjectStorage(), + SerializedPlanParser::global_context->getConfigRef(), + config_prefix, + object_storage_creator); + } + +} + +#endif diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskS3.h b/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskS3.h new file mode 100644 index 0000000000000..4f0d7a029529c --- /dev/null +++ b/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskS3.h @@ -0,0 +1,57 @@ +/* + * 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. + */ + +#pragma once + + +#include +#include +#include "CompactObjectStorageDiskTransaction.h" + +#if USE_AWS_S3 +namespace local_engine +{ +class GlutenDiskS3 : public DB::DiskObjectStorage +{ +public: + GlutenDiskS3( + const String & name_, + const String & object_key_prefix_, + DB::MetadataStoragePtr metadata_storage_, + DB::ObjectStoragePtr object_storage_, + const Poco::Util::AbstractConfiguration & config, + const String & config_prefix, + std::function creator) + : DiskObjectStorage(name_, object_key_prefix_, metadata_storage_, object_storage_, config, config_prefix), + object_storage_creator(creator) {} + + DB::DiskTransactionPtr createTransaction() override; + + std::unique_ptr readFile( + const String & path, + const ReadSettings & settings, + std::optional read_hint, + std::optional file_size) const override; + + DiskObjectStoragePtr createDiskObjectStorage() override; + +private: + std::function object_storage_creator; +}; +} + +#endif diff --git a/cpp-ch/local-engine/Disks/registerGlutenDisks.cpp b/cpp-ch/local-engine/Disks/registerGlutenDisks.cpp index 8a920edcce77a..52398b5f2fa74 100644 --- a/cpp-ch/local-engine/Disks/registerGlutenDisks.cpp +++ b/cpp-ch/local-engine/Disks/registerGlutenDisks.cpp @@ -25,6 +25,10 @@ #include #endif +#if USE_AWS_S3 +#include +#endif + #include "registerGlutenDisks.h" namespace local_engine @@ -52,16 +56,20 @@ void registerGlutenDisks(bool global_skip_access_check) bool) -> DB::DiskPtr { bool skip_access_check = global_skip_access_check || config.getBool(config_prefix + ".skip_access_check", false); + auto object_storage_creator = [name, skip_access_check, config_prefix]( + const Poco::Util::AbstractConfiguration & conf, DB::ContextPtr ctx) -> DB::ObjectStoragePtr + { return DB::ObjectStorageFactory::instance().create(name, conf, config_prefix, ctx, skip_access_check); }; auto object_storage = DB::ObjectStorageFactory::instance().create(name, config, config_prefix, context, skip_access_check); auto metadata_storage = DB::MetadataStorageFactory::instance().create(name, config, config_prefix, object_storage, "local"); - DB::DiskObjectStoragePtr disk = std::make_shared( + DB::DiskObjectStoragePtr disk = std::make_shared( name, object_storage->getCommonKeyPrefix(), std::move(metadata_storage), std::move(object_storage), config, - config_prefix); + config_prefix, + object_storage_creator); disk->startup(context, skip_access_check); return disk; diff --git a/cpp-ch/local-engine/Storages/Mergetree/SparkMergeTreeWriter.cpp b/cpp-ch/local-engine/Storages/Mergetree/SparkMergeTreeWriter.cpp index caee87cb9416c..6fee65efe5931 100644 --- a/cpp-ch/local-engine/Storages/Mergetree/SparkMergeTreeWriter.cpp +++ b/cpp-ch/local-engine/Storages/Mergetree/SparkMergeTreeWriter.cpp @@ -202,42 +202,24 @@ void SparkMergeTreeWriter::commitPartToRemoteStorageIfNeeded() auto read_settings = context->getReadSettings(); auto write_settings = context->getWriteSettings(); Stopwatch watch; - - // Temporary support for S3 - bool s3_disk = dest_storage->getStoragePolicy()->getAnyDisk()->getName().contains("s3"); for (const auto & merge_tree_data_part : new_parts.unsafeGet()) { String local_relative_path = storage->getRelativeDataPath() + "/" + merge_tree_data_part->name; String remote_relative_path = dest_storage->getRelativeDataPath() + "/" + merge_tree_data_part->name; - if (s3_disk) + std::vector files; + storage->getStoragePolicy()->getAnyDisk()->listFiles(local_relative_path, files); + auto src_disk = storage->getStoragePolicy()->getAnyDisk(); + auto dest_disk = dest_storage->getStoragePolicy()->getAnyDisk(); + auto tx = dest_disk->createTransaction(); + for (const auto & file : files) { - storage->getStoragePolicy()->getAnyDisk()->copyDirectoryContent( - local_relative_path, - dest_storage->getStoragePolicy()->getAnyDisk(), - remote_relative_path, - read_settings, - write_settings, - nullptr); + auto read_buffer = src_disk->readFile(local_relative_path + "/" + file, read_settings); + auto write_buffer = tx->writeFile(remote_relative_path + "/" + file, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, write_settings); + copyData(*read_buffer, *write_buffer); + write_buffer->finalize(); } - else - { - std::vector files; - storage->getStoragePolicy()->getAnyDisk()->listFiles(local_relative_path, files); - auto src_disk = storage->getStoragePolicy()->getAnyDisk(); - auto dest_disk = dest_storage->getStoragePolicy()->getAnyDisk(); - auto tx = dest_disk->createTransaction(); - for (const auto & file : files) - { - auto read_buffer = src_disk->readFile(local_relative_path + "/" + file, read_settings); - auto write_buffer = tx->writeFile(remote_relative_path + "/" + file, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, write_settings); - copyData(*read_buffer, *write_buffer); - write_buffer->finalize(); - } - tx->commit(); - } - - + tx->commit(); LOG_DEBUG( &Poco::Logger::get("SparkMergeTreeWriter"), "Upload part {} to disk {} success.", From fdd37fc83c971db08b0ff7038e8c97aec28bdb71 Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Thu, 1 Aug 2024 07:55:12 +0800 Subject: [PATCH 36/61] [VL] Daily Update Velox Version (2024_08_01) (#6664) --- ep/build-velox/src/get_velox.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index d8dfa09d45755..6536afcb1c7a0 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_07_31 +VELOX_BRANCH=2024_08_01 VELOX_HOME="" OS=`uname -s` From 49e2d17359444cd64263bdd8ec30032abff3d617 Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Thu, 1 Aug 2024 08:17:25 +0800 Subject: [PATCH 37/61] [VL] Gluten-it: --auto-cluster-resource to automatically set up CPU cores and memory sizes for local cluster (#6655) --- .github/workflows/velox_docker.yml | 18 +- .../apache/gluten/integration/BaseMixin.java | 11 +- .../integration/command/DataGenMixin.java | 2 +- .../integration/command/SparkRunModes.java | 310 ++++++++++++++---- .../org/apache/gluten/integration/Suite.scala | 3 - .../clickbench/ClickBenchSuite.scala | 2 - .../gluten/integration/ds/TpcdsSuite.scala | 2 - .../gluten/integration/h/TpchSuite.scala | 2 - 8 files changed, 261 insertions(+), 89 deletions(-) diff --git a/.github/workflows/velox_docker.yml b/.github/workflows/velox_docker.yml index ca3317d12d40e..913a55cbd4225 100644 --- a/.github/workflows/velox_docker.yml +++ b/.github/workflows/velox_docker.yml @@ -296,7 +296,7 @@ jobs: cd tools/gluten-it \ && GLUTEN_IT_JVM_ARGS=-Xmx3G sbin/gluten-it.sh parameterized \ --local --preset=velox --benchmark-type=ds --error-on-memleak --queries=q67,q95 -s=30.0 --threads=12 --shuffle-partitions=72 --iterations=1 \ - --data-gen-strategy=skip -m=OffHeapExecutionMemory \ + --data-gen=skip -m=OffHeapExecutionMemory \ -d=ISOLATION:OFF,spark.gluten.memory.isolation=false \ -d=OFFHEAP_SIZE:6g,spark.memory.offHeap.size=6g \ -d=OFFHEAP_SIZE:4g,spark.memory.offHeap.size=4g \ @@ -308,7 +308,7 @@ jobs: cd tools/gluten-it \ && GLUTEN_IT_JVM_ARGS=-Xmx3G sbin/gluten-it.sh parameterized \ --local --preset=velox --benchmark-type=ds --error-on-memleak --queries=q67 -s=30.0 --threads=12 --shuffle-partitions=72 --iterations=1 \ - --data-gen-strategy=skip -m=OffHeapExecutionMemory \ + --data-gen=skip -m=OffHeapExecutionMemory \ -d=ISOLATION:ON,spark.gluten.memory.isolation=true,spark.memory.storageFraction=0.1 \ -d=OFFHEAP_SIZE:6g,spark.memory.offHeap.size=6g \ -d=OFFHEAP_SIZE:4g,spark.memory.offHeap.size=4g \ @@ -319,7 +319,7 @@ jobs: cd tools/gluten-it \ && GLUTEN_IT_JVM_ARGS=-Xmx3G sbin/gluten-it.sh parameterized \ --local --preset=velox --benchmark-type=ds --error-on-memleak --queries=q95 -s=30.0 --threads=12 --shuffle-partitions=72 --iterations=1 \ - --data-gen-strategy=skip -m=OffHeapExecutionMemory \ + --data-gen=skip -m=OffHeapExecutionMemory \ -d=ISOLATION:ON,spark.gluten.memory.isolation=true,spark.memory.storageFraction=0.1 \ -d=OFFHEAP_SIZE:6g,spark.memory.offHeap.size=6g \ -d=OFFHEAP_SIZE:4g,spark.memory.offHeap.size=4g \ @@ -330,7 +330,7 @@ jobs: cd tools/gluten-it \ && GLUTEN_IT_JVM_ARGS=-Xmx3G sbin/gluten-it.sh parameterized \ --local --preset=velox --benchmark-type=ds --error-on-memleak --queries=q23a,q23b -s=30.0 --threads=12 --shuffle-partitions=72 --iterations=1 \ - --data-gen-strategy=skip -m=OffHeapExecutionMemory \ + --data-gen=skip -m=OffHeapExecutionMemory \ -d=ISOLATION:OFF,spark.gluten.memory.isolation=false \ -d=OFFHEAP_SIZE:2g,spark.memory.offHeap.size=2g \ -d=FLUSH_MODE:DISABLED,spark.gluten.sql.columnar.backend.velox.flushablePartialAggregation=false,spark.gluten.sql.columnar.backend.velox.maxPartialAggregationMemoryRatio=1.0,spark.gluten.sql.columnar.backend.velox.maxExtendedPartialAggregationMemoryRatio=1.0,spark.gluten.sql.columnar.backend.velox.abandonPartialAggregationMinPct=100,spark.gluten.sql.columnar.backend.velox.abandonPartialAggregationMinRows=0 \ @@ -341,7 +341,7 @@ jobs: cd tools/gluten-it \ && GLUTEN_IT_JVM_ARGS=-Xmx3G sbin/gluten-it.sh parameterized \ --local --preset=velox --benchmark-type=ds --error-on-memleak --queries=q23a,q23b -s=30.0 --threads=12 --shuffle-partitions=72 --iterations=1 \ - --data-gen-strategy=skip -m=OffHeapExecutionMemory \ + --data-gen=skip -m=OffHeapExecutionMemory \ -d=ISOLATION:ON,spark.gluten.memory.isolation=true,spark.memory.storageFraction=0.1 \ -d=OFFHEAP_SIZE:2g,spark.memory.offHeap.size=2g \ -d=FLUSH_MODE:DISABLED,spark.gluten.sql.columnar.backend.velox.flushablePartialAggregation=false,spark.gluten.sql.columnar.backend.velox.maxPartialAggregationMemoryRatio=1.0,spark.gluten.sql.columnar.backend.velox.maxExtendedPartialAggregationMemoryRatio=1.0,spark.gluten.sql.columnar.backend.velox.abandonPartialAggregationMinPct=100,spark.gluten.sql.columnar.backend.velox.abandonPartialAggregationMinRows=0 \ @@ -352,7 +352,7 @@ jobs: cd tools/gluten-it \ && GLUTEN_IT_JVM_ARGS=-Xmx3G sbin/gluten-it.sh parameterized \ --local --preset=velox --benchmark-type=ds --error-on-memleak --queries=q97 -s=30.0 --threads=12 --shuffle-partitions=72 --iterations=1 \ - --data-gen-strategy=skip -m=OffHeapExecutionMemory \ + --data-gen=skip -m=OffHeapExecutionMemory \ -d=ISOLATION:OFF,spark.gluten.memory.isolation=false \ -d=ISOLATION:ON,spark.gluten.memory.isolation=true,spark.memory.storageFraction=0.1 \ -d=OFFHEAP_SIZE:2g,spark.memory.offHeap.size=2g \ @@ -408,7 +408,7 @@ jobs: cd tools/gluten-it \ && GLUTEN_IT_JVM_ARGS=-Xmx6G sbin/gluten-it.sh queries \ --local --preset=velox --benchmark-type=ds --error-on-memleak -s=30.0 --off-heap-size=8g --threads=12 --shuffle-partitions=72 --iterations=1 \ - --data-gen-strategy=skip --random-kill-tasks --no-session-reuse + --data-gen=skip --random-kill-tasks --no-session-reuse # run-tpc-test-ubuntu-sf30: # needs: build-native-lib-centos-7 @@ -457,10 +457,10 @@ jobs: # cd tools/gluten-it \ # && GLUTEN_IT_JVM_ARGS=-Xmx6G sbin/gluten-it.sh queries-compare \ # --local --preset=velox --benchmark-type=h --error-on-memleak -s=30.0 --off-heap-size=8g --threads=12 --shuffle-partitions=72 --iterations=1 \ - # --data-gen-strategy=skip --shard=${{ matrix.shard }} \ + # --data-gen=skip --shard=${{ matrix.shard }} \ # && GLUTEN_IT_JVM_ARGS=-Xmx6G sbin/gluten-it.sh queries-compare \ # --local --preset=velox --benchmark-type=ds --error-on-memleak -s=30.0 --off-heap-size=8g --threads=12 --shuffle-partitions=72 --iterations=1 \ - # --data-gen-strategy=skip --shard=${{ matrix.shard }} + # --data-gen=skip --shard=${{ matrix.shard }} run-tpc-test-centos8-uniffle: needs: build-native-lib-centos-7 diff --git a/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/BaseMixin.java b/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/BaseMixin.java index 47aa0a0cb59ff..08c55d78a67a7 100644 --- a/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/BaseMixin.java +++ b/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/BaseMixin.java @@ -61,10 +61,7 @@ public class BaseMixin { private int hsUiPort; @CommandLine.ArgGroup(exclusive = true, multiplicity = "1") - SparkRunModes.ModeEnumeration runModeEnumeration; - - @CommandLine.Option(names = {"--off-heap-size"}, description = "Off heap memory size per executor", defaultValue = "6g") - private String offHeapSize; + SparkRunModes.Mode.Enumeration runModeEnumeration; @CommandLine.Option(names = {"--disable-aqe"}, description = "Disable Spark SQL adaptive query execution", defaultValue = "false") private boolean disableAqe; @@ -133,19 +130,19 @@ public Integer runActions(Action[] actions) { case "h": suite = new TpchSuite(runModeEnumeration.getSparkMasterUrl(), actions, testConf, baselineConf, extraSparkConfScala, level, errorOnMemLeak, dataDir, enableUi, - enableHsUi, hsUiPort, offHeapSize, disableAqe, disableBhj, + enableHsUi, hsUiPort, disableAqe, disableBhj, disableWscg, shufflePartitions, scanPartitions); break; case "ds": suite = new TpcdsSuite(runModeEnumeration.getSparkMasterUrl(), actions, testConf, baselineConf, extraSparkConfScala, level, errorOnMemLeak, dataDir, enableUi, - enableHsUi, hsUiPort, offHeapSize, disableAqe, disableBhj, + enableHsUi, hsUiPort, disableAqe, disableBhj, disableWscg, shufflePartitions, scanPartitions); break; case "clickbench": suite = new ClickBenchSuite(runModeEnumeration.getSparkMasterUrl(), actions, testConf, baselineConf, extraSparkConfScala, level, errorOnMemLeak, dataDir, enableUi, - enableHsUi, hsUiPort, offHeapSize, disableAqe, disableBhj, + enableHsUi, hsUiPort, disableAqe, disableBhj, disableWscg, shufflePartitions, scanPartitions); break; default: diff --git a/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/command/DataGenMixin.java b/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/command/DataGenMixin.java index 3854d078e261c..fa51054f0f92f 100644 --- a/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/command/DataGenMixin.java +++ b/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/command/DataGenMixin.java @@ -21,7 +21,7 @@ import picocli.CommandLine; public class DataGenMixin { - @CommandLine.Option(names = {"--data-gen-strategy"}, description = "The strategy of data generation, accepted values: skip, once, always", defaultValue = "always") + @CommandLine.Option(names = {"--data-gen"}, description = "The strategy of data generation, accepted values: skip, once, always", defaultValue = "always") private String dataGenStrategy; @CommandLine.Option(names = {"-s", "--scale"}, description = "The scale factor of sample TPC-H dataset", defaultValue = "0.1") diff --git a/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/command/SparkRunModes.java b/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/command/SparkRunModes.java index 56ef68db90573..6750b90e9e495 100644 --- a/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/command/SparkRunModes.java +++ b/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/command/SparkRunModes.java @@ -16,64 +16,50 @@ */ package org.apache.gluten.integration.command; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; import org.apache.spark.launcher.SparkLauncher; import org.apache.spark.util.Utils; import picocli.CommandLine; +import javax.management.*; import java.io.File; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.Optional; -import java.util.Set; +import java.lang.management.ManagementFactory; +import java.util.*; import java.util.stream.Collectors; public final class SparkRunModes { + private static T findNonNull(T... objects) { + final List nonNullObjects = Arrays.stream(objects).filter(Objects::nonNull).collect(Collectors.toList()); + Preconditions.checkState(nonNullObjects.size() == 1, "There are zero or more than one non-null objects: " + nonNullObjects); + return nonNullObjects.get(0); + } + public interface Mode { String getSparkMasterUrl(); - Map extraSparkConf(); - } - public static class ModeEnumeration implements Mode { - @CommandLine.ArgGroup(exclusive = false) - LocalMode localMode; + Map extraSparkConf(); - @CommandLine.ArgGroup(exclusive = false) - LocalClusterMode localClusterMode; + class Enumeration implements Mode { + @CommandLine.ArgGroup(exclusive = false) + LocalMode localMode; - private Mode getActiveMode() { - int enabledModeCount = 0; - if (localMode != null) { - enabledModeCount++; - } - if (localClusterMode != null) { - enabledModeCount++; - } + @CommandLine.ArgGroup(exclusive = false) + LocalClusterMode localClusterMode; - if (enabledModeCount != 1) { - throw new IllegalStateException("Only one single run mode can be specified"); + private Mode getActiveMode() { + return findNonNull(localMode, localClusterMode); } - if (localMode != null) { - return localMode; + @Override + public String getSparkMasterUrl() { + return getActiveMode().getSparkMasterUrl(); } - if (localClusterMode != null) { - return localClusterMode; + @Override + public Map extraSparkConf() { + return getActiveMode().extraSparkConf(); } - - throw new IllegalStateException("unreachable code"); - } - - @Override - public String getSparkMasterUrl() { - return getActiveMode().getSparkMasterUrl(); - } - - @Override - public Map extraSparkConf() { - return getActiveMode().extraSparkConf(); } } @@ -84,6 +70,9 @@ private static class LocalMode implements Mode { @CommandLine.Option(names = {"--threads"}, description = "Local mode: Run Spark locally with as many worker threads", defaultValue = "4") private int localThreads; + @CommandLine.Option(names = {"--off-heap-size"}, description = "Local mode: Total off-heap memory size", defaultValue = "6g") + private String offHeapSize; + @Override public String getSparkMasterUrl() { if (!enabled) { @@ -95,15 +84,71 @@ public String getSparkMasterUrl() { @Override public Map extraSparkConf() { - return Collections.emptyMap(); + return ImmutableMap.builder() + .put("spark.memory.offHeap.enabled", "true") + .put("spark.memory.offHeap.size", offHeapSize) + .build(); } } - private static class LocalClusterMode implements Mode { - // We should transfer the jars to be tested in the integration testing to executors - public static final String[] EXTRA_JARS = new String[]{"gluten-package-1.2.0-SNAPSHOT.jar"}; + private interface ClusterResource { + int lcWorkers(); - @CommandLine.Option(names = {"--local-cluster"}, description = "Run in Spark local cluster mode", required = true) + int lcWorkerCores(); + + long lcWorkerHeapMem(); // in MiB. + + int lcExecutorCores(); + + long lcExecutorHeapMem(); // in MiB. + + long lcExecutorOffHeapMem(); // in MiB. + + class Enumeration implements ClusterResource { + @CommandLine.ArgGroup(exclusive = false) + ManualClusterResource manual; + + @CommandLine.ArgGroup(exclusive = false) + AutoClusterResource auto; + + private ClusterResource getActive() { + return findNonNull(manual, auto); + } + + @Override + public int lcWorkers() { + return getActive().lcWorkers(); + } + + @Override + public int lcWorkerCores() { + return getActive().lcWorkerCores(); + } + + @Override + public long lcWorkerHeapMem() { + return getActive().lcWorkerHeapMem(); + } + + @Override + public int lcExecutorCores() { + return getActive().lcExecutorCores(); + } + + @Override + public long lcExecutorHeapMem() { + return getActive().lcExecutorHeapMem(); + } + + @Override + public long lcExecutorOffHeapMem() { + return getActive().lcExecutorOffHeapMem(); + } + } + } + + private static class ManualClusterResource implements ClusterResource { + @CommandLine.Option(names = {"--manual-cluster-resource"}, description = "Local cluster mode: Manually configure cluster resource", required = true) private boolean enabled; @CommandLine.Option(names = {"--workers"}, description = "Local cluster mode: Number of workers", defaultValue = "2") @@ -112,14 +157,163 @@ private static class LocalClusterMode implements Mode { @CommandLine.Option(names = {"--worker-cores"}, description = "Local cluster mode: Number of cores per worker", defaultValue = "2") private int lcWorkerCores; - @CommandLine.Option(names = {"--worker-mem"}, description = "Local cluster mode: Memory per worker", defaultValue = "4g") - private String lcWorkerMem; + @CommandLine.Option(names = {"--worker-heap-size"}, description = "Local cluster mode: Heap memory per worker", defaultValue = "4g") + private String lcWorkerHeapMem; @CommandLine.Option(names = {"--executor-cores"}, description = "Local cluster mode: Number of cores per executor", defaultValue = "1") private int lcExecutorCores; - @CommandLine.Option(names = {"--executor-mem"}, description = "Local cluster mode: Memory per executor", defaultValue = "2g") - private String lcExecutorMem; + @CommandLine.Option(names = {"--executor-heap-size"}, description = "Local cluster mode: Heap memory per executor", defaultValue = "2g") + private String lcExecutorHeapMem; + + @CommandLine.Option(names = {"--executor-off-heap-size"}, description = "Local cluster mode: Off-heap memory per executor", defaultValue = "6g") + private String lcExecutorOffHeapMem; + + @Override + public int lcWorkers() { + ensureEnabled(); + return lcWorkers; + } + + @Override + public int lcWorkerCores() { + ensureEnabled(); + return lcWorkerCores; + } + + @Override + public long lcWorkerHeapMem() { + ensureEnabled(); + return Utils.byteStringAsMb(lcWorkerHeapMem); + } + + @Override + public int lcExecutorCores() { + ensureEnabled(); + return lcExecutorCores; + } + + @Override + public long lcExecutorHeapMem() { + ensureEnabled(); + return Utils.byteStringAsMb(lcExecutorHeapMem); + } + + @Override + public long lcExecutorOffHeapMem() { + ensureEnabled(); + return Utils.byteStringAsMb(lcExecutorOffHeapMem); + } + + private void ensureEnabled() { + if (!enabled) { + throw new IllegalStateException("Manual cluster resource is not enabled"); + } + } + } + + private static class AutoClusterResource implements ClusterResource { + @CommandLine.Option(names = {"--auto-cluster-resource"}, description = "Local cluster mode: Automatically configure cluster resource", required = true) + private boolean enabled; + + private final int lcWorkers; + private final int lcWorkerCores; + private final long lcWorkerHeapMem; + private final int lcExecutorCores; + private final long lcExecutorHeapMem; + private final long lcExecutorOffHeapMem; + + public AutoClusterResource() { + final int totalCores = Runtime.getRuntime().availableProcessors(); + final long totalMem = (long) (getTotalMem() * 0.8); + Preconditions.checkState(totalMem >= 64, "--auto-cluster-resource mode requires for at least 64 MiB physical memory available. Current: " + totalMem); + Preconditions.checkState(totalCores >= 1, "--auto-cluster-resource mode requires for at least 1 CPU core available. Current: " + totalCores); + if (totalCores % 2 == 1) { + // Platform has an odd number of CPU cores. + this.lcWorkers = 1; + this.lcWorkerCores = totalCores; + this.lcExecutorCores = 1; + } else { + // Platform has an even number of CPU cores. + this.lcWorkers = 2; + this.lcWorkerCores = totalCores / this.lcWorkers; + if (lcWorkerCores % 2 == 1) { + this.lcExecutorCores = 1; + } else { + this.lcExecutorCores = 2; + } + } + Preconditions.checkState(totalCores % this.lcExecutorCores == 0); + final int numExecutors = totalCores / this.lcExecutorCores; + Preconditions.checkState(this.lcWorkerCores % this.lcExecutorCores == 0); + final int numExecutorsPerWorker = this.lcWorkerCores / this.lcExecutorCores; + final long executorMem = totalMem / numExecutors; + this.lcExecutorHeapMem = (long) (executorMem * 0.33); + this.lcExecutorOffHeapMem = (long) (executorMem * 0.67); + this.lcWorkerHeapMem = this.lcExecutorHeapMem * numExecutorsPerWorker; + System.out.printf("Automatically configured cluster resource settings: %n" + + " lcWorkers: [%d]%n" + + " lcWorkerCores: [%d]%n" + + " lcWorkerHeapMem: [%dMiB]%n" + + " lcExecutorCores: [%d]%n" + + " lcExecutorHeapMem: [%dMiB]%n" + + " lcExecutorOffHeapMem: [%dMiB]%n", + lcWorkers, + lcWorkerCores, + lcWorkerHeapMem, + lcExecutorCores, + lcExecutorHeapMem, + lcExecutorOffHeapMem); + } + + private static long getTotalMem() { + try { + final MBeanServer mBeanServer = ManagementFactory.getPlatformMBeanServer(); + final Object attribute = mBeanServer.getAttribute(new ObjectName("java.lang", "type", "OperatingSystem"), "TotalPhysicalMemorySize"); + final long totalMem = Long.parseLong(attribute.toString()) / 1024 / 1024; + return totalMem; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + public int lcWorkers() { + return lcWorkers; + } + + @Override + public int lcWorkerCores() { + return lcWorkerCores; + } + + @Override + public long lcWorkerHeapMem() { + return lcWorkerHeapMem; + } + + @Override + public int lcExecutorCores() { + return lcExecutorCores; + } + + @Override + public long lcExecutorHeapMem() { + return lcExecutorHeapMem; + } + + @Override + public long lcExecutorOffHeapMem() { + return lcExecutorOffHeapMem; + } + } + + private static class LocalClusterMode implements Mode { + @CommandLine.Option(names = {"--local-cluster"}, description = "Run in Spark local cluster mode", required = true) + private boolean enabled; + + @CommandLine.ArgGroup(exclusive = true, multiplicity = "1") + ClusterResource.Enumeration resourceEnumeration; @Override public String getSparkMasterUrl() { @@ -132,26 +326,16 @@ public String getSparkMasterUrl() { if (!System.getenv().containsKey("SPARK_SCALA_VERSION")) { throw new IllegalArgumentException("SPARK_SCALA_VERSION not set! Please set it first or use --local instead. Example: export SPARK_SCALA_VERSION=2.12"); } - return String.format("local-cluster[%d,%d,%d]", lcWorkers, lcWorkerCores, Utils.byteStringAsMb(lcWorkerMem)); + return String.format("local-cluster[%d,%d,%d]", resourceEnumeration.lcWorkers(), resourceEnumeration.lcWorkerCores(), resourceEnumeration.lcWorkerHeapMem()); } @Override public Map extraSparkConf() { - final Set extraJarSet = Arrays.stream(EXTRA_JARS).collect(Collectors.toSet()); - String classpath = System.getProperty("java.class.path"); - String[] classPathValues = classpath.split(File.pathSeparator); - Optional extraClassPath = Arrays.stream(classPathValues).filter(classPath -> { - File file = new File(classPath); - return file.exists() && file.isFile() && extraJarSet.contains(file.getName()); - }).map(classPath -> { - File file = new File(classPath); - return file.getAbsolutePath(); - }).reduce((s1, s2) -> s1 + File.pathSeparator + s2); - final Map extras = new HashMap<>(); - extras.put(SparkLauncher.EXECUTOR_CORES, String.valueOf(lcExecutorCores)); - extras.put(SparkLauncher.EXECUTOR_MEMORY, lcExecutorMem); - extraClassPath.ifPresent(path -> extras.put("spark.executor.extraClassPath", path)); + extras.put(SparkLauncher.EXECUTOR_CORES, String.valueOf(resourceEnumeration.lcExecutorCores())); + extras.put(SparkLauncher.EXECUTOR_MEMORY, String.format("%dm", resourceEnumeration.lcExecutorHeapMem())); + extras.put("spark.memory.offHeap.enabled", "true"); + extras.put("spark.memory.offHeap.size", String.format("%dm", resourceEnumeration.lcExecutorOffHeapMem())); return extras; } } diff --git a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/Suite.scala b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/Suite.scala index 070c43e9be2f9..64fc179ce4ad3 100644 --- a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/Suite.scala +++ b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/Suite.scala @@ -38,7 +38,6 @@ abstract class Suite( private val enableUi: Boolean, private val enableHsUi: Boolean, private val hsUiPort: Int, - private val offHeapSize: String, private val disableAqe: Boolean, private val disableBhj: Boolean, private val disableWscg: Boolean, @@ -75,8 +74,6 @@ abstract class Suite( sessionSwitcher .defaultConf() .setWarningOnOverriding("spark.unsafe.exceptionOnMemoryLeak", s"$errorOnMemLeak") - sessionSwitcher.defaultConf().setWarningOnOverriding("spark.memory.offHeap.enabled", "true") - sessionSwitcher.defaultConf().setWarningOnOverriding("spark.memory.offHeap.size", offHeapSize) if (!enableUi) { sessionSwitcher.defaultConf().setWarningOnOverriding("spark.ui.enabled", "false") diff --git a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/clickbench/ClickBenchSuite.scala b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/clickbench/ClickBenchSuite.scala index f75431941d7a2..da0c6b8ce0496 100644 --- a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/clickbench/ClickBenchSuite.scala +++ b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/clickbench/ClickBenchSuite.scala @@ -41,7 +41,6 @@ class ClickBenchSuite( val enableUi: Boolean, val enableHsUi: Boolean, val hsUiPort: Int, - val offHeapSize: String, val disableAqe: Boolean, val disableBhj: Boolean, val disableWscg: Boolean, @@ -58,7 +57,6 @@ class ClickBenchSuite( enableUi, enableHsUi, hsUiPort, - offHeapSize, disableAqe, disableBhj, disableWscg, diff --git a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/ds/TpcdsSuite.scala b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/ds/TpcdsSuite.scala index 190623614b9be..9e6fa1edb6431 100644 --- a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/ds/TpcdsSuite.scala +++ b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/ds/TpcdsSuite.scala @@ -36,7 +36,6 @@ class TpcdsSuite( val enableUi: Boolean, val enableHsUi: Boolean, val hsUiPort: Int, - val offHeapSize: String, val disableAqe: Boolean, val disableBhj: Boolean, val disableWscg: Boolean, @@ -53,7 +52,6 @@ class TpcdsSuite( enableUi, enableHsUi, hsUiPort, - offHeapSize, disableAqe, disableBhj, disableWscg, diff --git a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/h/TpchSuite.scala b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/h/TpchSuite.scala index 86fcaea0a6287..fc3ad13106495 100644 --- a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/h/TpchSuite.scala +++ b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/h/TpchSuite.scala @@ -36,7 +36,6 @@ class TpchSuite( val enableUi: Boolean, val enableHsUi: Boolean, val hsUiPort: Int, - val offHeapSize: String, val disableAqe: Boolean, val disableBhj: Boolean, val disableWscg: Boolean, @@ -53,7 +52,6 @@ class TpchSuite( enableUi, enableHsUi, hsUiPort, - offHeapSize, disableAqe, disableBhj, disableWscg, From edae5b8046aa79ea16cf3cb3b011d6eac1fe7acd Mon Sep 17 00:00:00 2001 From: Mingliang Zhu Date: Thu, 1 Aug 2024 08:46:25 +0800 Subject: [PATCH 38/61] [VL] Support date type in window range frame (#6653) --- .../gluten/execution/TestOperator.scala | 7 +++ .../columnar/rewrite/PullOutPreProject.scala | 22 +++---- .../gluten/utils/PullOutProjectHelper.scala | 58 ++++++++++++++++--- 3 files changed, 65 insertions(+), 22 deletions(-) diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/TestOperator.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/TestOperator.scala index 0fb5fb54900b4..86a62a4471a5e 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/TestOperator.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/TestOperator.scala @@ -354,6 +354,13 @@ class TestOperator extends VeloxWholeStageTransformerSuite with AdaptiveSparkPla Seq("sort", "streaming").foreach { windowType => withSQLConf("spark.gluten.sql.columnar.backend.velox.window.type" -> windowType) { + runQueryAndCompare( + "select max(l_partkey) over" + + " (partition by l_suppkey order by l_commitdate" + + " RANGE BETWEEN 1 PRECEDING AND CURRENT ROW) from lineitem ") { + checkSparkOperatorMatch[WindowExecTransformer] + } + runQueryAndCompare( "select max(l_partkey) over" + " (partition by l_suppkey order by l_orderkey" + diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/rewrite/PullOutPreProject.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/rewrite/PullOutPreProject.scala index 73b8ab2607eb4..51cdb76a15597 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/rewrite/PullOutPreProject.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/rewrite/PullOutPreProject.scala @@ -76,17 +76,7 @@ object PullOutPreProject extends RewriteSingleNode with PullOutProjectHelper { } case _ => false }.isDefined) || - window.windowExpression.exists(_.find { - case we: WindowExpression => - we.windowSpec.frameSpecification match { - case swf: SpecifiedWindowFrame - if needPreComputeRangeFrame(swf) && supportPreComputeRangeFrame( - we.windowSpec.orderSpec) => - true - case _ => false - } - case _ => false - }.isDefined) + windowNeedPreComputeRangeFrame(window) case plan if SparkShimLoader.getSparkShims.isWindowGroupLimitExec(plan) => val window = SparkShimLoader.getSparkShims .getWindowGroupLimitExecShim(plan) @@ -176,14 +166,16 @@ object PullOutPreProject extends RewriteSingleNode with PullOutProjectHelper { case window: WindowExec if needsPreProject(window) => val expressionMap = new mutable.HashMap[Expression, NamedExpression]() - // Handle orderSpec. - val newOrderSpec = getNewSortOrder(window.orderSpec, expressionMap) - - // Handle partitionSpec. + // Handle foldable orderSpec and foldable partitionSpec. Spark analyzer rule + // ExtractWindowExpressions will extract expressions from non-foldable orderSpec and + // partitionSpec. + var newOrderSpec = getNewSortOrder(window.orderSpec, expressionMap) val newPartitionSpec = window.partitionSpec.map(replaceExpressionWithAttribute(_, expressionMap)) // Handle windowExpressions. + newOrderSpec = rewriteOrderSpecs(window, newOrderSpec, expressionMap) + val newWindowExpressions = window.windowExpression.toIndexedSeq.map { _.transform { case we: WindowExpression => rewriteWindowExpression(we, newOrderSpec, expressionMap) diff --git a/gluten-core/src/main/scala/org/apache/gluten/utils/PullOutProjectHelper.scala b/gluten-core/src/main/scala/org/apache/gluten/utils/PullOutProjectHelper.scala index 12055f9e97210..85be57493f02e 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/utils/PullOutProjectHelper.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/utils/PullOutProjectHelper.scala @@ -22,8 +22,10 @@ import org.apache.gluten.exception.{GlutenException, GlutenNotSupportException} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateFunction} import org.apache.spark.sql.execution.aggregate._ +import org.apache.spark.sql.execution.window.WindowExec import org.apache.spark.sql.types.{ByteType, DateType, IntegerType, LongType, ShortType} +import java.sql.Date import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable @@ -161,14 +163,32 @@ trait PullOutProjectHelper { case _: PreComputeRangeFrameBound => bound case _ if !bound.foldable => bound case _ if bound.foldable => + val orderExpr = if (expressionMap.contains(orderSpec.child)) { + expressionMap(orderSpec.child).asInstanceOf[Alias].child + } else { + orderSpec.child + } val a = expressionMap .getOrElseUpdate( bound.canonicalized, - Alias(Add(orderSpec.child, bound), generatePreAliasName)()) + Alias(Add(orderExpr, bound), generatePreAliasName)()) PreComputeRangeFrameBound(a.asInstanceOf[Alias], bound) } } + protected def windowNeedPreComputeRangeFrame(w: WindowExec): Boolean = + w.windowExpression.exists(_.find { + case we: WindowExpression => + we.windowSpec.frameSpecification match { + case swf: SpecifiedWindowFrame + if needPreComputeRangeFrame(swf) && supportPreComputeRangeFrame( + we.windowSpec.orderSpec) => + true + case _ => false + } + case _ => false + }.isDefined) + protected def needPreComputeRangeFrame(swf: SpecifiedWindowFrame): Boolean = { BackendsApiManager.getSettings.needPreComputeRangeFrameBoundary && swf.frameType == RangeFrame && @@ -185,6 +205,36 @@ trait PullOutProjectHelper { } } + /** + * Convert DateType to IntType for orderSpec if needPreComputeRangeFrame, because spark's frame + * type does not support DateType. It does not affect the correctness of sort. + */ + protected def rewriteOrderSpecs( + window: WindowExec, + orderSpecs: Seq[SortOrder], + expressionMap: mutable.HashMap[Expression, NamedExpression]): Seq[SortOrder] = { + if (windowNeedPreComputeRangeFrame(window)) { + // This is guaranteed by Spark, but we still check it here + if (orderSpecs.size != 1) { + throw new GlutenException( + s"A range window frame with value boundaries expects one and only one " + + s"order by expression: ${orderSpecs.mkString(",")}") + } + val orderSpec = orderSpecs.head + orderSpec.child.dataType match { + case DateType => + val alias = Alias( + DateDiff(orderSpec.child, Literal(Date.valueOf("1970-01-01"))), + generatePreAliasName)() + expressionMap.getOrElseUpdate(alias.toAttribute, alias) + Seq(orderSpec.copy(child = alias.toAttribute)) + case _ => orderSpecs + } + } else { + orderSpecs + } + } + protected def rewriteWindowExpression( we: WindowExpression, orderSpecs: Seq[SortOrder], @@ -202,12 +252,6 @@ trait PullOutProjectHelper { val newWindowSpec = we.windowSpec.frameSpecification match { case swf: SpecifiedWindowFrame if needPreComputeRangeFrame(swf) => - // This is guaranteed by Spark, but we still check it here - if (orderSpecs.size != 1) { - throw new GlutenException( - s"A range window frame with value boundaries expects one and only one " + - s"order by expression: ${orderSpecs.mkString(",")}") - } val orderSpec = orderSpecs.head val lowerFrameCol = preComputeRangeFrameBoundary(swf.lower, orderSpec, expressionMap) val upperFrameCol = preComputeRangeFrameBoundary(swf.upper, orderSpec, expressionMap) From 61bc50626194da294c998e366b80c7c367dd3baf Mon Sep 17 00:00:00 2001 From: LiuNeng <1398775315@qq.com> Date: Thu, 1 Aug 2024 10:30:31 +0800 Subject: [PATCH 39/61] [CH] Fix some test cases too slow (#6659) fix ut slow , optimize lock in queryContextManager Co-authored-by: liuneng1994 --- ...ickHouseMergeTreePathBasedWriteSuite.scala | 28 +++---- .../GlutenClickHouseMergeTreeWriteSuite.scala | 75 +++++++++---------- cpp-ch/local-engine/Common/ConcurrentMap.h | 18 +++-- cpp-ch/local-engine/Common/QueryContext.cpp | 17 ++--- 4 files changed, 69 insertions(+), 69 deletions(-) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreePathBasedWriteSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreePathBasedWriteSuite.scala index ed6953b81a32d..34ffecb45833e 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreePathBasedWriteSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreePathBasedWriteSuite.scala @@ -749,8 +749,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite } } - // FIXME: very slow after https://github.com/apache/incubator-gluten/pull/6558 - ignore("test mergetree path based write with bucket table") { + test("test mergetree path based write with bucket table") { val dataPath = s"$basePath/lineitem_mergetree_bucket" clearDataPath(dataPath) @@ -760,8 +759,8 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite sourceDF.write .format("clickhouse") - .partitionBy("l_shipdate") - .option("clickhouse.orderByKey", "l_orderkey,l_returnflag") + .partitionBy("l_returnflag") + .option("clickhouse.orderByKey", "l_orderkey") .option("clickhouse.primaryKey", "l_orderkey") .option("clickhouse.numBuckets", "4") .option("clickhouse.bucketColumnNames", "l_partkey") @@ -808,13 +807,13 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite val buckets = ClickHouseTableV2.getTable(fileIndex.deltaLog).bucketOption assert(buckets.isDefined) assertResult(4)(buckets.get.numBuckets) - assertResult("l_orderkey,l_returnflag")( + assertResult("l_orderkey")( buckets.get.sortColumnNames .mkString(",")) assertResult("l_partkey")( buckets.get.bucketColumnNames .mkString(",")) - assertResult("l_orderkey,l_returnflag")( + assertResult("l_orderkey")( ClickHouseTableV2 .getTable(fileIndex.deltaLog) .orderByKeyOption @@ -827,20 +826,21 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite .get .mkString(",")) assertResult(1)(ClickHouseTableV2.getTable(fileIndex.deltaLog).partitionColumns.size) - assertResult("l_shipdate")( + assertResult("l_returnflag")( ClickHouseTableV2 .getTable(fileIndex.deltaLog) .partitionColumns .head) val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddMergeTreeParts]) - assertResult(10089)(addFiles.size) + assertResult(12)(addFiles.size) assertResult(600572)(addFiles.map(_.rows).sum) - assertResult(4)(addFiles.count(_.partitionValues("l_shipdate").equals("1992-06-01"))) - assertResult(4)(addFiles.count(_.partitionValues("l_shipdate").equals("1993-01-01"))) - assertResult(4)(addFiles.count(_.partitionValues("l_shipdate").equals("1995-01-21"))) - assertResult(1)(addFiles.count( - f => f.partitionValues("l_shipdate").equals("1995-01-21") && f.bucketNum.equals("00000"))) + assertResult(4)(addFiles.count(_.partitionValues("l_returnflag").equals("A"))) + assertResult(4)(addFiles.count(_.partitionValues("l_returnflag").equals("N"))) + assertResult(4)(addFiles.count(_.partitionValues("l_returnflag").equals("R"))) + assertResult(1)( + addFiles.count( + f => f.partitionValues("l_returnflag").equals("A") && f.bucketNum.equals("00000"))) } // check part pruning effect of filter on bucket column val df = spark.sql(s""" @@ -855,7 +855,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite .flatMap(partition => partition.asInstanceOf[GlutenMergeTreePartition].partList) .map(_.name) .distinct - assertResult(4)(touchedParts.size) + assertResult(12)(touchedParts.size) // test upsert on partitioned & bucketed table upsertSourceTableAndCheck(dataPath) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteSuite.scala index 84218f26a07f5..3b7606daac6b5 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeWriteSuite.scala @@ -801,39 +801,37 @@ class GlutenClickHouseMergeTreeWriteSuite } } - // FIXME: very slow after https://github.com/apache/incubator-gluten/pull/6558 - ignore("test mergetree write with bucket table") { + test("test mergetree write with bucket table") { spark.sql(s""" |DROP TABLE IF EXISTS lineitem_mergetree_bucket; |""".stripMargin) - spark.sql( - s""" - |CREATE TABLE IF NOT EXISTS lineitem_mergetree_bucket - |( - | l_orderkey bigint, - | l_partkey bigint, - | l_suppkey bigint, - | l_linenumber bigint, - | l_quantity double, - | l_extendedprice double, - | l_discount double, - | l_tax double, - | l_returnflag string, - | l_linestatus string, - | l_shipdate date, - | l_commitdate date, - | l_receiptdate date, - | l_shipinstruct string, - | l_shipmode string, - | l_comment string - |) - |USING clickhouse - |PARTITIONED BY (l_shipdate) - |CLUSTERED BY (l_partkey) - |${if (sparkVersion.equals("3.2")) "" else "SORTED BY (l_orderkey, l_returnflag)"} INTO 4 BUCKETS - |LOCATION '$basePath/lineitem_mergetree_bucket' - |""".stripMargin) + spark.sql(s""" + |CREATE TABLE IF NOT EXISTS lineitem_mergetree_bucket + |( + | l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity double, + | l_extendedprice double, + | l_discount double, + | l_tax double, + | l_returnflag string, + | l_linestatus string, + | l_shipdate date, + | l_commitdate date, + | l_receiptdate date, + | l_shipinstruct string, + | l_shipmode string, + | l_comment string + |) + |USING clickhouse + |PARTITIONED BY (l_returnflag) + |CLUSTERED BY (l_partkey) + |${if (sparkVersion.equals("3.2")) "" else "SORTED BY (l_orderkey)"} INTO 4 BUCKETS + |LOCATION '$basePath/lineitem_mergetree_bucket' + |""".stripMargin) spark.sql(s""" | insert into table lineitem_mergetree_bucket @@ -881,7 +879,7 @@ class GlutenClickHouseMergeTreeWriteSuite if (sparkVersion.equals("3.2")) { assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).orderByKeyOption.isEmpty) } else { - assertResult("l_orderkey,l_returnflag")( + assertResult("l_orderkey")( ClickHouseTableV2 .getTable(fileIndex.deltaLog) .orderByKeyOption @@ -890,20 +888,21 @@ class GlutenClickHouseMergeTreeWriteSuite } assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).primaryKeyOption.isEmpty) assertResult(1)(ClickHouseTableV2.getTable(fileIndex.deltaLog).partitionColumns.size) - assertResult("l_shipdate")( + assertResult("l_returnflag")( ClickHouseTableV2 .getTable(fileIndex.deltaLog) .partitionColumns .head) val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddMergeTreeParts]) - assertResult(10089)(addFiles.size) + assertResult(12)(addFiles.size) assertResult(600572)(addFiles.map(_.rows).sum) - assertResult(4)(addFiles.count(_.partitionValues("l_shipdate").equals("1992-06-01"))) - assertResult(4)(addFiles.count(_.partitionValues("l_shipdate").equals("1993-01-01"))) - assertResult(4)(addFiles.count(_.partitionValues("l_shipdate").equals("1995-01-21"))) - assertResult(1)(addFiles.count( - f => f.partitionValues("l_shipdate").equals("1995-01-21") && f.bucketNum.equals("00000"))) + assertResult(4)(addFiles.count(_.partitionValues("l_returnflag").equals("A"))) + assertResult(4)(addFiles.count(_.partitionValues("l_returnflag").equals("N"))) + assertResult(4)(addFiles.count(_.partitionValues("l_returnflag").equals("R"))) + assertResult(1)( + addFiles.count( + f => f.partitionValues("l_returnflag").equals("A") && f.bucketNum.equals("00000"))) } // check part pruning effect of filter on bucket column val df = spark.sql(s""" @@ -918,7 +917,7 @@ class GlutenClickHouseMergeTreeWriteSuite .flatMap(partition => partition.asInstanceOf[GlutenMergeTreePartition].partList) .map(_.name) .distinct - assertResult(4)(touchedParts.size) + assertResult(12)(touchedParts.size) // test upsert on partitioned & bucketed table upsertSourceTableAndCheck("lineitem_mergetree_bucket") diff --git a/cpp-ch/local-engine/Common/ConcurrentMap.h b/cpp-ch/local-engine/Common/ConcurrentMap.h index c56926ff505ce..1719d9b255eaa 100644 --- a/cpp-ch/local-engine/Common/ConcurrentMap.h +++ b/cpp-ch/local-engine/Common/ConcurrentMap.h @@ -27,13 +27,13 @@ class ConcurrentMap public: void insert(const K & key, const V & value) { - std::lock_guard lock{mutex}; + std::unique_lock lock{mutex}; map.insert({key, value}); } V get(const K & key) { - std::lock_guard lock{mutex}; + std::shared_lock lock{mutex}; auto it = map.find(key); if (it == map.end()) { @@ -44,24 +44,30 @@ class ConcurrentMap void erase(const K & key) { - std::lock_guard lock{mutex}; + std::unique_lock lock{mutex}; map.erase(key); } void clear() { - std::lock_guard lock{mutex}; + std::unique_lock lock{mutex}; map.clear(); } + bool contains(const K & key) + { + std::shared_lock lock{mutex}; + return map.contains(key); + } + size_t size() const { - std::lock_guard lock{mutex}; + std::shared_lock lock{mutex}; return map.size(); } private: std::unordered_map map; - mutable std::mutex mutex; + mutable std::shared_mutex mutex; }; } diff --git a/cpp-ch/local-engine/Common/QueryContext.cpp b/cpp-ch/local-engine/Common/QueryContext.cpp index 68934adad3671..2d5780a6e35b2 100644 --- a/cpp-ch/local-engine/Common/QueryContext.cpp +++ b/cpp-ch/local-engine/Common/QueryContext.cpp @@ -24,6 +24,7 @@ #include #include #include +#include #include #include #include @@ -48,8 +49,7 @@ struct QueryContext ContextMutablePtr query_context; }; -std::unordered_map> query_map; -std::mutex query_map_mutex; +ConcurrentMap> query_map; int64_t QueryContextManager::initializeQuery() { @@ -72,9 +72,8 @@ int64_t QueryContextManager::initializeQuery() query_context->thread_group->memory_tracker.setSoftLimit(memory_limit); query_context->thread_group->memory_tracker.setHardLimit(memory_limit + config.extra_memory_hard_limit); - std::lock_guard lock_guard(query_map_mutex); int64_t id = reinterpret_cast(query_context->thread_group.get()); - query_map.emplace(id, query_context); + query_map.insert(id, query_context); return id; } @@ -84,9 +83,8 @@ DB::ContextMutablePtr QueryContextManager::currentQueryContext() { throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Thread group not found."); } - std::lock_guard lock_guard(query_map_mutex); int64_t id = reinterpret_cast(CurrentThread::getGroup().get()); - return query_map[id]->query_context; + return query_map.get(id)->query_context; } void QueryContextManager::logCurrentPerformanceCounters(ProfileEvents::Counters & counters) @@ -116,10 +114,9 @@ void QueryContextManager::logCurrentPerformanceCounters(ProfileEvents::Counters size_t QueryContextManager::currentPeakMemory(int64_t id) { - std::lock_guard lock_guard(query_map_mutex); if (!query_map.contains(id)) throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "context released {}", id); - return query_map[id]->thread_group->memory_tracker.getPeak(); + return query_map.get(id)->thread_group->memory_tracker.getPeak(); } void QueryContextManager::finalizeQuery(int64_t id) @@ -130,8 +127,7 @@ void QueryContextManager::finalizeQuery(int64_t id) } std::shared_ptr context; { - std::lock_guard lock_guard(query_map_mutex); - context = query_map[id]; + context = query_map.get(id); } auto query_context = context->thread_status->getQueryContext(); if (!query_context) @@ -152,7 +148,6 @@ void QueryContextManager::finalizeQuery(int64_t id) context->thread_status.reset(); query_context.reset(); { - std::lock_guard lock_guard(query_map_mutex); query_map.erase(id); } } From 21618c9e40be6c7240ae17c737ef8eed2be1dcec Mon Sep 17 00:00:00 2001 From: exmy Date: Thu, 1 Aug 2024 11:24:49 +0800 Subject: [PATCH 40/61] [GLUTEN-6656][CELEBORN] Fix CelebornColumnarShuffleWriter assertion failed (#6657) --- .../shuffle/CHCelebornColumnarShuffleWriter.scala | 7 ++++++- .../spark/shuffle/CelebornColumnarShuffleWriter.scala | 10 +++++++--- .../shuffle/VeloxCelebornColumnarShuffleWriter.scala | 7 ++++++- 3 files changed, 19 insertions(+), 5 deletions(-) diff --git a/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornColumnarShuffleWriter.scala b/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornColumnarShuffleWriter.scala index 5279426c14bc7..ae22a08908193 100644 --- a/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornColumnarShuffleWriter.scala +++ b/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornColumnarShuffleWriter.scala @@ -71,7 +71,12 @@ class CHCelebornColumnarShuffleWriter[K, V]( } } - assert(nativeShuffleWriter != -1L) + // If all of the ColumnarBatch have empty rows, the nativeShuffleWriter still equals -1 + if (nativeShuffleWriter == -1L) { + handleEmptyIterator() + return + } + splitResult = jniWrapper.stop(nativeShuffleWriter) dep.metrics("splitTime").add(splitResult.getSplitTime) diff --git a/gluten-celeborn/common/src/main/scala/org/apache/spark/shuffle/CelebornColumnarShuffleWriter.scala b/gluten-celeborn/common/src/main/scala/org/apache/spark/shuffle/CelebornColumnarShuffleWriter.scala index 8082934c23ba7..f71fadd4cd64e 100644 --- a/gluten-celeborn/common/src/main/scala/org/apache/spark/shuffle/CelebornColumnarShuffleWriter.scala +++ b/gluten-celeborn/common/src/main/scala/org/apache/spark/shuffle/CelebornColumnarShuffleWriter.scala @@ -111,9 +111,7 @@ abstract class CelebornColumnarShuffleWriter[K, V]( @throws[IOException] final override def write(records: Iterator[Product2[K, V]]): Unit = { if (!records.hasNext) { - partitionLengths = new Array[Long](dep.partitioner.numPartitions) - client.mapperEnd(shuffleId, mapId, context.attemptNumber, numMappers) - mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths, mapId) + handleEmptyIterator() return } internalWrite(records) @@ -161,4 +159,10 @@ abstract class CelebornColumnarShuffleWriter[K, V]( client.mapperEnd(shuffleId, mapId, context.attemptNumber, numMappers) writeMetrics.incWriteTime(System.nanoTime - pushMergedDataTime) } + + def handleEmptyIterator(): Unit = { + partitionLengths = new Array[Long](dep.partitioner.numPartitions) + client.mapperEnd(shuffleId, mapId, context.attemptNumber, numMappers) + mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths, mapId) + } } diff --git a/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornColumnarShuffleWriter.scala b/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornColumnarShuffleWriter.scala index 8f613c72835a7..c14d46a52c125 100644 --- a/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornColumnarShuffleWriter.scala +++ b/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornColumnarShuffleWriter.scala @@ -82,7 +82,12 @@ class VeloxCelebornColumnarShuffleWriter[K, V]( } } - assert(nativeShuffleWriter != -1L) + // If all of the ColumnarBatch have empty rows, the nativeShuffleWriter still equals -1 + if (nativeShuffleWriter == -1L) { + handleEmptyIterator() + return + } + val startTime = System.nanoTime() splitResult = jniWrapper.stop(nativeShuffleWriter) From b8a7d80df35183df97761ae5f499236a0c099f09 Mon Sep 17 00:00:00 2001 From: Kyligence Git Date: Wed, 31 Jul 2024 22:50:37 -0500 Subject: [PATCH 41/61] [GLUTEN-1632][CH]Daily Update Clickhouse Version (20240801) (#6665) * [GLUTEN-1632][CH]Daily Update Clickhouse Version (20240801) * Fix build due to https://github.com/ClickHouse/ClickHouse/pull/66528 > It's being misused and leading to unnecessary copies. We misused using getSettings() in all place except CHUtils.cpp --------- Co-authored-by: kyligence-git Co-authored-by: Chang Chen --- cpp-ch/clickhouse.version | 4 ++-- cpp-ch/local-engine/Common/CHUtil.cpp | 2 +- cpp-ch/local-engine/Parser/CrossRelParser.cpp | 2 +- cpp-ch/local-engine/Parser/JoinRelParser.cpp | 2 +- cpp-ch/local-engine/Parser/MergeTreeRelParser.cpp | 2 +- cpp-ch/local-engine/Shuffle/PartitionWriter.h | 2 +- .../Storages/Mergetree/MetaDataHelper.cpp | 2 +- .../SubstraitSource/ExcelTextFormatFile.cpp | 14 +++++++------- .../Storages/SubstraitSource/FormatFile.cpp | 4 ++-- .../local-engine/tests/benchmark_local_engine.cpp | 2 +- cpp-ch/local-engine/tests/gtest_ch_join.cpp | 2 +- 11 files changed, 19 insertions(+), 19 deletions(-) diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version index a69c80926c466..dd638c97424a1 100644 --- a/cpp-ch/clickhouse.version +++ b/cpp-ch/clickhouse.version @@ -1,4 +1,4 @@ CH_ORG=Kyligence -CH_BRANCH=rebase_ch/20240730 -CH_COMMIT=f69def8b6a8 +CH_BRANCH=rebase_ch/20240801 +CH_COMMIT=768e193bd4d diff --git a/cpp-ch/local-engine/Common/CHUtil.cpp b/cpp-ch/local-engine/Common/CHUtil.cpp index e642a67a5ff3d..88867e290abb0 100644 --- a/cpp-ch/local-engine/Common/CHUtil.cpp +++ b/cpp-ch/local-engine/Common/CHUtil.cpp @@ -1013,7 +1013,7 @@ void BackendInitializerUtil::updateConfig(const DB::ContextMutablePtr & context, // configs cannot be updated per query // settings can be updated per query - auto settings = context->getSettings(); // make a copy + auto settings = context->getSettingsCopy(); // make a copy initSettings(backend_conf_map, settings); updateNewSettings(context, settings); } diff --git a/cpp-ch/local-engine/Parser/CrossRelParser.cpp b/cpp-ch/local-engine/Parser/CrossRelParser.cpp index debfc2a1eac86..454f0387f0943 100644 --- a/cpp-ch/local-engine/Parser/CrossRelParser.cpp +++ b/cpp-ch/local-engine/Parser/CrossRelParser.cpp @@ -56,7 +56,7 @@ std::shared_ptr createCrossTableJoin(substrait::CrossRel_JoinType { auto & global_context = SerializedPlanParser::global_context; auto table_join = std::make_shared( - global_context->getSettings(), global_context->getGlobalTemporaryVolume(), global_context->getTempDataOnDisk()); + global_context->getSettingsRef(), global_context->getGlobalTemporaryVolume(), global_context->getTempDataOnDisk()); std::pair kind_and_strictness = JoinUtil::getCrossJoinKindAndStrictness(join_type); table_join->setKind(kind_and_strictness.first); diff --git a/cpp-ch/local-engine/Parser/JoinRelParser.cpp b/cpp-ch/local-engine/Parser/JoinRelParser.cpp index 8d0891bd03850..8fbaf2feca717 100644 --- a/cpp-ch/local-engine/Parser/JoinRelParser.cpp +++ b/cpp-ch/local-engine/Parser/JoinRelParser.cpp @@ -55,7 +55,7 @@ std::shared_ptr createDefaultTableJoin(substrait::JoinRel_JoinTyp { auto & global_context = SerializedPlanParser::global_context; auto table_join = std::make_shared( - global_context->getSettings(), global_context->getGlobalTemporaryVolume(), global_context->getTempDataOnDisk()); + global_context->getSettingsRef(), global_context->getGlobalTemporaryVolume(), global_context->getTempDataOnDisk()); std::pair kind_and_strictness = JoinUtil::getJoinKindAndStrictness(join_type, is_existence_join); table_join->setKind(kind_and_strictness.first); diff --git a/cpp-ch/local-engine/Parser/MergeTreeRelParser.cpp b/cpp-ch/local-engine/Parser/MergeTreeRelParser.cpp index 15e44b7ee591d..967f8ba70c9ac 100644 --- a/cpp-ch/local-engine/Parser/MergeTreeRelParser.cpp +++ b/cpp-ch/local-engine/Parser/MergeTreeRelParser.cpp @@ -185,7 +185,7 @@ DB::QueryPlanPtr MergeTreeRelParser::parseReadRel( auto ranges = merge_tree_table.extractRange(selected_parts); std::string ret; - if (context->getSettings().tryGetString("enabled_driver_filter_mergetree_index", ret) && ret == "'true'") + if (context->getSettingsRef().tryGetString("enabled_driver_filter_mergetree_index", ret) && ret == "'true'") storage->analysisPartsByRanges(*reinterpret_cast(read_step.get()), ranges); else storage->wrapRangesInDataParts(*reinterpret_cast(read_step.get()), ranges); diff --git a/cpp-ch/local-engine/Shuffle/PartitionWriter.h b/cpp-ch/local-engine/Shuffle/PartitionWriter.h index 0c3c0be50f2d6..78eb845e1db12 100644 --- a/cpp-ch/local-engine/Shuffle/PartitionWriter.h +++ b/cpp-ch/local-engine/Shuffle/PartitionWriter.h @@ -140,7 +140,7 @@ class SortBasedPartitionWriter : public PartitionWriter { max_merge_block_size = options->split_size; max_sort_buffer_size = options->max_sort_buffer_size; - max_merge_block_bytes = SerializedPlanParser::global_context->getSettings().prefer_external_sort_block_bytes; + max_merge_block_bytes = SerializedPlanParser::global_context->getSettingsRef().prefer_external_sort_block_bytes; } public: String getName() const override { return "SortBasedPartitionWriter"; } diff --git a/cpp-ch/local-engine/Storages/Mergetree/MetaDataHelper.cpp b/cpp-ch/local-engine/Storages/Mergetree/MetaDataHelper.cpp index 8d43af06829ce..ebc01544fd1f6 100644 --- a/cpp-ch/local-engine/Storages/Mergetree/MetaDataHelper.cpp +++ b/cpp-ch/local-engine/Storages/Mergetree/MetaDataHelper.cpp @@ -80,7 +80,7 @@ void restoreMetaData(CustomStorageMergeTreePtr & storage, const MergeTreeTable & return; // Increase the speed of metadata recovery - auto max_concurrency = std::max(10UL, SerializedPlanParser::global_context->getSettings().max_threads.value); + auto max_concurrency = std::max(10UL, SerializedPlanParser::global_context->getSettingsRef().max_threads.value); auto max_threads = std::min(max_concurrency, not_exists_part.size()); FreeThreadPool thread_pool( CurrentMetrics::LocalThread, diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/ExcelTextFormatFile.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/ExcelTextFormatFile.cpp index 428ef57968880..3bb73a856e8a5 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/ExcelTextFormatFile.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/ExcelTextFormatFile.cpp @@ -99,14 +99,14 @@ DB::FormatSettings ExcelTextFormatFile::createFormatSettings() format_settings.csv.null_representation = file_info.text().null_value(); bool empty_as_null = true; - if (context->getSettings().has(BackendInitializerUtil::EXCEL_EMPTY_AS_NULL)) - empty_as_null = context->getSettings().getString(BackendInitializerUtil::EXCEL_EMPTY_AS_NULL) == "'true'"; + if (context->getSettingsRef().has(BackendInitializerUtil::EXCEL_EMPTY_AS_NULL)) + empty_as_null = context->getSettingsRef().getString(BackendInitializerUtil::EXCEL_EMPTY_AS_NULL) == "'true'"; format_settings.try_infer_integers = 0; - if (!context->getSettings().has(BackendInitializerUtil::EXCEL_NUMBER_FORCE)) + if (!context->getSettingsRef().has(BackendInitializerUtil::EXCEL_NUMBER_FORCE)) format_settings.try_infer_integers = 1; - if (context->getSettings().has(BackendInitializerUtil::EXCEL_NUMBER_FORCE) - && context->getSettings().getString(BackendInitializerUtil::EXCEL_NUMBER_FORCE) == "'true'") + if (context->getSettingsRef().has(BackendInitializerUtil::EXCEL_NUMBER_FORCE) + && context->getSettingsRef().getString(BackendInitializerUtil::EXCEL_NUMBER_FORCE) == "'true'") format_settings.try_infer_integers = 1; if (format_settings.csv.null_representation.empty() || empty_as_null) @@ -131,8 +131,8 @@ DB::FormatSettings ExcelTextFormatFile::createFormatSettings() { format_settings.csv.allow_single_quotes = false; - if (context->getSettings().has(BackendInitializerUtil::EXCEL_QUOTE_STRICT) - && context->getSettings().getString(BackendInitializerUtil::EXCEL_QUOTE_STRICT) == "'true'") + if (context->getSettingsRef().has(BackendInitializerUtil::EXCEL_QUOTE_STRICT) + && context->getSettingsRef().getString(BackendInitializerUtil::EXCEL_QUOTE_STRICT) == "'true'") format_settings.csv.allow_double_quotes = false; else format_settings.csv.allow_double_quotes = true; diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.cpp index 00acaf58398cc..e449ede988ee8 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.cpp @@ -95,8 +95,8 @@ FormatFilePtr FormatFileUtil::createFile( #if USE_HIVE if (file.has_text()) { - if (context->getSettings().has(BackendInitializerUtil::USE_EXCEL_PARSER) - && context->getSettings().getString(BackendInitializerUtil::USE_EXCEL_PARSER) == "'true'") + if (context->getSettingsRef().has(BackendInitializerUtil::USE_EXCEL_PARSER) + && context->getSettingsRef().getString(BackendInitializerUtil::USE_EXCEL_PARSER) == "'true'") return std::make_shared(context, file, read_buffer_builder); else return std::make_shared(context, file, read_buffer_builder); diff --git a/cpp-ch/local-engine/tests/benchmark_local_engine.cpp b/cpp-ch/local-engine/tests/benchmark_local_engine.cpp index 22b55ecf7d217..3402d5026284d 100644 --- a/cpp-ch/local-engine/tests/benchmark_local_engine.cpp +++ b/cpp-ch/local-engine/tests/benchmark_local_engine.cpp @@ -808,7 +808,7 @@ QueryPlanPtr readFromMergeTree(MergeTreeWithSnapshot storage) QueryPlanPtr joinPlan(QueryPlanPtr left, QueryPlanPtr right, String left_key, String right_key, size_t block_size = 8192) { auto join = std::make_shared( - global_context->getSettings(), global_context->getGlobalTemporaryVolume(), global_context->getTempDataOnDisk()); + global_context->getSettingsRef(), global_context->getGlobalTemporaryVolume(), global_context->getTempDataOnDisk()); auto left_columns = left->getCurrentDataStream().header.getColumnsWithTypeAndName(); auto right_columns = right->getCurrentDataStream().header.getColumnsWithTypeAndName(); join->setKind(JoinKind::Left); diff --git a/cpp-ch/local-engine/tests/gtest_ch_join.cpp b/cpp-ch/local-engine/tests/gtest_ch_join.cpp index 93b567f3b8779..4885881f9e22f 100644 --- a/cpp-ch/local-engine/tests/gtest_ch_join.cpp +++ b/cpp-ch/local-engine/tests/gtest_ch_join.cpp @@ -86,7 +86,7 @@ TEST(TestJoin, simple) right_plan.addStep(std::make_unique(Pipe(right_table))); auto join = std::make_shared( - global_context->getSettings(), global_context->getGlobalTemporaryVolume(), global_context->getTempDataOnDisk()); + global_context->getSettingsRef(), global_context->getGlobalTemporaryVolume(), global_context->getTempDataOnDisk()); join->setKind(JoinKind::Left); join->setStrictness(JoinStrictness::All); join->setColumnsFromJoinedTable(right.getNamesAndTypesList()); From d476b370e55df08576ffd3407625a86f94518b6e Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Thu, 1 Aug 2024 15:03:52 +0800 Subject: [PATCH 42/61] [CORE] Propagate SQLConf to code block of TaskResources.runUnsafe (#6658) --- .../gluten/test/VeloxBackendTestBase.java | 2 +- .../gluten/planner/plan/GlutenPlanModel.scala | 4 +-- .../org/apache/spark/util/SparkTaskUtil.scala | 5 +++ .../org/apache/spark/util/TaskResources.scala | 24 ++++++++++--- .../spark/TreeMemoryConsumerTest.java | 34 +++++++------------ .../gluten/utils/TaskResourceSuite.scala | 29 ++++++++++++++-- .../gluten/columnarbatch/ColumnarBatches.java | 1 - .../apache/gluten/sql/shims/SparkShims.scala | 4 +-- .../sql/shims/spark32/Spark32Shims.scala | 6 ++-- .../org/apache/spark/TaskContextUtils.scala | 13 ++++--- .../sql/shims/spark33/Spark33Shims.scala | 6 ++-- .../org/apache/spark/TaskContextUtils.scala | 13 ++++--- .../sql/shims/spark34/Spark34Shims.scala | 6 ++-- .../org/apache/spark/TaskContextUtils.scala | 13 ++++--- .../sql/shims/spark35/Spark35Shims.scala | 6 ++-- .../org/apache/spark/TaskContextUtils.scala | 13 ++++--- tools/gluten-it/sbin/gluten-it.sh | 1 + 17 files changed, 106 insertions(+), 74 deletions(-) diff --git a/backends-velox/src/test/java/org/apache/gluten/test/VeloxBackendTestBase.java b/backends-velox/src/test/java/org/apache/gluten/test/VeloxBackendTestBase.java index 1d7df23566dfb..2f0087608039d 100644 --- a/backends-velox/src/test/java/org/apache/gluten/test/VeloxBackendTestBase.java +++ b/backends-velox/src/test/java/org/apache/gluten/test/VeloxBackendTestBase.java @@ -53,7 +53,7 @@ public MetricRegistry metricRegistry() { @Override public SparkConf conf() { final SparkConf conf = new SparkConf(); - conf.set(GlutenConfig.GLUTEN_NUM_TASK_SLOTS_PER_EXECUTOR_KEY(), "0"); + conf.set(GlutenConfig.COLUMNAR_VELOX_CONNECTOR_IO_THREADS().key(), "0"); return conf; } diff --git a/gluten-core/src/main/scala/org/apache/gluten/planner/plan/GlutenPlanModel.scala b/gluten-core/src/main/scala/org/apache/gluten/planner/plan/GlutenPlanModel.scala index 7417d9a5d7297..727613f563f0e 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/planner/plan/GlutenPlanModel.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/planner/plan/GlutenPlanModel.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.execution.{ColumnarToRowExec, LeafExecNode, SparkPlan} import org.apache.spark.util.{SparkTaskUtil, TaskResources} -import java.util.Objects +import java.util.{Objects, Properties} object GlutenPlanModel { def apply(): PlanModel[SparkPlan] = { @@ -71,7 +71,7 @@ object GlutenPlanModel { } private object PlanModelImpl extends PlanModel[SparkPlan] { - private val fakeTc = SparkShimLoader.getSparkShims.createTestTaskContext() + private val fakeTc = SparkShimLoader.getSparkShims.createTestTaskContext(new Properties()) private def fakeTc[T](body: => T): T = { assert(!TaskResources.inSparkTask()) SparkTaskUtil.setTaskContext(fakeTc) diff --git a/gluten-core/src/main/scala/org/apache/spark/util/SparkTaskUtil.scala b/gluten-core/src/main/scala/org/apache/spark/util/SparkTaskUtil.scala index 92a12b3c6e870..21ef7bb0ac9ae 100644 --- a/gluten-core/src/main/scala/org/apache/spark/util/SparkTaskUtil.scala +++ b/gluten-core/src/main/scala/org/apache/spark/util/SparkTaskUtil.scala @@ -17,6 +17,7 @@ package org.apache.spark.util import org.apache.spark.TaskContext +import org.apache.spark.memory.TaskMemoryManager object SparkTaskUtil { def setTaskContext(taskContext: TaskContext): Unit = { @@ -26,4 +27,8 @@ object SparkTaskUtil { def unsetTaskContext(): Unit = { TaskContext.unset() } + + def getTaskMemoryManager(taskContext: TaskContext): TaskMemoryManager = { + taskContext.taskMemoryManager() + } } diff --git a/gluten-core/src/main/scala/org/apache/spark/util/TaskResources.scala b/gluten-core/src/main/scala/org/apache/spark/util/TaskResources.scala index d8079abce8589..2ab2a41a4f9b2 100644 --- a/gluten-core/src/main/scala/org/apache/spark/util/TaskResources.scala +++ b/gluten-core/src/main/scala/org/apache/spark/util/TaskResources.scala @@ -25,7 +25,7 @@ import _root_.org.apache.gluten.sql.shims.SparkShimLoader import _root_.org.apache.gluten.utils.TaskListener import java.util -import java.util.{Collections, UUID} +import java.util.{Collections, Properties, UUID} import java.util.concurrent.atomic.AtomicLong import scala.collection.JavaConverters._ @@ -40,8 +40,16 @@ object TaskResources extends TaskListener with Logging { } val ACCUMULATED_LEAK_BYTES = new AtomicLong(0L) - private def newUnsafeTaskContext(): TaskContext = { - SparkShimLoader.getSparkShims.createTestTaskContext() + private def newUnsafeTaskContext(properties: Properties): TaskContext = { + SparkShimLoader.getSparkShims.createTestTaskContext(properties) + } + + implicit private class PropertiesOps(properties: Properties) { + def setIfMissing(key: String, value: String): Unit = { + if (!properties.containsKey(key)) { + properties.setProperty(key, value) + } + } } private def setUnsafeTaskContext(): Unit = { @@ -49,7 +57,15 @@ object TaskResources extends TaskListener with Logging { throw new UnsupportedOperationException( "TaskResources#runUnsafe should only be used outside Spark task") } - TaskContext.setTaskContext(newUnsafeTaskContext()) + val properties = new Properties() + SQLConf.get.getAllConfs.foreach { + case (key, value) if key.startsWith("spark") => + properties.put(key, value) + case _ => + } + properties.setIfMissing("spark.memory.offHeap.enabled", "true") + properties.setIfMissing("spark.memory.offHeap.size", "1TB") + TaskContext.setTaskContext(newUnsafeTaskContext(properties)) } private def unsetUnsafeTaskContext(): Unit = { diff --git a/gluten-core/src/test/java/org/apache/gluten/memory/memtarget/spark/TreeMemoryConsumerTest.java b/gluten-core/src/test/java/org/apache/gluten/memory/memtarget/spark/TreeMemoryConsumerTest.java index db018ffe40431..1632e5ef45106 100644 --- a/gluten-core/src/test/java/org/apache/gluten/memory/memtarget/spark/TreeMemoryConsumerTest.java +++ b/gluten-core/src/test/java/org/apache/gluten/memory/memtarget/spark/TreeMemoryConsumerTest.java @@ -24,6 +24,7 @@ import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.util.TaskResources$; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; import java.util.Collections; @@ -31,13 +32,18 @@ import scala.Function0; public class TreeMemoryConsumerTest { - @Test - public void testIsolated() { - final SQLConf conf = new SQLConf(); + @Before + public void setUp() throws Exception { + final SQLConf conf = SQLConf.get(); + conf.setConfString("spark.memory.offHeap.enabled", "true"); + conf.setConfString("spark.memory.offHeap.size", "400"); conf.setConfString( GlutenConfig.COLUMNAR_CONSERVATIVE_TASK_OFFHEAP_SIZE_IN_BYTES().key(), "100"); + } + + @Test + public void testIsolated() { test( - conf, () -> { final TreeMemoryConsumers.Factory factory = TreeMemoryConsumers.isolated(); final TreeMemoryTarget consumer = @@ -55,11 +61,7 @@ public void testIsolated() { @Test public void testShared() { - final SQLConf conf = new SQLConf(); - conf.setConfString( - GlutenConfig.COLUMNAR_CONSERVATIVE_TASK_OFFHEAP_SIZE_IN_BYTES().key(), "100"); test( - conf, () -> { final TreeMemoryConsumers.Factory factory = TreeMemoryConsumers.shared(); final TreeMemoryTarget consumer = @@ -77,11 +79,7 @@ public void testShared() { @Test public void testIsolatedAndShared() { - final SQLConf conf = new SQLConf(); - conf.setConfString( - GlutenConfig.COLUMNAR_CONSERVATIVE_TASK_OFFHEAP_SIZE_IN_BYTES().key(), "100"); test( - conf, () -> { final TreeMemoryTarget shared = TreeMemoryConsumers.shared() @@ -102,20 +100,12 @@ public void testIsolatedAndShared() { }); } - private void test(SQLConf conf, Runnable r) { + private void test(Runnable r) { TaskResources$.MODULE$.runUnsafe( new Function0() { @Override public Object apply() { - SQLConf.withExistingConf( - conf, - new Function0() { - @Override - public Object apply() { - r.run(); - return null; - } - }); + r.run(); return null; } }); diff --git a/gluten-core/src/test/scala/org/apache/gluten/utils/TaskResourceSuite.scala b/gluten-core/src/test/scala/org/apache/gluten/utils/TaskResourceSuite.scala index 898cf873c9cfd..0c91b4faf0046 100644 --- a/gluten-core/src/test/scala/org/apache/gluten/utils/TaskResourceSuite.scala +++ b/gluten-core/src/test/scala/org/apache/gluten/utils/TaskResourceSuite.scala @@ -16,13 +16,16 @@ */ package org.apache.gluten.utils -import org.apache.spark.util.{TaskResource, TaskResources} +import org.apache.spark.memory.{MemoryConsumer, MemoryMode} +import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.util.{SparkTaskUtil, TaskResource, TaskResources} import org.scalatest.funsuite.AnyFunSuite import java.util.UUID -class TaskResourceSuite extends AnyFunSuite { +class TaskResourceSuite extends AnyFunSuite with SQLHelper { test("Run unsafe") { val out = TaskResources.runUnsafe { 1 @@ -37,6 +40,28 @@ class TaskResourceSuite extends AnyFunSuite { } } + test("Run unsafe - propagate Spark config") { + val total = 128 * 1024 * 1024 + withSQLConf( + "spark.memory.offHeap.enabled" -> "true", + "spark.memory.offHeap.size" -> s"$total", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + TaskResources.runUnsafe { + assert(TaskResources.inSparkTask()) + assert(TaskResources.getLocalTaskContext() != null) + + val tmm = SparkTaskUtil.getTaskMemoryManager(TaskResources.getLocalTaskContext()) + val consumer = new MemoryConsumer(tmm, MemoryMode.OFF_HEAP) { + override def spill(size: Long, trigger: MemoryConsumer): Long = 0L + } + assert(consumer.acquireMemory(total) == total) + assert(consumer.acquireMemory(1) == 0) + + assert(!SQLConf.get.adaptiveExecutionEnabled) + } + } + } + test("Run unsafe - register resource") { var unregisteredCount = 0 TaskResources.runUnsafe { diff --git a/gluten-data/src/main/java/org/apache/gluten/columnarbatch/ColumnarBatches.java b/gluten-data/src/main/java/org/apache/gluten/columnarbatch/ColumnarBatches.java index cb68e032dc5ba..a72eaafae4d91 100644 --- a/gluten-data/src/main/java/org/apache/gluten/columnarbatch/ColumnarBatches.java +++ b/gluten-data/src/main/java/org/apache/gluten/columnarbatch/ColumnarBatches.java @@ -100,7 +100,6 @@ private static void transferVectors(ColumnarBatch from, ColumnarBatch target) { newVectors[i] = from.column(i); } FIELD_COLUMNS.set(target, newVectors); - System.out.println(); } catch (IllegalAccessException e) { throw new GlutenException(e); } diff --git a/shims/common/src/main/scala/org/apache/gluten/sql/shims/SparkShims.scala b/shims/common/src/main/scala/org/apache/gluten/sql/shims/SparkShims.scala index fed08b789802b..5a094698852e5 100644 --- a/shims/common/src/main/scala/org/apache/gluten/sql/shims/SparkShims.scala +++ b/shims/common/src/main/scala/org/apache/gluten/sql/shims/SparkShims.scala @@ -50,7 +50,7 @@ import org.apache.spark.storage.{BlockId, BlockManagerId} import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.parquet.schema.MessageType -import java.util.{ArrayList => JArrayList, Map => JMap} +import java.util.{Map => JMap, Properties} import scala.reflect.ClassTag @@ -156,7 +156,7 @@ trait SparkShims { def enableNativeWriteFilesByDefault(): Boolean = false - def createTestTaskContext(): TaskContext + def createTestTaskContext(properties: Properties): TaskContext def broadcastInternal[T: ClassTag](sc: SparkContext, value: T): Broadcast[T] = { // Since Spark 3.4, the `sc.broadcast` has been optimized to use `sc.broadcastInternal`. diff --git a/shims/spark32/src/main/scala/org/apache/gluten/sql/shims/spark32/Spark32Shims.scala b/shims/spark32/src/main/scala/org/apache/gluten/sql/shims/spark32/Spark32Shims.scala index 5ad63884af38b..995d5b087d3c0 100644 --- a/shims/spark32/src/main/scala/org/apache/gluten/sql/shims/spark32/Spark32Shims.scala +++ b/shims/spark32/src/main/scala/org/apache/gluten/sql/shims/spark32/Spark32Shims.scala @@ -53,7 +53,7 @@ import org.apache.spark.storage.{BlockId, BlockManagerId} import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.parquet.schema.MessageType -import java.util.{HashMap => JHashMap, Map => JMap} +import java.util.{HashMap => JHashMap, Map => JMap, Properties} class Spark32Shims extends SparkShims { override def getShimDescriptor: ShimDescriptor = SparkShimProvider.DESCRIPTOR @@ -156,8 +156,8 @@ class Spark32Shims extends SparkShims { List(session => GlutenParquetWriterInjects.getInstance().getExtendedColumnarPostRule(session)) } - override def createTestTaskContext(): TaskContext = { - TaskContextUtils.createTestTaskContext() + override def createTestTaskContext(properties: Properties): TaskContext = { + TaskContextUtils.createTestTaskContext(properties) } def setJobDescriptionOrTagForBroadcastExchange( diff --git a/shims/spark32/src/main/scala/org/apache/spark/TaskContextUtils.scala b/shims/spark32/src/main/scala/org/apache/spark/TaskContextUtils.scala index c4abeb411972f..ac7f926f1f696 100644 --- a/shims/spark32/src/main/scala/org/apache/spark/TaskContextUtils.scala +++ b/shims/spark32/src/main/scala/org/apache/spark/TaskContextUtils.scala @@ -19,17 +19,16 @@ package org.apache.spark import org.apache.spark.executor.TaskMetrics import org.apache.spark.memory.{TaskMemoryManager, UnifiedMemoryManager} import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.network.util.ByteUnit import java.util.Properties +import scala.collection.JavaConverters._ + object TaskContextUtils { - def createTestTaskContext(): TaskContext = { + def createTestTaskContext(properties: Properties): TaskContext = { val conf = new SparkConf() - conf.set("spark.memory.offHeap.enabled", "true") - conf.set("spark.memory.offHeap.size", "1TB") - val memoryManager = - new UnifiedMemoryManager(conf, ByteUnit.TiB.toBytes(2), ByteUnit.TiB.toBytes(1), 1) + conf.setAll(properties.asScala) + val memoryManager = UnifiedMemoryManager(conf, 1) new TaskContextImpl( -1, -1, @@ -37,7 +36,7 @@ object TaskContextUtils { -1L, -1, new TaskMemoryManager(memoryManager, -1L), - new Properties, + properties, MetricsSystem.createMetricsSystem("GLUTEN_UNSAFE", conf), TaskMetrics.empty, Map.empty diff --git a/shims/spark33/src/main/scala/org/apache/gluten/sql/shims/spark33/Spark33Shims.scala b/shims/spark33/src/main/scala/org/apache/gluten/sql/shims/spark33/Spark33Shims.scala index 8b12c2642c55b..7b606ea9774dc 100644 --- a/shims/spark33/src/main/scala/org/apache/gluten/sql/shims/spark33/Spark33Shims.scala +++ b/shims/spark33/src/main/scala/org/apache/gluten/sql/shims/spark33/Spark33Shims.scala @@ -56,7 +56,7 @@ import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.parquet.schema.MessageType import java.time.ZoneOffset -import java.util.{HashMap => JHashMap, Map => JMap} +import java.util.{HashMap => JHashMap, Map => JMap, Properties} class Spark33Shims extends SparkShims { override def getShimDescriptor: ShimDescriptor = SparkShimProvider.DESCRIPTOR @@ -248,8 +248,8 @@ class Spark33Shims extends SparkShims { List(session => GlutenParquetWriterInjects.getInstance().getExtendedColumnarPostRule(session)) } - override def createTestTaskContext(): TaskContext = { - TaskContextUtils.createTestTaskContext() + override def createTestTaskContext(properties: Properties): TaskContext = { + TaskContextUtils.createTestTaskContext(properties) } def setJobDescriptionOrTagForBroadcastExchange( diff --git a/shims/spark33/src/main/scala/org/apache/spark/TaskContextUtils.scala b/shims/spark33/src/main/scala/org/apache/spark/TaskContextUtils.scala index 6d7e686de2af4..c4fea992d5bf7 100644 --- a/shims/spark33/src/main/scala/org/apache/spark/TaskContextUtils.scala +++ b/shims/spark33/src/main/scala/org/apache/spark/TaskContextUtils.scala @@ -19,17 +19,16 @@ package org.apache.spark import org.apache.spark.executor.TaskMetrics import org.apache.spark.memory.{TaskMemoryManager, UnifiedMemoryManager} import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.network.util.ByteUnit import java.util.Properties +import scala.collection.JavaConverters._ + object TaskContextUtils { - def createTestTaskContext(): TaskContext = { + def createTestTaskContext(properties: Properties): TaskContext = { val conf = new SparkConf() - conf.set("spark.memory.offHeap.enabled", "true") - conf.set("spark.memory.offHeap.size", "1TB") - val memoryManager = - new UnifiedMemoryManager(conf, ByteUnit.TiB.toBytes(2), ByteUnit.TiB.toBytes(1), 1) + conf.setAll(properties.asScala) + val memoryManager = UnifiedMemoryManager(conf, 1) new TaskContextImpl( -1, -1, @@ -37,7 +36,7 @@ object TaskContextUtils { -1L, -1, new TaskMemoryManager(memoryManager, -1L), - new Properties, + properties, MetricsSystem.createMetricsSystem("GLUTEN_UNSAFE", conf), TaskMetrics.empty, 1, diff --git a/shims/spark34/src/main/scala/org/apache/gluten/sql/shims/spark34/Spark34Shims.scala b/shims/spark34/src/main/scala/org/apache/gluten/sql/shims/spark34/Spark34Shims.scala index cd7e4347d6b52..8b1b7649a98a4 100644 --- a/shims/spark34/src/main/scala/org/apache/gluten/sql/shims/spark34/Spark34Shims.scala +++ b/shims/spark34/src/main/scala/org/apache/gluten/sql/shims/spark34/Spark34Shims.scala @@ -58,7 +58,7 @@ import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.parquet.schema.MessageType import java.time.ZoneOffset -import java.util.{HashMap => JHashMap, Map => JMap} +import java.util.{HashMap => JHashMap, Map => JMap, Properties} import scala.reflect.ClassTag @@ -299,8 +299,8 @@ class Spark34Shims extends SparkShims { override def enableNativeWriteFilesByDefault(): Boolean = true - override def createTestTaskContext(): TaskContext = { - TaskContextUtils.createTestTaskContext() + override def createTestTaskContext(properties: Properties): TaskContext = { + TaskContextUtils.createTestTaskContext(properties) } override def broadcastInternal[T: ClassTag](sc: SparkContext, value: T): Broadcast[T] = { diff --git a/shims/spark34/src/main/scala/org/apache/spark/TaskContextUtils.scala b/shims/spark34/src/main/scala/org/apache/spark/TaskContextUtils.scala index 976851eb96cd1..7a81b612112f9 100644 --- a/shims/spark34/src/main/scala/org/apache/spark/TaskContextUtils.scala +++ b/shims/spark34/src/main/scala/org/apache/spark/TaskContextUtils.scala @@ -19,17 +19,16 @@ package org.apache.spark import org.apache.spark.executor.TaskMetrics import org.apache.spark.memory.{TaskMemoryManager, UnifiedMemoryManager} import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.network.util.ByteUnit import java.util.Properties +import scala.collection.JavaConverters._ + object TaskContextUtils { - def createTestTaskContext(): TaskContext = { + def createTestTaskContext(properties: Properties): TaskContext = { val conf = new SparkConf() - conf.set("spark.memory.offHeap.enabled", "true") - conf.set("spark.memory.offHeap.size", "1TB") - val memoryManager = - new UnifiedMemoryManager(conf, ByteUnit.TiB.toBytes(2), ByteUnit.TiB.toBytes(1), 1) + conf.setAll(properties.asScala) + val memoryManager = UnifiedMemoryManager(conf, 1) new TaskContextImpl( -1, -1, @@ -38,7 +37,7 @@ object TaskContextUtils { -1, -1, new TaskMemoryManager(memoryManager, -1L), - new Properties, + properties, MetricsSystem.createMetricsSystem("GLUTEN_UNSAFE", conf), TaskMetrics.empty, 1, diff --git a/shims/spark35/src/main/scala/org/apache/gluten/sql/shims/spark35/Spark35Shims.scala b/shims/spark35/src/main/scala/org/apache/gluten/sql/shims/spark35/Spark35Shims.scala index bb41b7e73ecad..93785d7a26595 100644 --- a/shims/spark35/src/main/scala/org/apache/gluten/sql/shims/spark35/Spark35Shims.scala +++ b/shims/spark35/src/main/scala/org/apache/gluten/sql/shims/spark35/Spark35Shims.scala @@ -58,7 +58,7 @@ import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.parquet.schema.MessageType import java.time.ZoneOffset -import java.util.{HashMap => JHashMap, Map => JMap} +import java.util.{HashMap => JHashMap, Map => JMap, Properties} import scala.reflect.ClassTag @@ -326,8 +326,8 @@ class Spark35Shims extends SparkShims { override def enableNativeWriteFilesByDefault(): Boolean = true - override def createTestTaskContext(): TaskContext = { - TaskContextUtils.createTestTaskContext() + override def createTestTaskContext(properties: Properties): TaskContext = { + TaskContextUtils.createTestTaskContext(properties) } override def broadcastInternal[T: ClassTag](sc: SparkContext, value: T): Broadcast[T] = { diff --git a/shims/spark35/src/main/scala/org/apache/spark/TaskContextUtils.scala b/shims/spark35/src/main/scala/org/apache/spark/TaskContextUtils.scala index 976851eb96cd1..7a81b612112f9 100644 --- a/shims/spark35/src/main/scala/org/apache/spark/TaskContextUtils.scala +++ b/shims/spark35/src/main/scala/org/apache/spark/TaskContextUtils.scala @@ -19,17 +19,16 @@ package org.apache.spark import org.apache.spark.executor.TaskMetrics import org.apache.spark.memory.{TaskMemoryManager, UnifiedMemoryManager} import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.network.util.ByteUnit import java.util.Properties +import scala.collection.JavaConverters._ + object TaskContextUtils { - def createTestTaskContext(): TaskContext = { + def createTestTaskContext(properties: Properties): TaskContext = { val conf = new SparkConf() - conf.set("spark.memory.offHeap.enabled", "true") - conf.set("spark.memory.offHeap.size", "1TB") - val memoryManager = - new UnifiedMemoryManager(conf, ByteUnit.TiB.toBytes(2), ByteUnit.TiB.toBytes(1), 1) + conf.setAll(properties.asScala) + val memoryManager = UnifiedMemoryManager(conf, 1) new TaskContextImpl( -1, -1, @@ -38,7 +37,7 @@ object TaskContextUtils { -1, -1, new TaskMemoryManager(memoryManager, -1L), - new Properties, + properties, MetricsSystem.createMetricsSystem("GLUTEN_UNSAFE", conf), TaskMetrics.empty, 1, diff --git a/tools/gluten-it/sbin/gluten-it.sh b/tools/gluten-it/sbin/gluten-it.sh index b21038ccdef6a..00ff78e349977 100755 --- a/tools/gluten-it/sbin/gluten-it.sh +++ b/tools/gluten-it/sbin/gluten-it.sh @@ -48,6 +48,7 @@ $JAVA_HOME/bin/java $GLUTEN_IT_JVM_ARGS \ --add-opens=java.base/java.util.concurrent=ALL-UNNAMED \ --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED \ --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED \ + --add-opens=java.base/jdk.internal.misc=ALL-UNNAMED \ --add-opens=java.base/sun.nio.ch=ALL-UNNAMED \ --add-opens=java.base/sun.nio.cs=ALL-UNNAMED \ --add-opens=java.base/sun.security.action=ALL-UNNAMED \ From d86b0253a8e5dc34677c8397b10810d66713214b Mon Sep 17 00:00:00 2001 From: Xiduo You Date: Thu, 1 Aug 2024 16:20:29 +0800 Subject: [PATCH 43/61] [INFRA] Support automatically label new pull requests (#6668) --- .github/labeler.yml | 95 +++++++++++++++++++++++++++++++++++ .github/workflows/labeler.yml | 30 +++++++++++ 2 files changed, 125 insertions(+) create mode 100644 .github/labeler.yml create mode 100644 .github/workflows/labeler.yml diff --git a/.github/labeler.yml b/.github/labeler.yml new file mode 100644 index 0000000000000..878f9fae05ccd --- /dev/null +++ b/.github/labeler.yml @@ -0,0 +1,95 @@ +# 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. + + +INFRA: + - changed-files: + - any-glob-to-any-file: [ + '.github/**/*', + '.asf.yaml', + '.gitattributes', + 'LICENSE', + 'LICENSE-binary', + 'NOTICE', + 'NOTICE-binary' + ] + +BUILD: + - changed-files: + - any-glob-to-any-file: [ + 'dev/**/*', + 'cpp/compile.sh', + 'ep/build-velox/src/get_velox.sh', + 'ep/build-velox/src/build_velox.sh', + 'ep/build-clickhouse/src/build_clickhouse.sh' + ] + +DOCS: + - changed-files: + - any-glob-to-any-file: [ + 'docs/**/*', + '**/README.md', + '**/CONTRIBUTING.md' + ] + +CORE: + - changed-files: + - any-glob-to-any-file: [ + 'gluten-core/**/*', + 'shims/**/*', + 'gluten-ras/**/*', + 'gluten-ui/**/*', + 'gluten-ut/**/*', + 'package/**/*', + 'pom.xml' + ] + +VELOX: + - changed-files: + - any-glob-to-any-file: [ + 'gluten-data/**/*', + 'backends-velox/**/*', + 'ep/build-velox/**/*', + 'cpp/**/*' + ] + +CLICKHOUSE: + - changed-files: + - any-glob-to-any-file: [ + 'backends-clickhouse/**/*', + 'ep/build-clickhouse/**/*', + 'cpp-ch/**/*' + ] + +DATA_LAKE: + - changed-files: + - any-glob-to-any-file: [ + 'gluten-iceberg/**/*', + 'gluten-delta/**/*' + ] + +RSS: + - changed-files: + - any-glob-to-any-file: [ + 'gluten-celeborn/**/*', + 'gluten-uniffle/**/*' + ] + +TOOLS: + - changed-files: + - any-glob-to-any-file: [ + 'tools/**/*' + ] + diff --git a/.github/workflows/labeler.yml b/.github/workflows/labeler.yml new file mode 100644 index 0000000000000..d2ddaec8f8359 --- /dev/null +++ b/.github/workflows/labeler.yml @@ -0,0 +1,30 @@ +# 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. + +name: "Label pull requests" +on: pull_request_target + +jobs: + label: + name: Label pull requests + runs-on: ubuntu-latest + permissions: + contents: read + pull-requests: write + steps: + - uses: actions/labeler@v5 + with: + repo-token: "${{ secrets.GITHUB_TOKEN }}" + sync-labels: true From bedb2ee0ba4bf1bb071234368f5d095c6b912afd Mon Sep 17 00:00:00 2001 From: Nicholas Jiang Date: Thu, 1 Aug 2024 19:01:17 +0800 Subject: [PATCH 44/61] [GLUTEN-6483] Support Uniffle 0.9.0 (#6484) --- .github/workflows/velox_docker.yml | 37 ++++++++++--------- .../gluten/uniffle/UniffleShuffleManager.java | 14 +------ .../VeloxUniffleColumnarShuffleWriter.java | 35 ++++++++++-------- pom.xml | 2 +- tools/gluten-it/pom.xml | 2 +- 5 files changed, 42 insertions(+), 48 deletions(-) diff --git a/.github/workflows/velox_docker.yml b/.github/workflows/velox_docker.yml index 913a55cbd4225..ee6bf9797d41b 100644 --- a/.github/workflows/velox_docker.yml +++ b/.github/workflows/velox_docker.yml @@ -492,38 +492,39 @@ jobs: wget https://downloads.apache.org/maven/maven-3/3.8.8/binaries/apache-maven-3.8.8-bin.tar.gz tar -xvf apache-maven-3.8.8-bin.tar.gz mv apache-maven-3.8.8 /usr/lib/maven - - name: Build for Spark ${{ matrix.spark }} - run: | - cd $GITHUB_WORKSPACE/ && \ - export MAVEN_HOME=/usr/lib/maven && \ - export PATH=${PATH}:${MAVEN_HOME}/bin && \ - $MVN_CMD clean install -P${{ matrix.spark }} -Pbackends-velox -Puniffle -DskipTests - - name: TPC-H SF1.0 && TPC-DS SF1.0 Parquet local spark3.2 with uniffle 0.8.0 + - name: Build for Uniffle 0.9.0 run: | export MAVEN_HOME=/usr/lib/maven && \ export PATH=${PATH}:${MAVEN_HOME}/bin && \ export JAVA_HOME=/usr/lib/jvm/java-1.8.0-openjdk && \ cd /opt && \ - git clone -b branch-0.8 https://github.com/apache/incubator-uniffle.git && \ + git clone -b v0.9.0 https://github.com/apache/incubator-uniffle.git && \ cd incubator-uniffle && \ - sed -i '250d' ./server/src/main/java/org/apache/uniffle/server/ShuffleTaskManager.java && \ - sed -i '228d' ./server/src/main/java/org/apache/uniffle/server/ShuffleTaskManager.java && \ - sed -i '226d' ./server/src/main/java/org/apache/uniffle/server/ShuffleTaskManager.java && \ - $MVN_CMD clean install -Phadoop2.8 -DskipTests + $MVN_CMD clean install -Phadoop2.8,spark3 -DskipTests cd /opt && \ - wget -nv https://archive.apache.org/dist/incubator/uniffle/0.8.0/apache-uniffle-0.8.0-incubating-bin.tar.gz && \ - tar xzf apache-uniffle-0.8.0-incubating-bin.tar.gz -C /opt/ && mv /opt/rss-0.8.0-hadoop2.8 /opt/uniffle && \ + wget -nv https://archive.apache.org/dist/incubator/uniffle/0.9.0/apache-uniffle-0.9.0-incubating-bin.tar.gz && \ + tar xzf apache-uniffle-0.9.0-incubating-bin.tar.gz -C /opt/ && mv /opt/rss-0.9.0-hadoop2.8 /opt/uniffle && \ wget -nv https://archive.apache.org/dist/hadoop/common/hadoop-2.8.5/hadoop-2.8.5.tar.gz && \ tar xzf hadoop-2.8.5.tar.gz -C /opt/ - rm -f /opt/uniffle/jars/server/shuffle-server-0.8.0-SNAPSHOT.jar - cp /opt/incubator-uniffle/server/target/shuffle-server-0.8.1-SNAPSHOT.jar /opt/uniffle/jars/server/ rm -rf /opt/incubator-uniffle cd /opt/uniffle && mkdir shuffle_data && \ bash -c "echo -e 'XMX_SIZE=16g\nHADOOP_HOME=/opt/hadoop-2.8.5' > ./bin/rss-env.sh" && \ bash -c "echo -e 'rss.coordinator.shuffle.nodes.max 1\nrss.rpc.server.port 19999' > ./conf/coordinator.conf" && \ - bash -c "echo -e 'rss.server.app.expired.withoutHeartbeat 7200000\nrss.server.heartbeat.delay 3000\nrss.rpc.server.port 19997\nrss.jetty.http.port 19996\nrss.server.netty.port 19995\nrss.storage.basePath /opt/uniffle/shuffle_data\nrss.storage.type MEMORY_LOCALFILE\nrss.coordinator.quorum localhost:19999\nrss.server.flush.thread.alive 10\nrss.server.single.buffer.flush.threshold 64m' > ./conf/server.conf" && \ + bash -c "echo -e 'rss.server.app.expired.withoutHeartbeat 7200000\nrss.server.heartbeat.delay 3000\nrss.rpc.server.port 19997\nrss.rpc.server.type GRPC_NETTY\nrss.jetty.http.port 19996\nrss.server.netty.port 19995\nrss.storage.basePath /opt/uniffle/shuffle_data\nrss.storage.type MEMORY_LOCALFILE\nrss.coordinator.quorum localhost:19999\nrss.server.flush.thread.alive 10\nrss.server.single.buffer.flush.threshold 64m' > ./conf/server.conf" && \ bash ./bin/start-coordinator.sh && bash ./bin/start-shuffle-server.sh - cd $GITHUB_WORKSPACE/tools/gluten-it && $MVN_CMD clean install -Pspark-3.2 -Puniffle && \ + - name: Build for Spark ${{ matrix.spark }} + run: | + export MAVEN_HOME=/usr/lib/maven && \ + export PATH=${PATH}:${MAVEN_HOME}/bin && \ + cd $GITHUB_WORKSPACE/ && \ + $MVN_CMD clean install -P${{ matrix.spark }} -Pbackends-velox -Puniffle -DskipTests + - name: TPC-H SF1.0 && TPC-DS SF1.0 Parquet local spark3.2 with uniffle-0.9.0 + run: | + export MAVEN_HOME=/usr/lib/maven && \ + export PATH=${PATH}:${MAVEN_HOME}/bin && \ + export JAVA_HOME=/usr/lib/jvm/java-1.8.0-openjdk && \ + cd $GITHUB_WORKSPACE/tools/gluten-it && \ + $MVN_CMD clean install -Pspark-3.2 -Puniffle && \ GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ --local --preset=velox-with-uniffle --benchmark-type=h --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 diff --git a/gluten-uniffle/velox/src/main/java/org/apache/spark/shuffle/gluten/uniffle/UniffleShuffleManager.java b/gluten-uniffle/velox/src/main/java/org/apache/spark/shuffle/gluten/uniffle/UniffleShuffleManager.java index f91141c1eb843..eb66dae907826 100644 --- a/gluten-uniffle/velox/src/main/java/org/apache/spark/shuffle/gluten/uniffle/UniffleShuffleManager.java +++ b/gluten-uniffle/velox/src/main/java/org/apache/spark/shuffle/gluten/uniffle/UniffleShuffleManager.java @@ -16,9 +16,7 @@ */ package org.apache.spark.shuffle.gluten.uniffle; -import org.apache.spark.ShuffleDependency; import org.apache.spark.SparkConf; -import org.apache.spark.SparkEnv; import org.apache.spark.TaskContext; import org.apache.spark.executor.ShuffleWriteMetrics; import org.apache.spark.shuffle.ColumnarShuffleDependency; @@ -36,21 +34,11 @@ public class UniffleShuffleManager extends RssShuffleManager { private static final Logger LOG = LoggerFactory.getLogger(UniffleShuffleManager.class); - private boolean isDriver() { - return "driver".equals(SparkEnv.get().executorId()); - } - public UniffleShuffleManager(SparkConf conf, boolean isDriver) { super(conf, isDriver); conf.set(RssSparkConfig.SPARK_RSS_CONFIG_PREFIX + RssSparkConfig.RSS_ROW_BASED.key(), "false"); } - @Override - public ShuffleHandle registerShuffle( - int shuffleId, ShuffleDependency dependency) { - return super.registerShuffle(shuffleId, dependency); - } - @Override public ShuffleWriter getWriter( ShuffleHandle handle, long mapId, TaskContext context, ShuffleWriteMetricsReporter metrics) { @@ -62,7 +50,7 @@ public ShuffleWriter getWriter( ColumnarShuffleDependency dependency = (ColumnarShuffleDependency) rssHandle.getDependency(); setPusherAppId(rssHandle); - String taskId = "" + context.taskAttemptId() + "_" + context.attemptNumber(); + String taskId = context.taskAttemptId() + "_" + context.attemptNumber(); ShuffleWriteMetrics writeMetrics; if (metrics != null) { writeMetrics = new WriteMetrics(metrics); diff --git a/gluten-uniffle/velox/src/main/java/org/apache/spark/shuffle/writer/VeloxUniffleColumnarShuffleWriter.java b/gluten-uniffle/velox/src/main/java/org/apache/spark/shuffle/writer/VeloxUniffleColumnarShuffleWriter.java index b84c9d4ee6012..a80e34fb1d995 100644 --- a/gluten-uniffle/velox/src/main/java/org/apache/spark/shuffle/writer/VeloxUniffleColumnarShuffleWriter.java +++ b/gluten-uniffle/velox/src/main/java/org/apache/spark/shuffle/writer/VeloxUniffleColumnarShuffleWriter.java @@ -41,6 +41,7 @@ import org.apache.spark.util.SparkResourceUtil; import org.apache.uniffle.client.api.ShuffleWriteClient; import org.apache.uniffle.common.ShuffleBlockInfo; +import org.apache.uniffle.common.exception.RssException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -61,19 +62,18 @@ public class VeloxUniffleColumnarShuffleWriter extends RssShuffleWriter columnarDep; private final SparkConf sparkConf; @@ -125,13 +125,13 @@ public VeloxUniffleColumnarShuffleWriter( } @Override - protected void writeImpl(Iterator> records) throws IOException { + protected void writeImpl(Iterator> records) { if (!records.hasNext() && !isMemoryShuffleEnabled) { super.sendCommit(); return; } // writer already init - partitionPusher = new PartitionPusher(this); + PartitionPusher partitionPusher = new PartitionPusher(this); while (records.hasNext()) { ColumnarBatch cb = (ColumnarBatch) (records.next()._2()); if (cb.numRows() == 0 || cb.numCols() == 0) { @@ -194,7 +194,12 @@ public long spill(MemoryTarget self, Spiller.Phase phase, long size) { if (nativeShuffleWriter == -1L) { throw new IllegalStateException("nativeShuffleWriter should not be -1L"); } - splitResult = jniWrapper.stop(nativeShuffleWriter); + SplitResult splitResult; + try { + splitResult = jniWrapper.stop(nativeShuffleWriter); + } catch (IOException e) { + throw new RssException(e); + } columnarDep.metrics().get("shuffleWallTime").get().add(System.nanoTime() - startTime); columnarDep .metrics() @@ -220,7 +225,7 @@ public long spill(MemoryTarget self, Spiller.Phase phase, long size) { long writeDurationMs = System.nanoTime() - pushMergedDataTime; shuffleWriteMetrics.incWriteTime(writeDurationMs); LOG.info( - "Finish write shuffle with rest write {} ms", + "Finish write shuffle with rest write {} ms", TimeUnit.MILLISECONDS.toNanos(writeDurationMs)); } diff --git a/pom.xml b/pom.xml index bb59ad2a7faa0..cbec5befba87c 100644 --- a/pom.xml +++ b/pom.xml @@ -66,7 +66,7 @@ 2.4.0 24 0.5.1 - 0.8.0 + 0.9.0 15.0.0 15.0.0-gluten arrow-memory-unsafe diff --git a/tools/gluten-it/pom.xml b/tools/gluten-it/pom.xml index 43bc3ae092b5d..5ab633252f305 100644 --- a/tools/gluten-it/pom.xml +++ b/tools/gluten-it/pom.xml @@ -22,7 +22,7 @@ 2.12 3 0.3.2-incubating - 0.8.0 + 0.9.0 1.2.0-SNAPSHOT 32.0.1-jre 1.1 From e472e5dac31343caf167f74c9e95f89c146fd049 Mon Sep 17 00:00:00 2001 From: LiuNeng <1398775315@qq.com> Date: Thu, 1 Aug 2024 20:51:45 +0800 Subject: [PATCH 45/61] [CH] Hotfix a configuration bug in shuffle writer liuneng1994 commented 2 hours ago What changes were proposed in this pull request? Hotfix a configuration bug in shuffle writer How was this patch tested? unit tests (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) --- cpp-ch/local-engine/Shuffle/PartitionWriter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp b/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp index a2ef0888aeff5..58be564213cb0 100644 --- a/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp +++ b/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp @@ -324,7 +324,7 @@ PartitionWriter::PartitionWriter(CachedShuffleWriter * shuffle_writer_, LoggerPt partition_block_buffer[partition_id] = std::make_shared(options->split_size); partition_buffer[partition_id] = std::make_shared(); } - settings.loadFromContext(SerializedPlanParser::global_context); + settings = MemoryConfig::loadFromContext(SerializedPlanParser::global_context); } size_t PartitionWriter::bytes() const From acb50c472ab340e1d3c01a3b94ba720697c0c64b Mon Sep 17 00:00:00 2001 From: Yan Ma Date: Fri, 2 Aug 2024 01:35:24 +0800 Subject: [PATCH 46/61] Set default validation log level to WARN (#6676) Set default validation log level from INFO to WARN --- .../common/src/main/scala/org/apache/gluten/GlutenConfig.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index 69d2e21e67e54..c05069c38a08f 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -1504,7 +1504,7 @@ object GlutenConfig { .checkValue( logLevel => Set("TRACE", "DEBUG", "INFO", "WARN", "ERROR").contains(logLevel), "Valid values are 'trace', 'debug', 'info', 'warn' and 'error'.") - .createWithDefault("INFO") + .createWithDefault("WARN") val VALIDATION_PRINT_FAILURE_STACK_ = buildConf("spark.gluten.sql.validation.printStackOnFailure") From 8253f131256a6230cdb151ef496b354b886a1a85 Mon Sep 17 00:00:00 2001 From: Nicholas Jiang Date: Fri, 2 Aug 2024 08:57:57 +0800 Subject: [PATCH 47/61] [GLUTEN-6483][VL][DOC] Upgrade Uniffle version to 0.9.0 in Velox.md (#6680) --- docs/get-started/Velox.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/get-started/Velox.md b/docs/get-started/Velox.md index 6778bb9b72070..63239c39931ef 100644 --- a/docs/get-started/Velox.md +++ b/docs/get-started/Velox.md @@ -266,7 +266,7 @@ spark.dynamicAllocation.enabled false ## Uniffle support -Uniffle with velox backend supports [Uniffle](https://github.com/apache/incubator-uniffle) as remote shuffle service. Currently, the supported Uniffle versions are `0.8.0`. +Uniffle with velox backend supports [Uniffle](https://github.com/apache/incubator-uniffle) as remote shuffle service. Currently, the supported Uniffle versions are `0.9.0`. First refer to this URL(https://uniffle.apache.org/docs/intro) to get start with uniffle. From 413473bb4727bc0b99d20b61f4bdc0fa2ea66fd2 Mon Sep 17 00:00:00 2001 From: Zhen Wang <643348094@qq.com> Date: Fri, 2 Aug 2024 09:05:05 +0800 Subject: [PATCH 48/61] [CORE] Make collectQueryExecutionFallbackSummary as a public util method (#6679) --- .../spark/sql/execution/GlutenImplicits.scala | 231 +++++++++--------- 1 file changed, 117 insertions(+), 114 deletions(-) diff --git a/gluten-core/src/main/scala/org/apache/spark/sql/execution/GlutenImplicits.scala b/gluten-core/src/main/scala/org/apache/spark/sql/execution/GlutenImplicits.scala index eb42f0a884602..2e2af6517d9c2 100644 --- a/gluten-core/src/main/scala/org/apache/spark/sql/execution/GlutenImplicits.scala +++ b/gluten-core/src/main/scala/org/apache/spark/sql/execution/GlutenImplicits.scala @@ -20,7 +20,7 @@ import org.apache.gluten.execution.WholeStageTransformer import org.apache.gluten.extension.GlutenPlan import org.apache.gluten.utils.PlanUtil -import org.apache.spark.sql.{AnalysisException, Dataset} +import org.apache.spark.sql.{AnalysisException, Dataset, SparkSession} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{CommandResult, LogicalPlan} import org.apache.spark.sql.catalyst.util.StringUtils.PlanStringConcat @@ -87,131 +87,134 @@ object GlutenImplicits { } } - implicit class DatasetTransformer[T](dateset: Dataset[T]) { - private def isFinalAdaptivePlan(p: AdaptiveSparkPlanExec): Boolean = { - val args = p.argString(Int.MaxValue) - val index = args.indexOf("isFinalPlan=") - assert(index >= 0) - args.substring(index + "isFinalPlan=".length).trim.toBoolean - } + private def isFinalAdaptivePlan(p: AdaptiveSparkPlanExec): Boolean = { + val args = p.argString(Int.MaxValue) + val index = args.indexOf("isFinalPlan=") + assert(index >= 0) + args.substring(index + "isFinalPlan=".length).trim.toBoolean + } - private def collectFallbackNodes(plan: QueryPlan[_]): FallbackInfo = { - var numGlutenNodes = 0 - val fallbackNodeToReason = new mutable.HashMap[String, String] - - def collect(tmp: QueryPlan[_]): Unit = { - tmp.foreachUp { - case _: ExecutedCommandExec => - case _: CommandResultExec => - case _: V2CommandExec => - case _: DataWritingCommandExec => - case _: WholeStageCodegenExec => - case _: WholeStageTransformer => - case _: InputAdapter => - case _: ColumnarInputAdapter => - case _: InputIteratorTransformer => - case _: ColumnarToRowTransition => - case _: RowToColumnarTransition => - case p: ReusedExchangeExec => - case p: AdaptiveSparkPlanExec if isFinalAdaptivePlan(p) => - collect(p.executedPlan) - case p: AdaptiveSparkPlanExec => - // if we are here that means we are inside table cache. - val (innerNumGlutenNodes, innerFallbackNodeToReason) = - withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { - // re-plan manually to skip cached data - val newSparkPlan = QueryExecution.createSparkPlan( - dateset.sparkSession, - dateset.sparkSession.sessionState.planner, - p.inputPlan.logicalLink.get) - val newExecutedPlan = QueryExecution.prepareExecutedPlan( - dateset.sparkSession, - newSparkPlan - ) - processPlan( - newExecutedPlan, - new PlanStringConcat().append, - Some(plan => collectFallbackNodes(plan))) - } - numGlutenNodes += innerNumGlutenNodes - fallbackNodeToReason.++=(innerFallbackNodeToReason) - case p: QueryStageExec => collect(p.plan) - case p: GlutenPlan => - numGlutenNodes += 1 - p.innerChildren.foreach(collect) - case i: InMemoryTableScanExec => - if (PlanUtil.isGlutenTableCache(i)) { - numGlutenNodes += 1 - } else { - addFallbackNodeWithReason(i, "Columnar table cache is disabled", fallbackNodeToReason) + private def collectFallbackNodes(spark: SparkSession, plan: QueryPlan[_]): FallbackInfo = { + var numGlutenNodes = 0 + val fallbackNodeToReason = new mutable.HashMap[String, String] + + def collect(tmp: QueryPlan[_]): Unit = { + tmp.foreachUp { + case _: ExecutedCommandExec => + case _: CommandResultExec => + case _: V2CommandExec => + case _: DataWritingCommandExec => + case _: WholeStageCodegenExec => + case _: WholeStageTransformer => + case _: InputAdapter => + case _: ColumnarInputAdapter => + case _: InputIteratorTransformer => + case _: ColumnarToRowTransition => + case _: RowToColumnarTransition => + case p: ReusedExchangeExec => + case p: AdaptiveSparkPlanExec if isFinalAdaptivePlan(p) => + collect(p.executedPlan) + case p: AdaptiveSparkPlanExec => + // if we are here that means we are inside table cache. + val (innerNumGlutenNodes, innerFallbackNodeToReason) = + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + // re-plan manually to skip cached data + val newSparkPlan = QueryExecution.createSparkPlan( + spark, + spark.sessionState.planner, + p.inputPlan.logicalLink.get) + val newExecutedPlan = QueryExecution.prepareExecutedPlan( + spark, + newSparkPlan + ) + processPlan( + newExecutedPlan, + new PlanStringConcat().append, + Some(plan => collectFallbackNodes(spark, plan))) } - collect(i.relation.cachedPlan) - case _: AQEShuffleReadExec => // Ignore - case p: SparkPlan => - handleVanillaSparkPlan(p, fallbackNodeToReason) - p.innerChildren.foreach(collect) - case _ => - } + numGlutenNodes += innerNumGlutenNodes + fallbackNodeToReason.++=(innerFallbackNodeToReason) + case p: QueryStageExec => collect(p.plan) + case p: GlutenPlan => + numGlutenNodes += 1 + p.innerChildren.foreach(collect) + case i: InMemoryTableScanExec => + if (PlanUtil.isGlutenTableCache(i)) { + numGlutenNodes += 1 + } else { + addFallbackNodeWithReason(i, "Columnar table cache is disabled", fallbackNodeToReason) + } + collect(i.relation.cachedPlan) + case _: AQEShuffleReadExec => // Ignore + case p: SparkPlan => + handleVanillaSparkPlan(p, fallbackNodeToReason) + p.innerChildren.foreach(collect) + case _ => } - - collect(plan) - (numGlutenNodes, fallbackNodeToReason.toMap) } - private def collectQueryExecutionFallbackSummary(qe: QueryExecution): FallbackSummary = { - var totalNumGlutenNodes = 0 - var totalNumFallbackNodes = 0 - val totalPhysicalPlanDescription = new ArrayBuffer[String]() - val totalFallbackNodeToReason = new ArrayBuffer[Map[String, String]]() - - def handlePlanWithAQEAndTableCache( - plan: SparkPlan, - logicalPlan: LogicalPlan, - isMaterialized: Boolean): Unit = { - val concat = new PlanStringConcat() - val collectFallbackFunc = Some(plan => collectFallbackNodes(plan)) - val (numGlutenNodes, fallbackNodeToReason) = if (!isMaterialized) { - withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { - // AQE is not materialized, so the columnar rules are not applied. - // For this case, We apply columnar rules manually with disable AQE. - val qe = dateset.sparkSession.sessionState.executePlan(logicalPlan) - processPlan(qe.executedPlan, concat.append, collectFallbackFunc) - } - } else { - processPlan(plan, concat.append, collectFallbackFunc) - } - totalNumGlutenNodes += numGlutenNodes - totalNumFallbackNodes += fallbackNodeToReason.size - totalPhysicalPlanDescription.append(concat.toString()) - totalFallbackNodeToReason.append(fallbackNodeToReason) - } + collect(plan) + (numGlutenNodes, fallbackNodeToReason.toMap) + } - // For command-like query, e.g., `INSERT INTO TABLE ...` - qe.commandExecuted.foreach { - case r: CommandResult => - handlePlanWithAQEAndTableCache(r.commandPhysicalPlan, r.commandLogicalPlan, true) - case _ => // ignore + // collect fallback sumaary from query execution, make this method public as a util method + def collectQueryExecutionFallbackSummary( + spark: SparkSession, + qe: QueryExecution): FallbackSummary = { + var totalNumGlutenNodes = 0 + var totalNumFallbackNodes = 0 + val totalPhysicalPlanDescription = new ArrayBuffer[String]() + val totalFallbackNodeToReason = new ArrayBuffer[Map[String, String]]() + + def handlePlanWithAQEAndTableCache( + plan: SparkPlan, + logicalPlan: LogicalPlan, + isMaterialized: Boolean): Unit = { + val concat = new PlanStringConcat() + val collectFallbackFunc = Some(plan => collectFallbackNodes(spark, plan)) + val (numGlutenNodes, fallbackNodeToReason) = if (!isMaterialized) { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + // AQE is not materialized, so the columnar rules are not applied. + // For this case, We apply columnar rules manually with disable AQE. + val qe = spark.sessionState.executePlan(logicalPlan) + processPlan(qe.executedPlan, concat.append, collectFallbackFunc) + } + } else { + processPlan(plan, concat.append, collectFallbackFunc) } + totalNumGlutenNodes += numGlutenNodes + totalNumFallbackNodes += fallbackNodeToReason.size + totalPhysicalPlanDescription.append(concat.toString()) + totalFallbackNodeToReason.append(fallbackNodeToReason) + } - // For query, e.g., `SELECT * FROM ...` - if (qe.executedPlan.find(_.isInstanceOf[CommandResultExec]).isEmpty) { - val isMaterialized = qe.executedPlan.find { - case a: AdaptiveSparkPlanExec if isFinalAdaptivePlan(a) => true - case _ => false - }.isDefined - handlePlanWithAQEAndTableCache(qe.executedPlan, qe.analyzed, isMaterialized) - } + // For command-like query, e.g., `INSERT INTO TABLE ...` + qe.commandExecuted.foreach { + case r: CommandResult => + handlePlanWithAQEAndTableCache(r.commandPhysicalPlan, r.commandLogicalPlan, true) + case _ => // ignore + } - FallbackSummary( - totalNumGlutenNodes, - totalNumFallbackNodes, - totalPhysicalPlanDescription.toSeq, - totalFallbackNodeToReason.toSeq - ) + // For query, e.g., `SELECT * FROM ...` + if (qe.executedPlan.find(_.isInstanceOf[CommandResultExec]).isEmpty) { + val isMaterialized = qe.executedPlan.find { + case a: AdaptiveSparkPlanExec if isFinalAdaptivePlan(a) => true + case _ => false + }.isDefined + handlePlanWithAQEAndTableCache(qe.executedPlan, qe.analyzed, isMaterialized) } + FallbackSummary( + totalNumGlutenNodes, + totalNumFallbackNodes, + totalPhysicalPlanDescription.toSeq, + totalFallbackNodeToReason.toSeq + ) + } + + implicit class DatasetTransformer[T](dateset: Dataset[T]) { def fallbackSummary(): FallbackSummary = { - collectQueryExecutionFallbackSummary(dateset.queryExecution) + collectQueryExecutionFallbackSummary(dateset.sparkSession, dateset.queryExecution) } } } From cd624d711eaa55595535834ed6e63f2945757b2b Mon Sep 17 00:00:00 2001 From: lgbo Date: Fri, 2 Aug 2024 10:44:28 +0800 Subject: [PATCH 49/61] [GLUTEN-6557][CH] Try to replace sort merge join with hash join when cannot offload it (#6570) [CH] Try to replace sort merge join with hash join when cannot offload it --- .../clickhouse/CHSparkPlanExecApi.scala | 7 +- .../execution/CHHashJoinExecTransformer.scala | 60 +++++---- .../RewriteSortMergeJoinToHashJoinRule.scala | 122 ++++++++++++++++++ .../gluten/utils/CHJoinValidateUtil.scala | 12 +- .../GlutenClickHouseTPCDSAbstractSuite.scala | 7 +- ...kHouseTPCDSParquetSortMergeJoinSuite.scala | 22 +++- 6 files changed, 183 insertions(+), 47 deletions(-) create mode 100644 backends-clickhouse/src/main/scala/org/apache/gluten/extension/RewriteSortMergeJoinToHashJoinRule.scala diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala index b8a76b4210c3f..3069c4a3f6505 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala @@ -21,7 +21,7 @@ import org.apache.gluten.backendsapi.{BackendsApiManager, SparkPlanExecApi} import org.apache.gluten.exception.GlutenNotSupportException import org.apache.gluten.execution._ import org.apache.gluten.expression._ -import org.apache.gluten.extension.{CountDistinctWithoutExpand, FallbackBroadcastHashJoin, FallbackBroadcastHashJoinPrepQueryStage, RewriteToDateExpresstionRule} +import org.apache.gluten.extension.{CountDistinctWithoutExpand, FallbackBroadcastHashJoin, FallbackBroadcastHashJoinPrepQueryStage, RewriteSortMergeJoinToHashJoinRule, RewriteToDateExpresstionRule} import org.apache.gluten.extension.columnar.AddFallbackTagRule import org.apache.gluten.extension.columnar.MiscColumnarRules.TransformPreOverrides import org.apache.gluten.extension.columnar.transition.Convention @@ -555,8 +555,9 @@ class CHSparkPlanExecApi extends SparkPlanExecApi { * * @return */ - override def genExtendedQueryStagePrepRules(): List[SparkSession => Rule[SparkPlan]] = + override def genExtendedQueryStagePrepRules(): List[SparkSession => Rule[SparkPlan]] = { List(spark => FallbackBroadcastHashJoinPrepQueryStage(spark)) + } /** * Generate extended Analyzers. Currently only for ClickHouse backend. @@ -597,7 +598,7 @@ class CHSparkPlanExecApi extends SparkPlanExecApi { * @return */ override def genExtendedColumnarTransformRules(): List[SparkSession => Rule[SparkPlan]] = - List() + List(spark => RewriteSortMergeJoinToHashJoinRule(spark)) override def genInjectPostHocResolutionRules(): List[SparkSession => Rule[LogicalPlan]] = { List() diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashJoinExecTransformer.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashJoinExecTransformer.scala index c44156373528f..ed946e1d263d7 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashJoinExecTransformer.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashJoinExecTransformer.scala @@ -31,6 +31,35 @@ import org.apache.spark.sql.vectorized.ColumnarBatch import io.substrait.proto.JoinRel +object JoinTypeTransform { + def toNativeJoinType(joinType: JoinType): JoinType = { + joinType match { + case ExistenceJoin(_) => + LeftSemi + case _ => + joinType + } + } + + def toSubstraitType(joinType: JoinType): JoinRel.JoinType = { + joinType match { + case _: InnerLike => + JoinRel.JoinType.JOIN_TYPE_INNER + case FullOuter => + JoinRel.JoinType.JOIN_TYPE_OUTER + case LeftOuter | RightOuter => + JoinRel.JoinType.JOIN_TYPE_LEFT + case LeftSemi | ExistenceJoin(_) => + JoinRel.JoinType.JOIN_TYPE_LEFT_SEMI + case LeftAnti => + JoinRel.JoinType.JOIN_TYPE_ANTI + case _ => + // TODO: Support cross join with Cross Rel + JoinRel.JoinType.UNRECOGNIZED + } + } +} + case class CHShuffledHashJoinExecTransformer( leftKeys: Seq[Expression], rightKeys: Seq[Expression], @@ -57,7 +86,7 @@ case class CHShuffledHashJoinExecTransformer( override protected def doValidateInternal(): ValidationResult = { val shouldFallback = CHJoinValidateUtil.shouldFallback( - ShuffleHashJoinStrategy(joinType), + ShuffleHashJoinStrategy(finalJoinType), left.outputSet, right.outputSet, condition) @@ -66,6 +95,9 @@ case class CHShuffledHashJoinExecTransformer( } super.doValidateInternal() } + private val finalJoinType = JoinTypeTransform.toNativeJoinType(joinType) + override protected lazy val substraitJoinType: JoinRel.JoinType = + JoinTypeTransform.toSubstraitType(joinType) } case class CHBroadcastBuildSideRDD( @@ -171,27 +203,7 @@ case class CHBroadcastHashJoinExecTransformer( // Indeed, the ExistenceJoin is transformed into left any join in CH. // We don't have left any join in substrait, so use left semi join instead. // and isExistenceJoin is set to true to indicate that it is an existence join. - private val finalJoinType = joinType match { - case ExistenceJoin(_) => - LeftSemi - case _ => - joinType - } - override protected lazy val substraitJoinType: JoinRel.JoinType = { - joinType match { - case _: InnerLike => - JoinRel.JoinType.JOIN_TYPE_INNER - case FullOuter => - JoinRel.JoinType.JOIN_TYPE_OUTER - case LeftOuter | RightOuter => - JoinRel.JoinType.JOIN_TYPE_LEFT - case LeftSemi | ExistenceJoin(_) => - JoinRel.JoinType.JOIN_TYPE_LEFT_SEMI - case LeftAnti => - JoinRel.JoinType.JOIN_TYPE_ANTI - case _ => - // TODO: Support cross join with Cross Rel - JoinRel.JoinType.UNRECOGNIZED - } - } + private val finalJoinType = JoinTypeTransform.toNativeJoinType(joinType) + override protected lazy val substraitJoinType: JoinRel.JoinType = + JoinTypeTransform.toSubstraitType(joinType) } diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/RewriteSortMergeJoinToHashJoinRule.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/RewriteSortMergeJoinToHashJoinRule.scala new file mode 100644 index 0000000000000..8c5ada043fbb6 --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/RewriteSortMergeJoinToHashJoinRule.scala @@ -0,0 +1,122 @@ +/* + * 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.gluten.extension + +import org.apache.gluten.execution._ +import org.apache.gluten.utils.{CHJoinValidateUtil, ShuffleHashJoinStrategy} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.optimizer._ +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.adaptive.AQEShuffleReadExec +import org.apache.spark.sql.execution.joins._ + +// import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec +// If a SortMergeJoinExec cannot be offloaded, try to replace it with ShuffledHashJoinExec +// instead. +// This is rule is applied after spark plan nodes are transformed into columnar ones. +case class RewriteSortMergeJoinToHashJoinRule(session: SparkSession) + extends Rule[SparkPlan] + with Logging { + override def apply(plan: SparkPlan): SparkPlan = { + visitPlan(plan) + } + + private def visitPlan(plan: SparkPlan): SparkPlan = { + plan match { + case smj: SortMergeJoinExec => + tryReplaceSortMergeJoin(smj) + case other => + other.withNewChildren(other.children.map(visitPlan)) + } + } + + private def tryReplaceSortMergeJoin(smj: SortMergeJoinExec): SparkPlan = { + // cannot offload SortMergeJoin, try to replace it with ShuffledHashJoin + val needFallback = CHJoinValidateUtil.shouldFallback( + ShuffleHashJoinStrategy(smj.joinType), + smj.left.outputSet, + smj.right.outputSet, + smj.condition) + // also cannot offload HashJoin, don't replace it. + if (needFallback) { + logInfo(s"Cannot offload this join by hash join algorithm") + return smj + } else { + replaceSortMergeJoinWithHashJoin(smj) + } + } + + private def replaceSortMergeJoinWithHashJoin(smj: SortMergeJoinExec): SparkPlan = { + val newLeft = replaceSortMergeJoinChild(smj.left) + val newRight = replaceSortMergeJoinChild(smj.right) + // Some cases that we cannot handle. + if (newLeft == null || newRight == null) { + logInfo("Apply on sort merge children failed") + return smj + } + + var hashJoin = CHShuffledHashJoinExecTransformer( + smj.leftKeys, + smj.rightKeys, + smj.joinType, + BuildRight, + smj.condition, + newLeft, + newRight, + smj.isSkewJoin) + val validateResult = hashJoin.doValidate() + if (!validateResult.ok()) { + logError(s"Validation failed for ShuffledHashJoinExec: ${validateResult.reason()}") + return smj + } + hashJoin + } + + private def replaceSortMergeJoinChild(plan: SparkPlan): SparkPlan = { + plan match { + case sort: SortExecTransformer => + sort.child match { + case hashShuffle: ColumnarShuffleExchangeExec => + // drop sort node, return the shuffle node direclty + hashShuffle.withNewChildren(hashShuffle.children.map(visitPlan)) + case aqeShuffle: AQEShuffleReadExec => + // drop sort node, return the shuffle node direclty + aqeShuffle.withNewChildren(aqeShuffle.children.map(visitPlan)) + case columnarPlan: TransformSupport => + visitPlan(columnarPlan) + case _ => + // other cases that we don't know + logInfo(s"Expected ColumnarShuffleExchangeExec, got ${sort.child.getClass}") + null + } + case smj: SortMergeJoinExec => + val newChild = replaceSortMergeJoinWithHashJoin(smj) + if (newChild.isInstanceOf[SortMergeJoinExec]) { + null + } else { + newChild + } + case _: TransformSupport => visitPlan(plan) + case _ => + logInfo(s"Expected Columnar node, got ${plan.getClass}") + null + } + } +} diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/utils/CHJoinValidateUtil.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/utils/CHJoinValidateUtil.scala index 08b5ef5b2ef0d..0f5b5e2c4fd5a 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/utils/CHJoinValidateUtil.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/utils/CHJoinValidateUtil.scala @@ -18,7 +18,7 @@ package org.apache.gluten.utils import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.{AttributeSet, Expression} -import org.apache.spark.sql.catalyst.plans.JoinType +import org.apache.spark.sql.catalyst.plans._ trait JoinStrategy { val joinType: JoinType @@ -54,11 +54,8 @@ object CHJoinValidateUtil extends Logging { condition: Option[Expression]): Boolean = { var shouldFallback = false val joinType = joinStrategy.joinType - if (joinType.toString.contains("ExistenceJoin")) { - logError("Fallback for join type ExistenceJoin") - return true - } - if (joinType.sql.contains("INNER")) { + + if (!joinType.isInstanceOf[ExistenceJoin] && joinType.sql.contains("INNER")) { shouldFallback = false; } else if ( condition.isDefined && hasTwoTableColumn(leftOutputSet, rightOutputSet, condition.get) @@ -75,7 +72,8 @@ object CHJoinValidateUtil extends Logging { } else { shouldFallback = joinStrategy match { case SortMergeJoinStrategy(joinTy) => - joinTy.sql.contains("SEMI") || joinTy.sql.contains("ANTI") + joinTy.sql.contains("SEMI") || joinTy.sql.contains("ANTI") || joinTy.toString.contains( + "ExistenceJoin") case _ => false } } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSAbstractSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSAbstractSuite.scala index 6ca587bebc283..f2a1e5a71ca6c 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSAbstractSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCDSAbstractSuite.scala @@ -66,12 +66,7 @@ abstract class GlutenClickHouseTPCDSAbstractSuite // q16 smj + left semi + not condition // Q94 BroadcastHashJoin, LeftSemi, NOT condition - if (isAqe) { - Set(16, 94) | more - } else { - // q10, q35 smj + existence join - Set(10, 16, 35, 94) | more - } + Set(16, 94) | more } protected def excludedTpcdsQueries: Set[String] = Set( "q66" // inconsistent results diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetSortMergeJoinSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetSortMergeJoinSuite.scala index 7e480361bfe19..509c830545c60 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetSortMergeJoinSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetSortMergeJoinSuite.scala @@ -16,7 +16,7 @@ */ package org.apache.gluten.execution.tpcds -import org.apache.gluten.execution.{CHSortMergeJoinExecTransformer, GlutenClickHouseTPCDSAbstractSuite} +import org.apache.gluten.execution.{CHShuffledHashJoinExecTransformer, CHSortMergeJoinExecTransformer, GlutenClickHouseTPCDSAbstractSuite} import org.apache.gluten.test.FallbackUtil import org.apache.spark.SparkConf @@ -114,7 +114,7 @@ class GlutenClickHouseTPCDSParquetSortMergeJoinSuite extends GlutenClickHouseTPC } } - test("sort merge join: left semi join should fallback") { + test("sort merge join: left semi join should be replaced with hash join") { withSQLConf("spark.sql.autoBroadcastJoinThreshold" -> "-1") { val testSql = """SELECT count(*) cnt @@ -125,12 +125,16 @@ class GlutenClickHouseTPCDSParquetSortMergeJoinSuite extends GlutenClickHouseTPC val smjTransformers = df.queryExecution.executedPlan.collect { case f: CHSortMergeJoinExecTransformer => f } - assert(smjTransformers.isEmpty) - assert(FallbackUtil.hasFallback(df.queryExecution.executedPlan)) + val hashJoinTransformers = df.queryExecution.executedPlan.collect { + case f: CHShuffledHashJoinExecTransformer => f + } + assert(smjTransformers.size == 0) + assert(hashJoinTransformers.size > 0) + assert(!FallbackUtil.hasFallback(df.queryExecution.executedPlan)) } } - test("sort merge join: left anti join should fallback") { + test("sort merge join: left anti join should be replace with hash join") { withSQLConf("spark.sql.autoBroadcastJoinThreshold" -> "-1") { val testSql = """SELECT count(*) cnt @@ -141,8 +145,12 @@ class GlutenClickHouseTPCDSParquetSortMergeJoinSuite extends GlutenClickHouseTPC val smjTransformers = df.queryExecution.executedPlan.collect { case f: CHSortMergeJoinExecTransformer => f } - assert(smjTransformers.isEmpty) - assert(FallbackUtil.hasFallback(df.queryExecution.executedPlan)) + val hashJoinTransformers = df.queryExecution.executedPlan.collect { + case f: CHShuffledHashJoinExecTransformer => f + } + assert(smjTransformers.size == 0) + assert(hashJoinTransformers.size > 0) + assert(!FallbackUtil.hasFallback(df.queryExecution.executedPlan)) } } From 1dad0e6dfac058142834b700ba0aa925a9b75a85 Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Fri, 2 Aug 2024 11:16:08 +0800 Subject: [PATCH 50/61] [VL] Daily Update Velox Version (2024_08_02) (#6684) --- ep/build-velox/src/get_velox.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 6536afcb1c7a0..c98b8e9ec0da4 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_08_01 +VELOX_BRANCH=2024_08_02 VELOX_HOME="" OS=`uname -s` From 807d9f77cb412f3ab90ea9e15408019c52941399 Mon Sep 17 00:00:00 2001 From: WangGuangxin Date: Fri, 2 Aug 2024 11:26:48 +0800 Subject: [PATCH 51/61] [GLUTEN-6645][VL] Remove VeloxWriteQueue which may introduce deadlock (#6646) --- .../VeloxColumnarBatchIterator.scala | 76 ---------------- .../datasources/VeloxWriteQueue.scala | 90 ------------------- .../velox/VeloxFormatWriterInjects.scala | 31 +++---- cpp/core/jni/JniWrapper.cc | 14 +-- .../datasource/DatasourceJniWrapper.java | 3 +- .../org/apache/gluten/GlutenConfig.scala | 7 -- 6 files changed, 18 insertions(+), 203 deletions(-) delete mode 100644 backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/VeloxColumnarBatchIterator.scala delete mode 100644 backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/VeloxWriteQueue.scala diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/VeloxColumnarBatchIterator.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/VeloxColumnarBatchIterator.scala deleted file mode 100644 index 0e6aceddfc1a4..0000000000000 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/VeloxColumnarBatchIterator.scala +++ /dev/null @@ -1,76 +0,0 @@ -/* - * 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.spark.sql.execution.datasources - -import org.apache.gluten.exception.GlutenException - -import org.apache.spark.sql.execution.datasources.VeloxWriteQueue.EOS_BATCH -import org.apache.spark.sql.vectorized.ColumnarBatch - -import org.apache.arrow.memory.BufferAllocator -import org.apache.arrow.vector.types.pojo.Schema - -import java.util.concurrent.{ArrayBlockingQueue, TimeUnit} - -class VeloxColumnarBatchIterator(schema: Schema, allocator: BufferAllocator, queueSize: Int) - extends Iterator[ColumnarBatch] - with AutoCloseable { - private val writeQueue = new ArrayBlockingQueue[ColumnarBatch](queueSize) - private var currentBatch: Option[ColumnarBatch] = None - - def enqueue(batch: ColumnarBatch): Unit = { - // Throw exception if the queue is full. - if (!writeQueue.offer(batch, 30L, TimeUnit.MINUTES)) { - throw new GlutenException("VeloxParquetWriter: Timeout waiting for adding data") - } - } - - override def hasNext: Boolean = { - val batch = - try { - writeQueue.poll(30L, TimeUnit.MINUTES) - } catch { - case _: InterruptedException => - Thread.currentThread().interrupt() - EOS_BATCH - } - if (batch == null) { - throw new GlutenException("VeloxParquetWriter: Timeout waiting for data") - } - if (batch == EOS_BATCH) { - return false - } - currentBatch = Some(batch) - true - } - - override def next(): ColumnarBatch = { - try { - currentBatch match { - case Some(b) => b - case _ => - throw new IllegalStateException("VeloxParquetWriter: Fatal: Call hasNext() first!") - } - } finally { - currentBatch = None - } - } - - override def close(): Unit = { - allocator.close() - } -} diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/VeloxWriteQueue.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/VeloxWriteQueue.scala deleted file mode 100644 index 6e3e64796b329..0000000000000 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/VeloxWriteQueue.scala +++ /dev/null @@ -1,90 +0,0 @@ -/* - * 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.spark.sql.execution.datasources - -import org.apache.gluten.datasource.DatasourceJniWrapper -import org.apache.gluten.utils.iterator.Iterators -import org.apache.gluten.vectorized.ColumnarBatchInIterator - -import org.apache.spark.TaskContext -import org.apache.spark.sql.execution.datasources.VeloxWriteQueue.EOS_BATCH -import org.apache.spark.sql.vectorized.ColumnarBatch - -import org.apache.arrow.memory.BufferAllocator -import org.apache.arrow.vector.types.pojo.Schema - -import java.util.UUID -import java.util.concurrent.atomic.AtomicReference -import java.util.regex.Pattern - -import scala.collection.JavaConverters._ - -// TODO: This probably can be removed: Velox's Parquet writer already supports push-based write. -class VeloxWriteQueue( - tc: TaskContext, - dsHandle: Long, - schema: Schema, - allocator: BufferAllocator, - datasourceJniWrapper: DatasourceJniWrapper, - outputFileURI: String, - queueSize: Int) - extends AutoCloseable { - private val scanner = new VeloxColumnarBatchIterator(schema, allocator, queueSize) - private val writeException = new AtomicReference[Throwable] - - private val writeThread = new Thread( - () => { - TaskContext.setTaskContext(tc) - try { - datasourceJniWrapper.write( - dsHandle, - new ColumnarBatchInIterator( - Iterators.wrap(scanner).recyclePayload(_.close()).create().asJava)) - } catch { - case e: Exception => - writeException.set(e) - } - }, - "VeloxWriteQueue - " + UUID.randomUUID().toString - ) - - writeThread.start() - - private def checkWriteException(): Unit = { - // check if VeloxWriteQueue thread was failed - val exception = writeException.get() - if (exception != null) { - throw exception - } - } - - def enqueue(batch: ColumnarBatch): Unit = { - scanner.enqueue(batch) - checkWriteException() - } - - override def close(): Unit = { - scanner.enqueue(EOS_BATCH) - writeThread.join() - checkWriteException() - } -} - -object VeloxWriteQueue { - val EOS_BATCH = new ColumnarBatch(null) - val TAILING_FILENAME_REGEX = Pattern.compile("^(.*)/([^/]+)$") -} diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxFormatWriterInjects.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxFormatWriterInjects.scala index 6901bfffdf5e9..7da4da5f0784a 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxFormatWriterInjects.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxFormatWriterInjects.scala @@ -16,8 +16,7 @@ */ package org.apache.spark.sql.execution.datasources.velox -import org.apache.gluten.GlutenConfig -import org.apache.gluten.columnarbatch.ColumnarBatches +import org.apache.gluten.columnarbatch.{ColumnarBatches, ColumnarBatchJniWrapper} import org.apache.gluten.datasource.DatasourceJniWrapper import org.apache.gluten.exception.GlutenException import org.apache.gluten.exec.Runtimes @@ -31,7 +30,6 @@ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType import org.apache.spark.sql.utils.SparkArrowUtil -import org.apache.spark.util.TaskResources import com.google.common.base.Preconditions import org.apache.arrow.c.ArrowSchema @@ -74,29 +72,26 @@ trait VeloxFormatWriterInjects extends GlutenFormatWriterInjectsBase { cSchema.close() } - // FIXME: remove this once we support push-based write. - val queueSize = context.getConfiguration.getInt(GlutenConfig.VELOX_WRITER_QUEUE_SIZE.key, 64) - - val writeQueue = - new VeloxWriteQueue( - TaskResources.getLocalTaskContext(), - dsHandle, - arrowSchema, - allocator, - datasourceJniWrapper, - filePath, - queueSize) - new OutputWriter { override def write(row: InternalRow): Unit = { val batch = row.asInstanceOf[FakeRow].batch Preconditions.checkState(ColumnarBatches.isLightBatch(batch)) ColumnarBatches.retain(batch) - writeQueue.enqueue(batch) + val batchHandle = { + if (batch.numCols == 0) { + // the operation will find a zero column batch from a task-local pool + ColumnarBatchJniWrapper.create(runtime).getForEmptySchema(batch.numRows) + } else { + val offloaded = + ColumnarBatches.ensureOffloaded(ArrowBufferAllocators.contextInstance, batch) + ColumnarBatches.getNativeHandle(offloaded) + } + } + datasourceJniWrapper.writeBatch(dsHandle, batchHandle) + batch.close() } override def close(): Unit = { - writeQueue.close() datasourceJniWrapper.close(dsHandle) } diff --git a/cpp/core/jni/JniWrapper.cc b/cpp/core/jni/JniWrapper.cc index f39f9c92333e9..4fa45d9cbe067 100644 --- a/cpp/core/jni/JniWrapper.cc +++ b/cpp/core/jni/JniWrapper.cc @@ -1093,22 +1093,16 @@ JNIEXPORT void JNICALL Java_org_apache_gluten_datasource_DatasourceJniWrapper_cl JNI_METHOD_END() } -JNIEXPORT void JNICALL Java_org_apache_gluten_datasource_DatasourceJniWrapper_write( // NOLINT +JNIEXPORT void JNICALL Java_org_apache_gluten_datasource_DatasourceJniWrapper_writeBatch( // NOLINT JNIEnv* env, jobject wrapper, jlong dsHandle, - jobject jIter) { + jlong batchHandle) { JNI_METHOD_START auto ctx = gluten::getRuntime(env, wrapper); auto datasource = ObjectStore::retrieve(dsHandle); - auto iter = makeJniColumnarBatchIterator(env, jIter, ctx, nullptr); - while (true) { - auto batch = iter->next(); - if (!batch) { - break; - } - datasource->write(batch); - } + auto batch = ObjectStore::retrieve(batchHandle); + datasource->write(batch); JNI_METHOD_END() } diff --git a/gluten-data/src/main/java/org/apache/gluten/datasource/DatasourceJniWrapper.java b/gluten-data/src/main/java/org/apache/gluten/datasource/DatasourceJniWrapper.java index 5a34196c4eb76..11ed3fb7df8c9 100644 --- a/gluten-data/src/main/java/org/apache/gluten/datasource/DatasourceJniWrapper.java +++ b/gluten-data/src/main/java/org/apache/gluten/datasource/DatasourceJniWrapper.java @@ -19,7 +19,6 @@ import org.apache.gluten.exec.Runtime; import org.apache.gluten.exec.RuntimeAware; import org.apache.gluten.init.JniUtils; -import org.apache.gluten.vectorized.ColumnarBatchInIterator; import org.apache.spark.sql.execution.datasources.BlockStripes; @@ -53,7 +52,7 @@ public long nativeInitDatasource(String filePath, long cSchema, Map Date: Fri, 2 Aug 2024 15:10:44 +0800 Subject: [PATCH 52/61] [VL] Recover broken memory-trace option spark.gluten.backtrace.allocation (#6635) --- cpp/core/jni/JniWrapper.cc | 14 +++++++++++--- cpp/core/memory/AllocationListener.cc | 2 -- cpp/core/memory/AllocationListener.h | 2 -- cpp/velox/compute/VeloxBackend.cc | 3 --- cpp/velox/config/VeloxConfig.h | 3 --- docs/developers/HowTo.md | 2 +- .../org/apache/gluten/GlutenConfig.scala | 19 +++++++++++++++++-- 7 files changed, 29 insertions(+), 16 deletions(-) diff --git a/cpp/core/jni/JniWrapper.cc b/cpp/core/jni/JniWrapper.cc index 4fa45d9cbe067..add4aa54d2078 100644 --- a/cpp/core/jni/JniWrapper.cc +++ b/cpp/core/jni/JniWrapper.cc @@ -215,6 +215,10 @@ void JNI_OnUnload(JavaVM* vm, void* reserved) { gluten::getJniCommonState()->close(); } +namespace { +const std::string kBacktraceAllocation = "spark.gluten.memory.backtrace.allocation"; +} + JNIEXPORT jlong JNICALL Java_org_apache_gluten_exec_RuntimeJniWrapper_createRuntime( // NOLINT JNIEnv* env, jclass, @@ -226,13 +230,17 @@ JNIEXPORT jlong JNICALL Java_org_apache_gluten_exec_RuntimeJniWrapper_createRunt if (env->GetJavaVM(&vm) != JNI_OK) { throw gluten::GlutenException("Unable to get JavaVM instance"); } - + auto safeArray = gluten::getByteArrayElementsSafe(env, sessionConf); + auto sparkConf = gluten::parseConfMap(env, safeArray.elems(), safeArray.length()); auto backendType = jStringToCString(env, jbackendType); + std::unique_ptr listener = std::make_unique(vm, jlistener, reserveMemoryMethod, unreserveMemoryMethod); + bool backtrace = sparkConf.at(kBacktraceAllocation) == "true"; + if (backtrace) { + listener = std::make_unique(std::move(listener)); + } - auto safeArray = gluten::getByteArrayElementsSafe(env, sessionConf); - auto sparkConf = gluten::parseConfMap(env, safeArray.elems(), safeArray.length()); auto runtime = gluten::Runtime::create(backendType, std::move(listener), sparkConf); return reinterpret_cast(runtime); JNI_METHOD_END(kInvalidObjectHandle) diff --git a/cpp/core/memory/AllocationListener.cc b/cpp/core/memory/AllocationListener.cc index 2c876e9f19f78..5cbeeb6bd5c93 100644 --- a/cpp/core/memory/AllocationListener.cc +++ b/cpp/core/memory/AllocationListener.cc @@ -19,8 +19,6 @@ namespace gluten { -bool backtrace_allocation = false; - class NoopAllocationListener : public gluten::AllocationListener { public: void allocationChanged(int64_t diff) override { diff --git a/cpp/core/memory/AllocationListener.h b/cpp/core/memory/AllocationListener.h index 41797641fe14b..1751b6112ae26 100644 --- a/cpp/core/memory/AllocationListener.h +++ b/cpp/core/memory/AllocationListener.h @@ -23,8 +23,6 @@ namespace gluten { -extern bool backtrace_allocation; - class AllocationListener { public: static std::unique_ptr noop(); diff --git a/cpp/velox/compute/VeloxBackend.cc b/cpp/velox/compute/VeloxBackend.cc index a3658faa3a188..2dad6adf2e70e 100644 --- a/cpp/velox/compute/VeloxBackend.cc +++ b/cpp/velox/compute/VeloxBackend.cc @@ -104,9 +104,6 @@ void VeloxBackend::init(const std::unordered_map& conf FLAGS_gluten_velox_aysnc_timeout_on_task_stopping = backendConf_->get(kVeloxAsyncTimeoutOnTaskStopping, kVeloxAsyncTimeoutOnTaskStoppingDefault); - // Set backtrace_allocation - gluten::backtrace_allocation = backendConf_->get(kBacktraceAllocation, false); - // Setup and register. velox::filesystems::registerLocalFileSystem(); initJolFilesystem(); diff --git a/cpp/velox/config/VeloxConfig.h b/cpp/velox/config/VeloxConfig.h index 65c7cb61d94dc..792beda96f7d0 100644 --- a/cpp/velox/config/VeloxConfig.h +++ b/cpp/velox/config/VeloxConfig.h @@ -104,9 +104,6 @@ const int32_t kVeloxAsyncTimeoutOnTaskStoppingDefault = 30000; // 30s // udf const std::string kVeloxUdfLibraryPaths = "spark.gluten.sql.columnar.backend.velox.internal.udfLibraryPaths"; -// backtrace allocation -const std::string kBacktraceAllocation = "spark.gluten.backtrace.allocation"; - // VeloxShuffleReader print flag. const std::string kVeloxShuffleReaderPrintFlag = "spark.gluten.velox.shuffleReaderPrintFlag"; diff --git a/docs/developers/HowTo.md b/docs/developers/HowTo.md index a13bf02ebede3..5b16c965fe633 100644 --- a/docs/developers/HowTo.md +++ b/docs/developers/HowTo.md @@ -163,7 +163,7 @@ wait to add # How to track the memory exhaust problem -When your gluten spark jobs failed because of OOM, you can track the memory allocation's call stack by configuring `spark.gluten.backtrace.allocation = true`. +When your gluten spark jobs failed because of OOM, you can track the memory allocation's call stack by configuring `spark.gluten.memory.backtrace.allocation = true`. The above configuration will use `BacktraceAllocationListener` wrapping from `SparkAllocationListener` to create `VeloxMemoryManager`. `BacktraceAllocationListener` will check every allocation, if a single allocation bytes exceeds a fixed value or the accumulative allocation bytes exceeds 1/2/3...G, diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index 82097811200d0..eb32a25e6a5b1 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -241,6 +241,8 @@ class GlutenConfig(conf: SQLConf) extends Logging { def memoryIsolation: Boolean = conf.getConf(COLUMNAR_MEMORY_ISOLATION) + def memoryBacktraceAllocation: Boolean = conf.getConf(COLUMNAR_MEMORY_BACKTRACE_ALLOCATION) + def numTaskSlotsPerExecutor: Int = { val numSlots = conf.getConf(NUM_TASK_SLOTS_PER_EXECUTOR) assert(numSlots > 0, s"Number of task slot not found. This should not happen.") @@ -659,7 +661,10 @@ object GlutenConfig { val keyWithDefault = ImmutableList.of( (SQLConf.CASE_SENSITIVE.key, SQLConf.CASE_SENSITIVE.defaultValueString), - (SQLConf.IGNORE_MISSING_FILES.key, SQLConf.IGNORE_MISSING_FILES.defaultValueString) + (SQLConf.IGNORE_MISSING_FILES.key, SQLConf.IGNORE_MISSING_FILES.defaultValueString), + ( + COLUMNAR_MEMORY_BACKTRACE_ALLOCATION.key, + COLUMNAR_MEMORY_BACKTRACE_ALLOCATION.defaultValueString) ) keyWithDefault.forEach(e => nativeConfMap.put(e._1, conf.getOrElse(e._1, e._2))) @@ -706,7 +711,9 @@ object GlutenConfig { (AWS_S3_RETRY_MODE.key, AWS_S3_RETRY_MODE.defaultValueString), ( COLUMNAR_VELOX_CONNECTOR_IO_THREADS.key, - conf.getOrElse(GLUTEN_NUM_TASK_SLOTS_PER_EXECUTOR_KEY, "-1")), + conf.getOrElse( + NUM_TASK_SLOTS_PER_EXECUTOR.key, + NUM_TASK_SLOTS_PER_EXECUTOR.defaultValueString)), (COLUMNAR_SHUFFLE_CODEC.key, ""), (COLUMNAR_SHUFFLE_CODEC_BACKEND.key, ""), ("spark.hadoop.input.connect.timeout", "180000"), @@ -1244,6 +1251,14 @@ object GlutenConfig { .booleanConf .createWithDefault(false) + val COLUMNAR_MEMORY_BACKTRACE_ALLOCATION = + buildConf("spark.gluten.memory.backtrace.allocation") + .internal() + .doc("Print backtrace information for large memory allocations. This helps debugging when " + + "Spark OOM happens due to large acquire requests.") + .booleanConf + .createWithDefault(false) + val COLUMNAR_MEMORY_OVER_ACQUIRED_RATIO = buildConf("spark.gluten.memory.overAcquiredMemoryRatio") .internal() From 94b79c7a8a3166d764c8d405a6aa3225365e924f Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Fri, 2 Aug 2024 15:18:57 +0800 Subject: [PATCH 53/61] [VL] Allow specifying maximum batch size for batch resizing (#6670) --- ...chAppender.java => VeloxBatchResizer.java} | 10 +- ....java => VeloxBatchResizerJniWrapper.java} | 11 +- .../velox/VeloxSparkPlanExecApi.scala | 5 +- ...xec.scala => VeloxResizeBatchesExec.scala} | 21 ++-- .../FlushableHashAggregateRule.scala | 2 +- .../v1-bhj-ras/spark32/1.txt | 12 +- .../v1-bhj-ras/spark32/10.txt | 6 +- .../v1-bhj-ras/spark32/11.txt | 12 +- .../v1-bhj-ras/spark32/12.txt | 12 +- .../v1-bhj-ras/spark32/13.txt | 18 +-- .../v1-bhj-ras/spark32/14.txt | 6 +- .../v1-bhj-ras/spark32/15.txt | 12 +- .../v1-bhj-ras/spark32/16.txt | 18 +-- .../v1-bhj-ras/spark32/17.txt | 6 +- .../v1-bhj-ras/spark32/18.txt | 12 +- .../v1-bhj-ras/spark32/19.txt | 6 +- .../v1-bhj-ras/spark32/20.txt | 18 +-- .../v1-bhj-ras/spark32/21.txt | 6 +- .../v1-bhj-ras/spark32/22.txt | 12 +- .../v1-bhj-ras/spark32/3.txt | 6 +- .../v1-bhj-ras/spark32/4.txt | 12 +- .../v1-bhj-ras/spark32/5.txt | 12 +- .../v1-bhj-ras/spark32/6.txt | 6 +- .../v1-bhj-ras/spark32/7.txt | 12 +- .../v1-bhj-ras/spark32/8.txt | 12 +- .../v1-bhj-ras/spark32/9.txt | 12 +- .../v1-bhj-ras/spark33/1.txt | 12 +- .../v1-bhj-ras/spark33/10.txt | 6 +- .../v1-bhj-ras/spark33/11.txt | 18 +-- .../v1-bhj-ras/spark33/12.txt | 12 +- .../v1-bhj-ras/spark33/13.txt | 18 +-- .../v1-bhj-ras/spark33/14.txt | 6 +- .../v1-bhj-ras/spark33/15.txt | 18 +-- .../v1-bhj-ras/spark33/16.txt | 18 +-- .../v1-bhj-ras/spark33/17.txt | 6 +- .../v1-bhj-ras/spark33/18.txt | 12 +- .../v1-bhj-ras/spark33/19.txt | 6 +- .../v1-bhj-ras/spark33/20.txt | 18 +-- .../v1-bhj-ras/spark33/21.txt | 6 +- .../v1-bhj-ras/spark33/22.txt | 18 +-- .../v1-bhj-ras/spark33/3.txt | 6 +- .../v1-bhj-ras/spark33/4.txt | 12 +- .../v1-bhj-ras/spark33/5.txt | 12 +- .../v1-bhj-ras/spark33/6.txt | 6 +- .../v1-bhj-ras/spark33/7.txt | 12 +- .../v1-bhj-ras/spark33/8.txt | 12 +- .../v1-bhj-ras/spark33/9.txt | 12 +- .../v1-bhj-ras/spark34/1.txt | 12 +- .../v1-bhj-ras/spark34/10.txt | 6 +- .../v1-bhj-ras/spark34/11.txt | 18 +-- .../v1-bhj-ras/spark34/12.txt | 12 +- .../v1-bhj-ras/spark34/13.txt | 18 +-- .../v1-bhj-ras/spark34/14.txt | 6 +- .../v1-bhj-ras/spark34/15.txt | 18 +-- .../v1-bhj-ras/spark34/16.txt | 18 +-- .../v1-bhj-ras/spark34/17.txt | 6 +- .../v1-bhj-ras/spark34/18.txt | 12 +- .../v1-bhj-ras/spark34/19.txt | 6 +- .../v1-bhj-ras/spark34/20.txt | 12 +- .../v1-bhj-ras/spark34/21.txt | 6 +- .../v1-bhj-ras/spark34/22.txt | 18 +-- .../v1-bhj-ras/spark34/3.txt | 6 +- .../v1-bhj-ras/spark34/4.txt | 12 +- .../v1-bhj-ras/spark34/5.txt | 12 +- .../v1-bhj-ras/spark34/6.txt | 6 +- .../v1-bhj-ras/spark34/7.txt | 12 +- .../v1-bhj-ras/spark34/8.txt | 12 +- .../v1-bhj-ras/spark34/9.txt | 12 +- .../tpch-approved-plan/v1-bhj/spark32/1.txt | 12 +- .../tpch-approved-plan/v1-bhj/spark32/10.txt | 6 +- .../tpch-approved-plan/v1-bhj/spark32/11.txt | 12 +- .../tpch-approved-plan/v1-bhj/spark32/12.txt | 12 +- .../tpch-approved-plan/v1-bhj/spark32/13.txt | 18 +-- .../tpch-approved-plan/v1-bhj/spark32/14.txt | 6 +- .../tpch-approved-plan/v1-bhj/spark32/15.txt | 12 +- .../tpch-approved-plan/v1-bhj/spark32/16.txt | 18 +-- .../tpch-approved-plan/v1-bhj/spark32/17.txt | 6 +- .../tpch-approved-plan/v1-bhj/spark32/18.txt | 12 +- .../tpch-approved-plan/v1-bhj/spark32/19.txt | 6 +- .../tpch-approved-plan/v1-bhj/spark32/20.txt | 18 +-- .../tpch-approved-plan/v1-bhj/spark32/21.txt | 6 +- .../tpch-approved-plan/v1-bhj/spark32/22.txt | 12 +- .../tpch-approved-plan/v1-bhj/spark32/3.txt | 6 +- .../tpch-approved-plan/v1-bhj/spark32/4.txt | 12 +- .../tpch-approved-plan/v1-bhj/spark32/5.txt | 12 +- .../tpch-approved-plan/v1-bhj/spark32/6.txt | 6 +- .../tpch-approved-plan/v1-bhj/spark32/7.txt | 12 +- .../tpch-approved-plan/v1-bhj/spark32/8.txt | 12 +- .../tpch-approved-plan/v1-bhj/spark32/9.txt | 12 +- .../tpch-approved-plan/v1-bhj/spark33/1.txt | 12 +- .../tpch-approved-plan/v1-bhj/spark33/10.txt | 6 +- .../tpch-approved-plan/v1-bhj/spark33/11.txt | 18 +-- .../tpch-approved-plan/v1-bhj/spark33/12.txt | 12 +- .../tpch-approved-plan/v1-bhj/spark33/13.txt | 18 +-- .../tpch-approved-plan/v1-bhj/spark33/14.txt | 6 +- .../tpch-approved-plan/v1-bhj/spark33/15.txt | 18 +-- .../tpch-approved-plan/v1-bhj/spark33/16.txt | 18 +-- .../tpch-approved-plan/v1-bhj/spark33/17.txt | 6 +- .../tpch-approved-plan/v1-bhj/spark33/18.txt | 12 +- .../tpch-approved-plan/v1-bhj/spark33/19.txt | 6 +- .../tpch-approved-plan/v1-bhj/spark33/20.txt | 18 +-- .../tpch-approved-plan/v1-bhj/spark33/21.txt | 6 +- .../tpch-approved-plan/v1-bhj/spark33/22.txt | 20 ++-- .../tpch-approved-plan/v1-bhj/spark33/3.txt | 6 +- .../tpch-approved-plan/v1-bhj/spark33/4.txt | 12 +- .../tpch-approved-plan/v1-bhj/spark33/5.txt | 12 +- .../tpch-approved-plan/v1-bhj/spark33/6.txt | 6 +- .../tpch-approved-plan/v1-bhj/spark33/7.txt | 12 +- .../tpch-approved-plan/v1-bhj/spark33/8.txt | 12 +- .../tpch-approved-plan/v1-bhj/spark33/9.txt | 12 +- .../tpch-approved-plan/v1-bhj/spark34/1.txt | 12 +- .../tpch-approved-plan/v1-bhj/spark34/10.txt | 6 +- .../tpch-approved-plan/v1-bhj/spark34/11.txt | 18 +-- .../tpch-approved-plan/v1-bhj/spark34/12.txt | 12 +- .../tpch-approved-plan/v1-bhj/spark34/13.txt | 18 +-- .../tpch-approved-plan/v1-bhj/spark34/14.txt | 6 +- .../tpch-approved-plan/v1-bhj/spark34/15.txt | 18 +-- .../tpch-approved-plan/v1-bhj/spark34/16.txt | 18 +-- .../tpch-approved-plan/v1-bhj/spark34/17.txt | 6 +- .../tpch-approved-plan/v1-bhj/spark34/18.txt | 12 +- .../tpch-approved-plan/v1-bhj/spark34/19.txt | 6 +- .../tpch-approved-plan/v1-bhj/spark34/20.txt | 12 +- .../tpch-approved-plan/v1-bhj/spark34/21.txt | 6 +- .../tpch-approved-plan/v1-bhj/spark34/22.txt | 20 ++-- .../tpch-approved-plan/v1-bhj/spark34/3.txt | 6 +- .../tpch-approved-plan/v1-bhj/spark34/4.txt | 12 +- .../tpch-approved-plan/v1-bhj/spark34/5.txt | 12 +- .../tpch-approved-plan/v1-bhj/spark34/6.txt | 6 +- .../tpch-approved-plan/v1-bhj/spark34/7.txt | 12 +- .../tpch-approved-plan/v1-bhj/spark34/8.txt | 12 +- .../tpch-approved-plan/v1-bhj/spark34/9.txt | 12 +- .../tpch-approved-plan/v1-ras/spark32/1.txt | 12 +- .../tpch-approved-plan/v1-ras/spark32/10.txt | 42 +++---- .../tpch-approved-plan/v1-ras/spark32/11.txt | 36 +++--- .../tpch-approved-plan/v1-ras/spark32/12.txt | 24 ++-- .../tpch-approved-plan/v1-ras/spark32/13.txt | 24 ++-- .../tpch-approved-plan/v1-ras/spark32/14.txt | 12 +- .../tpch-approved-plan/v1-ras/spark32/15.txt | 18 +-- .../tpch-approved-plan/v1-ras/spark32/16.txt | 30 ++--- .../tpch-approved-plan/v1-ras/spark32/17.txt | 18 +-- .../tpch-approved-plan/v1-ras/spark32/18.txt | 36 +++--- .../tpch-approved-plan/v1-ras/spark32/19.txt | 12 +- .../tpch-approved-plan/v1-ras/spark32/20.txt | 60 +++++----- .../tpch-approved-plan/v1-ras/spark32/21.txt | 60 +++++----- .../tpch-approved-plan/v1-ras/spark32/22.txt | 24 ++-- .../tpch-approved-plan/v1-ras/spark32/3.txt | 24 ++-- .../tpch-approved-plan/v1-ras/spark32/4.txt | 24 ++-- .../tpch-approved-plan/v1-ras/spark32/5.txt | 72 ++++++------ .../tpch-approved-plan/v1-ras/spark32/6.txt | 6 +- .../tpch-approved-plan/v1-ras/spark32/7.txt | 66 +++++------ .../tpch-approved-plan/v1-ras/spark32/8.txt | 96 +++++++-------- .../tpch-approved-plan/v1-ras/spark32/9.txt | 72 ++++++------ .../tpch-approved-plan/v1-ras/spark33/1.txt | 12 +- .../tpch-approved-plan/v1-ras/spark33/10.txt | 42 +++---- .../tpch-approved-plan/v1-ras/spark33/11.txt | 48 ++++---- .../tpch-approved-plan/v1-ras/spark33/12.txt | 24 ++-- .../tpch-approved-plan/v1-ras/spark33/13.txt | 24 ++-- .../tpch-approved-plan/v1-ras/spark33/14.txt | 12 +- .../tpch-approved-plan/v1-ras/spark33/15.txt | 24 ++-- .../tpch-approved-plan/v1-ras/spark33/16.txt | 30 ++--- .../tpch-approved-plan/v1-ras/spark33/17.txt | 18 +-- .../tpch-approved-plan/v1-ras/spark33/18.txt | 36 +++--- .../tpch-approved-plan/v1-ras/spark33/19.txt | 12 +- .../tpch-approved-plan/v1-ras/spark33/20.txt | 60 +++++----- .../tpch-approved-plan/v1-ras/spark33/21.txt | 60 +++++----- .../tpch-approved-plan/v1-ras/spark33/22.txt | 30 ++--- .../tpch-approved-plan/v1-ras/spark33/3.txt | 24 ++-- .../tpch-approved-plan/v1-ras/spark33/4.txt | 24 ++-- .../tpch-approved-plan/v1-ras/spark33/5.txt | 72 ++++++------ .../tpch-approved-plan/v1-ras/spark33/6.txt | 6 +- .../tpch-approved-plan/v1-ras/spark33/7.txt | 66 +++++------ .../tpch-approved-plan/v1-ras/spark33/8.txt | 96 +++++++-------- .../tpch-approved-plan/v1-ras/spark33/9.txt | 72 ++++++------ .../tpch-approved-plan/v1-ras/spark34/1.txt | 12 +- .../tpch-approved-plan/v1-ras/spark34/10.txt | 42 +++---- .../tpch-approved-plan/v1-ras/spark34/11.txt | 48 ++++---- .../tpch-approved-plan/v1-ras/spark34/12.txt | 24 ++-- .../tpch-approved-plan/v1-ras/spark34/13.txt | 24 ++-- .../tpch-approved-plan/v1-ras/spark34/14.txt | 12 +- .../tpch-approved-plan/v1-ras/spark34/15.txt | 24 ++-- .../tpch-approved-plan/v1-ras/spark34/16.txt | 30 ++--- .../tpch-approved-plan/v1-ras/spark34/17.txt | 18 +-- .../tpch-approved-plan/v1-ras/spark34/18.txt | 36 +++--- .../tpch-approved-plan/v1-ras/spark34/19.txt | 12 +- .../tpch-approved-plan/v1-ras/spark34/20.txt | 60 +++++----- .../tpch-approved-plan/v1-ras/spark34/21.txt | 60 +++++----- .../tpch-approved-plan/v1-ras/spark34/22.txt | 30 ++--- .../tpch-approved-plan/v1-ras/spark34/3.txt | 24 ++-- .../tpch-approved-plan/v1-ras/spark34/4.txt | 24 ++-- .../tpch-approved-plan/v1-ras/spark34/5.txt | 72 ++++++------ .../tpch-approved-plan/v1-ras/spark34/6.txt | 6 +- .../tpch-approved-plan/v1-ras/spark34/7.txt | 66 +++++------ .../tpch-approved-plan/v1-ras/spark34/8.txt | 96 +++++++-------- .../tpch-approved-plan/v1-ras/spark34/9.txt | 72 ++++++------ .../tpch-approved-plan/v1/spark32/1.txt | 12 +- .../tpch-approved-plan/v1/spark32/10.txt | 42 +++---- .../tpch-approved-plan/v1/spark32/11.txt | 36 +++--- .../tpch-approved-plan/v1/spark32/12.txt | 24 ++-- .../tpch-approved-plan/v1/spark32/13.txt | 24 ++-- .../tpch-approved-plan/v1/spark32/14.txt | 12 +- .../tpch-approved-plan/v1/spark32/15.txt | 18 +-- .../tpch-approved-plan/v1/spark32/16.txt | 30 ++--- .../tpch-approved-plan/v1/spark32/17.txt | 18 +-- .../tpch-approved-plan/v1/spark32/18.txt | 36 +++--- .../tpch-approved-plan/v1/spark32/19.txt | 12 +- .../tpch-approved-plan/v1/spark32/20.txt | 60 +++++----- .../tpch-approved-plan/v1/spark32/21.txt | 60 +++++----- .../tpch-approved-plan/v1/spark32/22.txt | 24 ++-- .../tpch-approved-plan/v1/spark32/3.txt | 24 ++-- .../tpch-approved-plan/v1/spark32/4.txt | 24 ++-- .../tpch-approved-plan/v1/spark32/5.txt | 72 ++++++------ .../tpch-approved-plan/v1/spark32/6.txt | 6 +- .../tpch-approved-plan/v1/spark32/7.txt | 66 +++++------ .../tpch-approved-plan/v1/spark32/8.txt | 96 +++++++-------- .../tpch-approved-plan/v1/spark32/9.txt | 72 ++++++------ .../tpch-approved-plan/v1/spark33/1.txt | 12 +- .../tpch-approved-plan/v1/spark33/10.txt | 42 +++---- .../tpch-approved-plan/v1/spark33/11.txt | 48 ++++---- .../tpch-approved-plan/v1/spark33/12.txt | 24 ++-- .../tpch-approved-plan/v1/spark33/13.txt | 24 ++-- .../tpch-approved-plan/v1/spark33/14.txt | 12 +- .../tpch-approved-plan/v1/spark33/15.txt | 24 ++-- .../tpch-approved-plan/v1/spark33/16.txt | 30 ++--- .../tpch-approved-plan/v1/spark33/17.txt | 18 +-- .../tpch-approved-plan/v1/spark33/18.txt | 36 +++--- .../tpch-approved-plan/v1/spark33/19.txt | 12 +- .../tpch-approved-plan/v1/spark33/20.txt | 60 +++++----- .../tpch-approved-plan/v1/spark33/21.txt | 60 +++++----- .../tpch-approved-plan/v1/spark33/22.txt | 32 ++--- .../tpch-approved-plan/v1/spark33/3.txt | 24 ++-- .../tpch-approved-plan/v1/spark33/4.txt | 24 ++-- .../tpch-approved-plan/v1/spark33/5.txt | 72 ++++++------ .../tpch-approved-plan/v1/spark33/6.txt | 6 +- .../tpch-approved-plan/v1/spark33/7.txt | 66 +++++------ .../tpch-approved-plan/v1/spark33/8.txt | 96 +++++++-------- .../tpch-approved-plan/v1/spark33/9.txt | 72 ++++++------ .../tpch-approved-plan/v1/spark34/1.txt | 12 +- .../tpch-approved-plan/v1/spark34/10.txt | 42 +++---- .../tpch-approved-plan/v1/spark34/11.txt | 48 ++++---- .../tpch-approved-plan/v1/spark34/12.txt | 24 ++-- .../tpch-approved-plan/v1/spark34/13.txt | 24 ++-- .../tpch-approved-plan/v1/spark34/14.txt | 12 +- .../tpch-approved-plan/v1/spark34/15.txt | 24 ++-- .../tpch-approved-plan/v1/spark34/16.txt | 30 ++--- .../tpch-approved-plan/v1/spark34/17.txt | 18 +-- .../tpch-approved-plan/v1/spark34/18.txt | 36 +++--- .../tpch-approved-plan/v1/spark34/19.txt | 12 +- .../tpch-approved-plan/v1/spark34/20.txt | 60 +++++----- .../tpch-approved-plan/v1/spark34/21.txt | 60 +++++----- .../tpch-approved-plan/v1/spark34/22.txt | 32 ++--- .../tpch-approved-plan/v1/spark34/3.txt | 24 ++-- .../tpch-approved-plan/v1/spark34/4.txt | 24 ++-- .../tpch-approved-plan/v1/spark34/5.txt | 72 ++++++------ .../tpch-approved-plan/v1/spark34/6.txt | 6 +- .../tpch-approved-plan/v1/spark34/7.txt | 66 +++++------ .../tpch-approved-plan/v1/spark34/8.txt | 96 +++++++-------- .../tpch-approved-plan/v1/spark34/9.txt | 72 ++++++------ .../gluten/execution/TestOperator.scala | 33 +++++- .../gluten/execution/VeloxTPCHSuite.scala | 1 + cpp/velox/CMakeLists.txt | 2 +- cpp/velox/jni/VeloxJniWrapper.cc | 7 +- cpp/velox/utils/VeloxBatchAppender.cc | 59 ---------- cpp/velox/utils/VeloxBatchResizer.cc | 111 ++++++++++++++++++ ...loxBatchAppender.h => VeloxBatchResizer.h} | 8 +- .../GlutenQueryExecutionErrorsSuite.scala | 2 +- .../org/apache/gluten/GlutenConfig.scala | 50 +++++--- 266 files changed, 3287 insertions(+), 3175 deletions(-) rename backends-velox/src/main/java/org/apache/gluten/utils/{VeloxBatchAppender.java => VeloxBatchResizer.java} (82%) rename backends-velox/src/main/java/org/apache/gluten/utils/{VeloxBatchAppenderJniWrapper.java => VeloxBatchResizerJniWrapper.java} (75%) rename backends-velox/src/main/scala/org/apache/gluten/execution/{VeloxAppendBatchesExec.scala => VeloxResizeBatchesExec.scala} (86%) delete mode 100644 cpp/velox/utils/VeloxBatchAppender.cc create mode 100644 cpp/velox/utils/VeloxBatchResizer.cc rename cpp/velox/utils/{VeloxBatchAppender.h => VeloxBatchResizer.h} (87%) diff --git a/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchAppender.java b/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchResizer.java similarity index 82% rename from backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchAppender.java rename to backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchResizer.java index 32b2289471f93..e2f11cd3510b8 100644 --- a/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchAppender.java +++ b/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchResizer.java @@ -25,13 +25,13 @@ import java.util.Iterator; -public final class VeloxBatchAppender { +public final class VeloxBatchResizer { public static ColumnarBatchOutIterator create( - int minOutputBatchSize, Iterator in) { - final Runtime runtime = Runtimes.contextInstance("VeloxBatchAppender"); + int minOutputBatchSize, int maxOutputBatchSize, Iterator in) { + final Runtime runtime = Runtimes.contextInstance("VeloxBatchResizer"); long outHandle = - VeloxBatchAppenderJniWrapper.create(runtime) - .create(minOutputBatchSize, new ColumnarBatchInIterator(in)); + VeloxBatchResizerJniWrapper.create(runtime) + .create(minOutputBatchSize, maxOutputBatchSize, new ColumnarBatchInIterator(in)); return new ColumnarBatchOutIterator(runtime, outHandle); } } diff --git a/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchAppenderJniWrapper.java b/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchResizerJniWrapper.java similarity index 75% rename from backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchAppenderJniWrapper.java rename to backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchResizerJniWrapper.java index 231e655539813..3011ced2ab379 100644 --- a/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchAppenderJniWrapper.java +++ b/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchResizerJniWrapper.java @@ -20,15 +20,15 @@ import org.apache.gluten.exec.RuntimeAware; import org.apache.gluten.vectorized.ColumnarBatchInIterator; -public class VeloxBatchAppenderJniWrapper implements RuntimeAware { +public class VeloxBatchResizerJniWrapper implements RuntimeAware { private final Runtime runtime; - private VeloxBatchAppenderJniWrapper(Runtime runtime) { + private VeloxBatchResizerJniWrapper(Runtime runtime) { this.runtime = runtime; } - public static VeloxBatchAppenderJniWrapper create(Runtime runtime) { - return new VeloxBatchAppenderJniWrapper(runtime); + public static VeloxBatchResizerJniWrapper create(Runtime runtime) { + return new VeloxBatchResizerJniWrapper(runtime); } @Override @@ -36,5 +36,6 @@ public long handle() { return runtime.getHandle(); } - public native long create(int minOutputBatchSize, ColumnarBatchInIterator itr); + public native long create( + int minOutputBatchSize, int maxOutputBatchSize, ColumnarBatchInIterator itr); } diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala index a32b00196aeb9..9e0d85e25a362 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala @@ -352,9 +352,10 @@ class VeloxSparkPlanExecApi extends SparkPlanExecApi { plan match { case shuffle: ColumnarShuffleExchangeExec if !shuffle.useSortBasedShuffle && - GlutenConfig.getConf.veloxCoalesceBatchesBeforeShuffle => + GlutenConfig.getConf.veloxResizeBatchesShuffleInput => + val range = GlutenConfig.getConf.veloxResizeBatchesShuffleInputRange val appendBatches = - VeloxAppendBatchesExec(shuffle.child, GlutenConfig.getConf.veloxMinBatchSizeForShuffle) + VeloxResizeBatchesExec(shuffle.child, range.min, range.max) shuffle.withNewChildren(Seq(appendBatches)) case _ => plan } diff --git a/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxAppendBatchesExec.scala b/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxResizeBatchesExec.scala similarity index 86% rename from backends-velox/src/main/scala/org/apache/gluten/execution/VeloxAppendBatchesExec.scala rename to backends-velox/src/main/scala/org/apache/gluten/execution/VeloxResizeBatchesExec.scala index 4b4db703de7a4..d05e68f857db7 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxAppendBatchesExec.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxResizeBatchesExec.scala @@ -17,7 +17,7 @@ package org.apache.gluten.execution import org.apache.gluten.extension.GlutenPlan -import org.apache.gluten.utils.VeloxBatchAppender +import org.apache.gluten.utils.VeloxBatchResizer import org.apache.gluten.utils.iterator.Iterators import org.apache.spark.rdd.RDD @@ -33,10 +33,13 @@ import java.util.concurrent.atomic.AtomicLong import scala.collection.JavaConverters._ /** - * An operator to coalesce input batches by appending the later batches to the one that comes - * earlier. + * An operator to resize input batches by appending the later batches to the one that comes earlier, + * or splitting one batch to smaller ones. */ -case class VeloxAppendBatchesExec(override val child: SparkPlan, minOutputBatchSize: Int) +case class VeloxResizeBatchesExec( + override val child: SparkPlan, + minOutputBatchSize: Int, + maxOutputBatchSize: Int) extends GlutenPlan with UnaryExecNode { @@ -45,7 +48,7 @@ case class VeloxAppendBatchesExec(override val child: SparkPlan, minOutputBatchS "numInputBatches" -> SQLMetrics.createMetric(sparkContext, "number of input batches"), "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), "numOutputBatches" -> SQLMetrics.createMetric(sparkContext, "number of output batches"), - "appendTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to append batches") + "selfTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to append / split batches") ) override def supportsColumnar: Boolean = true @@ -56,15 +59,15 @@ case class VeloxAppendBatchesExec(override val child: SparkPlan, minOutputBatchS val numInputBatches = longMetric("numInputBatches") val numOutputRows = longMetric("numOutputRows") val numOutputBatches = longMetric("numOutputBatches") - val appendTime = longMetric("appendTime") + val selfTime = longMetric("selfTime") child.executeColumnar().mapPartitions { in => // Append millis = Out millis - In millis. val appendMillis = new AtomicLong(0L) - - val appender = VeloxBatchAppender.create( + val appender = VeloxBatchResizer.create( minOutputBatchSize, + maxOutputBatchSize, Iterators .wrap(in) .collectReadMillis(inMillis => appendMillis.getAndAdd(-inMillis)) @@ -84,7 +87,7 @@ case class VeloxAppendBatchesExec(override val child: SparkPlan, minOutputBatchS .recyclePayload(_.close()) .recycleIterator { appender.close() - appendTime += appendMillis.get() + selfTime += appendMillis.get() } .create() .map { diff --git a/backends-velox/src/main/scala/org/apache/gluten/extension/FlushableHashAggregateRule.scala b/backends-velox/src/main/scala/org/apache/gluten/extension/FlushableHashAggregateRule.scala index 777bf553856ed..3137d6e6aef54 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/extension/FlushableHashAggregateRule.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/extension/FlushableHashAggregateRule.scala @@ -77,7 +77,7 @@ case class FlushableHashAggregateRule(session: SparkSession) extends Rule[SparkP private def canPropagate(plan: SparkPlan): Boolean = plan match { case _: ProjectExecTransformer => true - case _: VeloxAppendBatchesExec => true + case _: VeloxResizeBatchesExec => true case _ => false } } diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/1.txt index 53edb933c1fb4..deb09c6c1c0f6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/1.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (30) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ RegularHashAggregateExecTransformer (12) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ ProjectExecTransformer (5) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) @@ -58,9 +58,9 @@ Input [17]: [l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] @@ -87,9 +87,9 @@ Results [10]: [l_returnflag#X, l_linestatus#X, sum(l_quantity#X)#X AS sum_qty#X, Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/10.txt index 29f28ac26a6be..2b4b7266faa67 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/10.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (68) +- ^ InputIteratorTransformer (39) +- ShuffleQueryStage (37) +- ColumnarExchange (36) - +- VeloxAppendBatches (35) + +- VeloxResizeBatches (35) +- ^ ProjectExecTransformer (33) +- ^ FlushableHashAggregateExecTransformer (32) +- ^ ProjectExecTransformer (31) @@ -210,9 +210,9 @@ Input [9]: [c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] Arguments: false -(35) VeloxAppendBatches +(35) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (36) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/11.txt index b8f466eb9456c..356e7b09de8ef 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/11.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (60) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ FilterExecTransformer (31) +- ^ RegularHashAggregateExecTransformer (30) +- ^ InputIteratorTransformer (29) +- ShuffleQueryStage (27) +- ColumnarExchange (26) - +- VeloxAppendBatches (25) + +- VeloxResizeBatches (25) +- ^ ProjectExecTransformer (23) +- ^ FlushableHashAggregateExecTransformer (22) +- ^ ProjectExecTransformer (21) @@ -160,9 +160,9 @@ Input [3]: [ps_partkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] Arguments: false -(25) VeloxAppendBatches +(25) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (26) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] @@ -193,9 +193,9 @@ Arguments: (isnotnull(value#X) AND (cast(value#X as decimal(38,6)) > Subquery su Input [2]: [ps_partkey#X, value#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [2]: [ps_partkey#X, value#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [2]: [ps_partkey#X, value#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/12.txt index 1e60a93910a7b..824d4e33aeb72 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/12.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (44) +- ^ InputIteratorTransformer (27) +- ShuffleQueryStage (25) +- ColumnarExchange (24) - +- VeloxAppendBatches (23) + +- VeloxResizeBatches (23) +- ^ RegularHashAggregateExecTransformer (21) +- ^ InputIteratorTransformer (20) +- ShuffleQueryStage (18) +- ColumnarExchange (17) - +- VeloxAppendBatches (16) + +- VeloxResizeBatches (16) +- ^ ProjectExecTransformer (14) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) @@ -108,9 +108,9 @@ Input [3]: [l_shipmode#X, sum#X, sum#X] Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] Arguments: false -(16) VeloxAppendBatches +(16) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] -Arguments: X +Arguments: X, X (17) ColumnarExchange Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] @@ -137,9 +137,9 @@ Results [3]: [l_shipmode#X, sum(CASE WHEN ((o_orderpriority#X = 1-URGENT) OR (o_ Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/13.txt index c525944e12623..0387f9bdbd1c2 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/13.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (36) +- ShuffleQueryStage (34) +- ColumnarExchange (33) - +- VeloxAppendBatches (32) + +- VeloxResizeBatches (32) +- ^ RegularHashAggregateExecTransformer (30) +- ^ InputIteratorTransformer (29) +- ShuffleQueryStage (27) +- ColumnarExchange (26) - +- VeloxAppendBatches (25) + +- VeloxResizeBatches (25) +- ^ ProjectExecTransformer (23) +- ^ FlushableHashAggregateExecTransformer (22) +- ^ ProjectExecTransformer (21) @@ -19,7 +19,7 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ ProjectExecTransformer (13) +- ^ FlushableHashAggregateExecTransformer (12) +- ^ ProjectExecTransformer (11) @@ -112,9 +112,9 @@ Input [2]: [c_custkey#X, count#X] Input [3]: [hash_partition_key#X, c_custkey#X, count#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, count#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, count#X] @@ -156,9 +156,9 @@ Input [2]: [c_count#X, count#X] Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(25) VeloxAppendBatches +(25) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] -Arguments: X +Arguments: X, X (26) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] @@ -185,9 +185,9 @@ Results [2]: [c_count#X, count(1)#X AS custdist#X] Input [2]: [c_count#X, custdist#X] Arguments: false -(32) VeloxAppendBatches +(32) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] -Arguments: X +Arguments: X, X (33) ColumnarExchange Input [2]: [c_count#X, custdist#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/14.txt index fbfcf07030ebf..354637343992f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/14.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (35) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) @@ -97,9 +97,9 @@ Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/15.txt index b5073901c6206..2e51e95f07611 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/15.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (46) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ BroadcastHashJoinExecTransformer Inner BuildLeft (21) :- ^ InputIteratorTransformer (7) @@ -19,7 +19,7 @@ AdaptiveSparkPlan (46) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) @@ -101,9 +101,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] @@ -143,9 +143,9 @@ Input [6]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, supplier_no#X, total_ Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/16.txt index f3cef927551c5..5f255fdaee88f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/16.txt @@ -6,19 +6,19 @@ AdaptiveSparkPlan (59) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ RegularHashAggregateExecTransformer (20) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ ProjectExecTransformer (13) +- ^ FlushableHashAggregateExecTransformer (12) +- ^ ProjectExecTransformer (11) @@ -117,9 +117,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] @@ -157,9 +157,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, count#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] @@ -186,9 +186,9 @@ Results [4]: [p_brand#X, p_type#X, p_size#X, count(ps_suppkey#X)#X AS supplier_c Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/17.txt index 7c5359849d4e8..7f4e4e666c0d6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/17.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (36) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ FlushableHashAggregateExecTransformer (5) +- ^ InputIteratorTransformer (4) +- RowToVeloxColumnar (2) @@ -59,9 +59,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/18.txt index a0a6e26a6ac3c..6ee709d6b5a64 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/18.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (88) +- ^ InputIteratorTransformer (51) +- ShuffleQueryStage (49) +- ColumnarExchange (48) - +- VeloxAppendBatches (47) + +- VeloxResizeBatches (47) +- ^ ProjectExecTransformer (45) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) @@ -31,7 +31,7 @@ AdaptiveSparkPlan (88) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FlushableHashAggregateExecTransformer (11) : +- ^ Scan parquet (10) @@ -140,9 +140,9 @@ Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] @@ -276,9 +276,9 @@ Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] Arguments: false -(47) VeloxAppendBatches +(47) VeloxResizeBatches Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (48) ColumnarExchange Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/19.txt index d2ab9979f8a37..0555720ef0828 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/19.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (34) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) @@ -96,9 +96,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/20.txt index a1f1bb51cb989..fef19a90199b2 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/20.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (112) +- ^ InputIteratorTransformer (70) +- ShuffleQueryStage (68) +- ColumnarExchange (67) - +- VeloxAppendBatches (66) + +- VeloxResizeBatches (66) +- ^ ProjectExecTransformer (64) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (63) :- ^ ProjectExecTransformer (54) @@ -15,7 +15,7 @@ AdaptiveSparkPlan (112) : : +- AQEShuffleRead (8) : : +- ShuffleQueryStage (7) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) @@ -42,7 +42,7 @@ AdaptiveSparkPlan (112) : +- ^ InputIteratorTransformer (42) : +- ShuffleQueryStage (40) : +- ColumnarExchange (39) - : +- VeloxAppendBatches (38) + : +- VeloxResizeBatches (38) : +- ^ ProjectExecTransformer (36) : +- ^ FlushableHashAggregateExecTransformer (35) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (34) @@ -118,9 +118,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] @@ -255,9 +255,9 @@ Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(38) VeloxAppendBatches +(38) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (39) ColumnarExchange Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] @@ -370,9 +370,9 @@ Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] Input [2]: [s_name#X, s_address#X] Arguments: false -(66) VeloxAppendBatches +(66) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] -Arguments: X +Arguments: X, X (67) ColumnarExchange Input [2]: [s_name#X, s_address#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/21.txt index eb3f64d3efccc..b4158e426d5c1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/21.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (93) +- ^ InputIteratorTransformer (56) +- ShuffleQueryStage (54) +- ColumnarExchange (53) - +- VeloxAppendBatches (52) + +- VeloxResizeBatches (52) +- ^ ProjectExecTransformer (50) +- ^ FlushableHashAggregateExecTransformer (49) +- ^ ProjectExecTransformer (48) @@ -302,9 +302,9 @@ Input [2]: [s_name#X, count#X] Input [3]: [hash_partition_key#X, s_name#X, count#X] Arguments: false -(52) VeloxAppendBatches +(52) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, count#X] -Arguments: X +Arguments: X, X (53) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/22.txt index 4c149d9bb43ed..9db514620fa3c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/22.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (40) +- ^ InputIteratorTransformer (25) +- ShuffleQueryStage (23) +- ColumnarExchange (22) - +- VeloxAppendBatches (21) + +- VeloxResizeBatches (21) +- ^ RegularHashAggregateExecTransformer (19) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) @@ -95,9 +95,9 @@ Input [4]: [cntrycode#X, count#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] @@ -124,9 +124,9 @@ Results [3]: [cntrycode#X, count(1)#X AS numcust#X, sum(c_acctbal#X)#X AS totacc Input [3]: [cntrycode#X, numcust#X, totacctbal#X] Arguments: false -(21) VeloxAppendBatches +(21) VeloxResizeBatches Input [3]: [cntrycode#X, numcust#X, totacctbal#X] -Arguments: X +Arguments: X, X (22) ColumnarExchange Input [3]: [cntrycode#X, numcust#X, totacctbal#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/3.txt index df2a6de8d1846..e9e3670db8427 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/3.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (54) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ ProjectExecTransformer (22) @@ -160,9 +160,9 @@ Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/4.txt index 34cb6467cbc2f..0195ac2721596 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/4.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (46) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ RegularHashAggregateExecTransformer (22) +- ^ InputIteratorTransformer (21) +- ShuffleQueryStage (19) +- ColumnarExchange (18) - +- VeloxAppendBatches (17) + +- VeloxResizeBatches (17) +- ^ ProjectExecTransformer (15) +- ^ FlushableHashAggregateExecTransformer (14) +- ^ ProjectExecTransformer (13) @@ -114,9 +114,9 @@ Input [2]: [o_orderpriority#X, count#X] Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] Arguments: false -(17) VeloxAppendBatches +(17) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] -Arguments: X +Arguments: X, X (18) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] @@ -143,9 +143,9 @@ Results [2]: [o_orderpriority#X, count(1)#X AS order_count#X] Input [2]: [o_orderpriority#X, order_count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [2]: [o_orderpriority#X, order_count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [2]: [o_orderpriority#X, order_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/5.txt index 2c9ba4c938d4c..277d203a869c1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/5.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (102) +- ^ InputIteratorTransformer (64) +- ShuffleQueryStage (62) +- ColumnarExchange (61) - +- VeloxAppendBatches (60) + +- VeloxResizeBatches (60) +- ^ RegularHashAggregateExecTransformer (58) +- ^ InputIteratorTransformer (57) +- ShuffleQueryStage (55) +- ColumnarExchange (54) - +- VeloxAppendBatches (53) + +- VeloxResizeBatches (53) +- ^ ProjectExecTransformer (51) +- ^ FlushableHashAggregateExecTransformer (50) +- ^ ProjectExecTransformer (49) @@ -314,9 +314,9 @@ Input [3]: [n_name#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] Arguments: false -(53) VeloxAppendBatches +(53) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (54) ColumnarExchange Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] @@ -343,9 +343,9 @@ Results [2]: [n_name#X, sum(CheckOverflow((promote_precision(cast(l_extendedpric Input [2]: [n_name#X, revenue#X] Arguments: false -(60) VeloxAppendBatches +(60) VeloxResizeBatches Input [2]: [n_name#X, revenue#X] -Arguments: X +Arguments: X, X (61) ColumnarExchange Input [2]: [n_name#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/6.txt index b6f876d48e5a0..f30bcb0da8014 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/6.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (20) +- ^ InputIteratorTransformer (10) +- ShuffleQueryStage (8) +- ColumnarExchange (7) - +- VeloxAppendBatches (6) + +- VeloxResizeBatches (6) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ NoopFilter (2) @@ -46,9 +46,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(6) VeloxAppendBatches +(6) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (7) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/7.txt index 8cf20f7694c9e..35de09c78d482 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/7.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (95) +- ^ InputIteratorTransformer (59) +- ShuffleQueryStage (57) +- ColumnarExchange (56) - +- VeloxAppendBatches (55) + +- VeloxResizeBatches (55) +- ^ RegularHashAggregateExecTransformer (53) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50) +- ColumnarExchange (49) - +- VeloxAppendBatches (48) + +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ FlushableHashAggregateExecTransformer (45) +- ^ ProjectExecTransformer (44) @@ -284,9 +284,9 @@ Input [5]: [supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] @@ -313,9 +313,9 @@ Results [4]: [supp_nation#X, cust_nation#X, l_year#X, sum(volume#X)#X AS revenue Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] Arguments: false -(55) VeloxAppendBatches +(55) VeloxResizeBatches Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] -Arguments: X +Arguments: X, X (56) ColumnarExchange Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/8.txt index 045e283bd036c..e7fd9789b9a3c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/8.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (131) +- ^ InputIteratorTransformer (83) +- ShuffleQueryStage (81) +- ColumnarExchange (80) - +- VeloxAppendBatches (79) + +- VeloxResizeBatches (79) +- ^ ProjectExecTransformer (77) +- ^ RegularHashAggregateExecTransformer (76) +- ^ InputIteratorTransformer (75) +- ShuffleQueryStage (73) +- ColumnarExchange (72) - +- VeloxAppendBatches (71) + +- VeloxResizeBatches (71) +- ^ ProjectExecTransformer (69) +- ^ FlushableHashAggregateExecTransformer (68) +- ^ ProjectExecTransformer (67) @@ -415,9 +415,9 @@ Input [5]: [o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(71) VeloxAppendBatches +(71) VeloxResizeBatches Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (72) ColumnarExchange Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] @@ -448,9 +448,9 @@ Input [3]: [o_year#X, sum(CASE WHEN (nation#X = BRAZIL) THEN volume#X ELSE 0.000 Input [2]: [o_year#X, mkt_share#X] Arguments: false -(79) VeloxAppendBatches +(79) VeloxResizeBatches Input [2]: [o_year#X, mkt_share#X] -Arguments: X +Arguments: X, X (80) ColumnarExchange Input [2]: [o_year#X, mkt_share#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/9.txt index abdd1d7b9369a..813cf616f41c8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/9.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (100) +- ^ InputIteratorTransformer (63) +- ShuffleQueryStage (61) +- ColumnarExchange (60) - +- VeloxAppendBatches (59) + +- VeloxResizeBatches (59) +- ^ RegularHashAggregateExecTransformer (57) +- ^ InputIteratorTransformer (56) +- ShuffleQueryStage (54) +- ColumnarExchange (53) - +- VeloxAppendBatches (52) + +- VeloxResizeBatches (52) +- ^ ProjectExecTransformer (50) +- ^ FlushableHashAggregateExecTransformer (49) +- ^ ProjectExecTransformer (48) @@ -308,9 +308,9 @@ Input [4]: [nation#X, o_year#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] Arguments: false -(52) VeloxAppendBatches +(52) VeloxResizeBatches Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (53) ColumnarExchange Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] @@ -337,9 +337,9 @@ Results [3]: [nation#X, o_year#X, sum(amount#X)#X AS sum_profit#X] Input [3]: [nation#X, o_year#X, sum_profit#X] Arguments: false -(59) VeloxAppendBatches +(59) VeloxResizeBatches Input [3]: [nation#X, o_year#X, sum_profit#X] -Arguments: X +Arguments: X, X (60) ColumnarExchange Input [3]: [nation#X, o_year#X, sum_profit#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/1.txt index afea15af53d24..159a1598c9bf5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/1.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (30) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ RegularHashAggregateExecTransformer (12) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9), Statistics(X) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ ProjectExecTransformer (5) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) @@ -58,9 +58,9 @@ Input [17]: [l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] @@ -87,9 +87,9 @@ Results [10]: [l_returnflag#X, l_linestatus#X, sum(l_quantity#X)#X AS sum_qty#X, Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/10.txt index 7a509e435d075..2d91fcf6299e5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/10.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (68) +- ^ InputIteratorTransformer (39) +- ShuffleQueryStage (37), Statistics(X) +- ColumnarExchange (36) - +- VeloxAppendBatches (35) + +- VeloxResizeBatches (35) +- ^ ProjectExecTransformer (33) +- ^ FlushableHashAggregateExecTransformer (32) +- ^ ProjectExecTransformer (31) @@ -210,9 +210,9 @@ Input [9]: [c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] Arguments: false -(35) VeloxAppendBatches +(35) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (36) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/11.txt index 1c4e102aa0b98..b901a5db58150 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/11.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (60) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35), Statistics(X) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ FilterExecTransformer (31) +- ^ RegularHashAggregateExecTransformer (30) +- ^ InputIteratorTransformer (29) +- ShuffleQueryStage (27), Statistics(X) +- ColumnarExchange (26) - +- VeloxAppendBatches (25) + +- VeloxResizeBatches (25) +- ^ ProjectExecTransformer (23) +- ^ FlushableHashAggregateExecTransformer (22) +- ^ ProjectExecTransformer (21) @@ -160,9 +160,9 @@ Input [3]: [ps_partkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] Arguments: false -(25) VeloxAppendBatches +(25) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (26) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] @@ -193,9 +193,9 @@ Arguments: (isnotnull(value#X) AND (cast(value#X as decimal(38,6)) > Subquery su Input [2]: [ps_partkey#X, value#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [2]: [ps_partkey#X, value#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [2]: [ps_partkey#X, value#X] @@ -330,7 +330,7 @@ AdaptiveSparkPlan (102) +- ^ InputIteratorTransformer (81) +- ShuffleQueryStage (79), Statistics(X) +- ColumnarExchange (78) - +- VeloxAppendBatches (77) + +- VeloxResizeBatches (77) +- ^ FlushableHashAggregateExecTransformer (75) +- ^ ProjectExecTransformer (74) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (73) @@ -429,9 +429,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(77) VeloxAppendBatches +(77) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (78) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/12.txt index 12d0f6f0ff955..456b95e5f5ded 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/12.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (44) +- ^ InputIteratorTransformer (27) +- ShuffleQueryStage (25), Statistics(X) +- ColumnarExchange (24) - +- VeloxAppendBatches (23) + +- VeloxResizeBatches (23) +- ^ RegularHashAggregateExecTransformer (21) +- ^ InputIteratorTransformer (20) +- ShuffleQueryStage (18), Statistics(X) +- ColumnarExchange (17) - +- VeloxAppendBatches (16) + +- VeloxResizeBatches (16) +- ^ ProjectExecTransformer (14) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) @@ -108,9 +108,9 @@ Input [3]: [l_shipmode#X, sum#X, sum#X] Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] Arguments: false -(16) VeloxAppendBatches +(16) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] -Arguments: X +Arguments: X, X (17) ColumnarExchange Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] @@ -137,9 +137,9 @@ Results [3]: [l_shipmode#X, sum(CASE WHEN ((o_orderpriority#X = 1-URGENT) OR (o_ Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/13.txt index 1dfb80d78050a..6044e9fa526c3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/13.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (36) +- ShuffleQueryStage (34), Statistics(X) +- ColumnarExchange (33) - +- VeloxAppendBatches (32) + +- VeloxResizeBatches (32) +- ^ RegularHashAggregateExecTransformer (30) +- ^ InputIteratorTransformer (29) +- ShuffleQueryStage (27), Statistics(X) +- ColumnarExchange (26) - +- VeloxAppendBatches (25) + +- VeloxResizeBatches (25) +- ^ ProjectExecTransformer (23) +- ^ FlushableHashAggregateExecTransformer (22) +- ^ ProjectExecTransformer (21) @@ -19,7 +19,7 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ ProjectExecTransformer (13) +- ^ FlushableHashAggregateExecTransformer (12) +- ^ ProjectExecTransformer (11) @@ -112,9 +112,9 @@ Input [2]: [c_custkey#X, count#X] Input [3]: [hash_partition_key#X, c_custkey#X, count#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, count#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, count#X] @@ -156,9 +156,9 @@ Input [2]: [c_count#X, count#X] Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(25) VeloxAppendBatches +(25) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] -Arguments: X +Arguments: X, X (26) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] @@ -185,9 +185,9 @@ Results [2]: [c_count#X, count(1)#X AS custdist#X] Input [2]: [c_count#X, custdist#X] Arguments: false -(32) VeloxAppendBatches +(32) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] -Arguments: X +Arguments: X, X (33) ColumnarExchange Input [2]: [c_count#X, custdist#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/14.txt index 6f71859cba6d2..0f8e52cb2056e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/14.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (35) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) @@ -97,9 +97,9 @@ Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/15.txt index 40b342510de2b..dd611a9b29c8b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/15.txt @@ -5,7 +5,7 @@ AdaptiveSparkPlan (43) +- AQEShuffleRead (27) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ BroadcastHashJoinExecTransformer Inner BuildLeft (21) :- ^ InputIteratorTransformer (7) @@ -18,7 +18,7 @@ AdaptiveSparkPlan (43) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) @@ -100,9 +100,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] @@ -142,9 +142,9 @@ Input [6]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, supplier_no#X, total_ Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] @@ -247,7 +247,7 @@ AdaptiveSparkPlan (69) +- ^ InputIteratorTransformer (54) +- ShuffleQueryStage (52), Statistics(X) +- ColumnarExchange (51) - +- VeloxAppendBatches (50) + +- VeloxResizeBatches (50) +- ^ ProjectExecTransformer (48) +- ^ FlushableHashAggregateExecTransformer (47) +- ^ ProjectExecTransformer (46) @@ -294,9 +294,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(50) VeloxAppendBatches +(50) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (51) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/16.txt index d579c1a704e5e..13aa5f68019e0 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/16.txt @@ -6,19 +6,19 @@ AdaptiveSparkPlan (59) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ RegularHashAggregateExecTransformer (20) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ ProjectExecTransformer (13) +- ^ FlushableHashAggregateExecTransformer (12) +- ^ ProjectExecTransformer (11) @@ -117,9 +117,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] @@ -157,9 +157,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, count#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] @@ -186,9 +186,9 @@ Results [4]: [p_brand#X, p_type#X, p_size#X, count(ps_suppkey#X)#X AS supplier_c Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/17.txt index 1e2ed970aef4f..38801e9f690a8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/17.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (36) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9), Statistics(X) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ FlushableHashAggregateExecTransformer (5) +- ^ InputIteratorTransformer (4) +- RowToVeloxColumnar (2) @@ -59,9 +59,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/18.txt index 2e7ce455aebd0..62a2f7b0b48aa 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/18.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (88) +- ^ InputIteratorTransformer (51) +- ShuffleQueryStage (49), Statistics(X) +- ColumnarExchange (48) - +- VeloxAppendBatches (47) + +- VeloxResizeBatches (47) +- ^ ProjectExecTransformer (45) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) @@ -31,7 +31,7 @@ AdaptiveSparkPlan (88) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FlushableHashAggregateExecTransformer (11) : +- ^ Scan parquet (10) @@ -140,9 +140,9 @@ Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] @@ -276,9 +276,9 @@ Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] Arguments: false -(47) VeloxAppendBatches +(47) VeloxResizeBatches Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (48) ColumnarExchange Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/19.txt index 992c5328adbab..178fa021fb70e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/19.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (34) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) @@ -96,9 +96,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/20.txt index 4a899ae239be3..62a0f1a317b11 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/20.txt @@ -5,7 +5,7 @@ AdaptiveSparkPlan (109) +- AQEShuffleRead (69) +- ShuffleQueryStage (68), Statistics(X) +- ColumnarExchange (67) - +- VeloxAppendBatches (66) + +- VeloxResizeBatches (66) +- ^ ProjectExecTransformer (64) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (63) :- ^ ProjectExecTransformer (54) @@ -14,7 +14,7 @@ AdaptiveSparkPlan (109) : : +- AQEShuffleRead (8) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) @@ -41,7 +41,7 @@ AdaptiveSparkPlan (109) : +- ^ InputIteratorTransformer (42) : +- ShuffleQueryStage (40), Statistics(X) : +- ColumnarExchange (39) - : +- VeloxAppendBatches (38) + : +- VeloxResizeBatches (38) : +- ^ ProjectExecTransformer (36) : +- ^ FlushableHashAggregateExecTransformer (35) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (34) @@ -117,9 +117,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] @@ -254,9 +254,9 @@ Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(38) VeloxAppendBatches +(38) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (39) ColumnarExchange Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] @@ -369,9 +369,9 @@ Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] Input [2]: [s_name#X, s_address#X] Arguments: false -(66) VeloxAppendBatches +(66) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] -Arguments: X +Arguments: X, X (67) ColumnarExchange Input [2]: [s_name#X, s_address#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/21.txt index 7627c7f4f147b..045b4a38e3a3b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/21.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (92) +- ^ InputIteratorTransformer (56) +- ShuffleQueryStage (54), Statistics(X) +- ColumnarExchange (53) - +- VeloxAppendBatches (52) + +- VeloxResizeBatches (52) +- ^ ProjectExecTransformer (50) +- ^ FlushableHashAggregateExecTransformer (49) +- ^ ProjectExecTransformer (48) @@ -301,9 +301,9 @@ Input [2]: [s_name#X, count#X] Input [3]: [hash_partition_key#X, s_name#X, count#X] Arguments: false -(52) VeloxAppendBatches +(52) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, count#X] -Arguments: X +Arguments: X, X (53) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/22.txt index 9a0475d25d78e..829be008d8436 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/22.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (40) +- ^ InputIteratorTransformer (25) +- ShuffleQueryStage (23), Statistics(X) +- ColumnarExchange (22) - +- VeloxAppendBatches (21) + +- VeloxResizeBatches (21) +- ^ RegularHashAggregateExecTransformer (19) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) @@ -95,9 +95,9 @@ Input [4]: [cntrycode#X, count#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] @@ -124,9 +124,9 @@ Results [3]: [cntrycode#X, count(1)#X AS numcust#X, sum(c_acctbal#X)#X AS totacc Input [3]: [cntrycode#X, numcust#X, totacctbal#X] Arguments: false -(21) VeloxAppendBatches +(21) VeloxResizeBatches Input [3]: [cntrycode#X, numcust#X, totacctbal#X] -Arguments: X +Arguments: X, X (22) ColumnarExchange Input [3]: [cntrycode#X, numcust#X, totacctbal#X] @@ -223,7 +223,7 @@ AdaptiveSparkPlan (60) +- ^ InputIteratorTransformer (50) +- ShuffleQueryStage (48), Statistics(X) +- ColumnarExchange (47) - +- VeloxAppendBatches (46) + +- VeloxResizeBatches (46) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) +- ^ NoopFilter (42) @@ -263,9 +263,9 @@ Results [2]: [sum#X, count#X] Input [2]: [sum#X, count#X] Arguments: false -(46) VeloxAppendBatches +(46) VeloxResizeBatches Input [2]: [sum#X, count#X] -Arguments: X +Arguments: X, X (47) ColumnarExchange Input [2]: [sum#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/3.txt index 60843c0991ade..5ee5820d32d68 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/3.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (54) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ ProjectExecTransformer (22) @@ -160,9 +160,9 @@ Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/4.txt index b142ee1d8f33a..b9eb997a0cbbb 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/4.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (46) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ RegularHashAggregateExecTransformer (22) +- ^ InputIteratorTransformer (21) +- ShuffleQueryStage (19), Statistics(X) +- ColumnarExchange (18) - +- VeloxAppendBatches (17) + +- VeloxResizeBatches (17) +- ^ ProjectExecTransformer (15) +- ^ FlushableHashAggregateExecTransformer (14) +- ^ ProjectExecTransformer (13) @@ -114,9 +114,9 @@ Input [2]: [o_orderpriority#X, count#X] Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] Arguments: false -(17) VeloxAppendBatches +(17) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] -Arguments: X +Arguments: X, X (18) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] @@ -143,9 +143,9 @@ Results [2]: [o_orderpriority#X, count(1)#X AS order_count#X] Input [2]: [o_orderpriority#X, order_count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [2]: [o_orderpriority#X, order_count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [2]: [o_orderpriority#X, order_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/5.txt index c8a62c3aca1f4..a0d3ca93f02b1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/5.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (102) +- ^ InputIteratorTransformer (64) +- ShuffleQueryStage (62), Statistics(X) +- ColumnarExchange (61) - +- VeloxAppendBatches (60) + +- VeloxResizeBatches (60) +- ^ RegularHashAggregateExecTransformer (58) +- ^ InputIteratorTransformer (57) +- ShuffleQueryStage (55), Statistics(X) +- ColumnarExchange (54) - +- VeloxAppendBatches (53) + +- VeloxResizeBatches (53) +- ^ ProjectExecTransformer (51) +- ^ FlushableHashAggregateExecTransformer (50) +- ^ ProjectExecTransformer (49) @@ -314,9 +314,9 @@ Input [3]: [n_name#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] Arguments: false -(53) VeloxAppendBatches +(53) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (54) ColumnarExchange Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] @@ -343,9 +343,9 @@ Results [2]: [n_name#X, sum(CheckOverflow((promote_precision(cast(l_extendedpric Input [2]: [n_name#X, revenue#X] Arguments: false -(60) VeloxAppendBatches +(60) VeloxResizeBatches Input [2]: [n_name#X, revenue#X] -Arguments: X +Arguments: X, X (61) ColumnarExchange Input [2]: [n_name#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/6.txt index fa9c936a0ca11..1aaed506d7e08 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/6.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (20) +- ^ InputIteratorTransformer (10) +- ShuffleQueryStage (8), Statistics(X) +- ColumnarExchange (7) - +- VeloxAppendBatches (6) + +- VeloxResizeBatches (6) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ NoopFilter (2) @@ -46,9 +46,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(6) VeloxAppendBatches +(6) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (7) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/7.txt index f098b9ba837f5..8d95ec5e6f748 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/7.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (95) +- ^ InputIteratorTransformer (59) +- ShuffleQueryStage (57), Statistics(X) +- ColumnarExchange (56) - +- VeloxAppendBatches (55) + +- VeloxResizeBatches (55) +- ^ RegularHashAggregateExecTransformer (53) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50), Statistics(X) +- ColumnarExchange (49) - +- VeloxAppendBatches (48) + +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ FlushableHashAggregateExecTransformer (45) +- ^ ProjectExecTransformer (44) @@ -284,9 +284,9 @@ Input [5]: [supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] @@ -313,9 +313,9 @@ Results [4]: [supp_nation#X, cust_nation#X, l_year#X, sum(volume#X)#X AS revenue Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] Arguments: false -(55) VeloxAppendBatches +(55) VeloxResizeBatches Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] -Arguments: X +Arguments: X, X (56) ColumnarExchange Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/8.txt index becfc12a1b866..b74dc65358d2e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/8.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (131) +- ^ InputIteratorTransformer (83) +- ShuffleQueryStage (81), Statistics(X) +- ColumnarExchange (80) - +- VeloxAppendBatches (79) + +- VeloxResizeBatches (79) +- ^ ProjectExecTransformer (77) +- ^ RegularHashAggregateExecTransformer (76) +- ^ InputIteratorTransformer (75) +- ShuffleQueryStage (73), Statistics(X) +- ColumnarExchange (72) - +- VeloxAppendBatches (71) + +- VeloxResizeBatches (71) +- ^ ProjectExecTransformer (69) +- ^ FlushableHashAggregateExecTransformer (68) +- ^ ProjectExecTransformer (67) @@ -415,9 +415,9 @@ Input [5]: [o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(71) VeloxAppendBatches +(71) VeloxResizeBatches Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (72) ColumnarExchange Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] @@ -448,9 +448,9 @@ Input [3]: [o_year#X, sum(CASE WHEN (nation#X = BRAZIL) THEN volume#X ELSE 0.000 Input [2]: [o_year#X, mkt_share#X] Arguments: false -(79) VeloxAppendBatches +(79) VeloxResizeBatches Input [2]: [o_year#X, mkt_share#X] -Arguments: X +Arguments: X, X (80) ColumnarExchange Input [2]: [o_year#X, mkt_share#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/9.txt index a486fc65677b7..400fb12e1deaf 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/9.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (100) +- ^ InputIteratorTransformer (63) +- ShuffleQueryStage (61), Statistics(X) +- ColumnarExchange (60) - +- VeloxAppendBatches (59) + +- VeloxResizeBatches (59) +- ^ RegularHashAggregateExecTransformer (57) +- ^ InputIteratorTransformer (56) +- ShuffleQueryStage (54), Statistics(X) +- ColumnarExchange (53) - +- VeloxAppendBatches (52) + +- VeloxResizeBatches (52) +- ^ ProjectExecTransformer (50) +- ^ FlushableHashAggregateExecTransformer (49) +- ^ ProjectExecTransformer (48) @@ -308,9 +308,9 @@ Input [4]: [nation#X, o_year#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] Arguments: false -(52) VeloxAppendBatches +(52) VeloxResizeBatches Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (53) ColumnarExchange Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] @@ -337,9 +337,9 @@ Results [3]: [nation#X, o_year#X, sum(amount#X)#X AS sum_profit#X] Input [3]: [nation#X, o_year#X, sum_profit#X] Arguments: false -(59) VeloxAppendBatches +(59) VeloxResizeBatches Input [3]: [nation#X, o_year#X, sum_profit#X] -Arguments: X +Arguments: X, X (60) ColumnarExchange Input [3]: [nation#X, o_year#X, sum_profit#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/1.txt index 545f2e7e086df..2db104cfeb12e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/1.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (30) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ RegularHashAggregateExecTransformer (12) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9), Statistics(X) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ ProjectExecTransformer (5) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) @@ -58,9 +58,9 @@ Input [17]: [l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] @@ -87,9 +87,9 @@ Results [10]: [l_returnflag#X, l_linestatus#X, sum(l_quantity#X)#X AS sum_qty#X, Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/10.txt index 9325f007789b8..4c263cb4a280d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/10.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (68) +- ^ InputIteratorTransformer (39) +- ShuffleQueryStage (37), Statistics(X) +- ColumnarExchange (36) - +- VeloxAppendBatches (35) + +- VeloxResizeBatches (35) +- ^ ProjectExecTransformer (33) +- ^ FlushableHashAggregateExecTransformer (32) +- ^ ProjectExecTransformer (31) @@ -213,9 +213,9 @@ Input [9]: [c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] Arguments: false -(35) VeloxAppendBatches +(35) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (36) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/11.txt index c593374b9a001..83e60d925e400 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/11.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (60) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35), Statistics(X) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ FilterExecTransformer (31) +- ^ RegularHashAggregateExecTransformer (30) +- ^ InputIteratorTransformer (29) +- ShuffleQueryStage (27), Statistics(X) +- ColumnarExchange (26) - +- VeloxAppendBatches (25) + +- VeloxResizeBatches (25) +- ^ ProjectExecTransformer (23) +- ^ FlushableHashAggregateExecTransformer (22) +- ^ ProjectExecTransformer (21) @@ -162,9 +162,9 @@ Input [3]: [ps_partkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] Arguments: false -(25) VeloxAppendBatches +(25) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (26) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] @@ -195,9 +195,9 @@ Arguments: (isnotnull(value#X) AND (cast(value#X as decimal(38,6)) > Subquery su Input [2]: [ps_partkey#X, value#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [2]: [ps_partkey#X, value#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [2]: [ps_partkey#X, value#X] @@ -334,7 +334,7 @@ AdaptiveSparkPlan (102) +- ^ InputIteratorTransformer (81) +- ShuffleQueryStage (79), Statistics(X) +- ColumnarExchange (78) - +- VeloxAppendBatches (77) + +- VeloxResizeBatches (77) +- ^ FlushableHashAggregateExecTransformer (75) +- ^ ProjectExecTransformer (74) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (73) @@ -435,9 +435,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(77) VeloxAppendBatches +(77) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (78) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/12.txt index 93f3a4cf9ab77..354e97da2066a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/12.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (44) +- ^ InputIteratorTransformer (27) +- ShuffleQueryStage (25), Statistics(X) +- ColumnarExchange (24) - +- VeloxAppendBatches (23) + +- VeloxResizeBatches (23) +- ^ RegularHashAggregateExecTransformer (21) +- ^ InputIteratorTransformer (20) +- ShuffleQueryStage (18), Statistics(X) +- ColumnarExchange (17) - +- VeloxAppendBatches (16) + +- VeloxResizeBatches (16) +- ^ ProjectExecTransformer (14) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) @@ -109,9 +109,9 @@ Input [3]: [l_shipmode#X, sum#X, sum#X] Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] Arguments: false -(16) VeloxAppendBatches +(16) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] -Arguments: X +Arguments: X, X (17) ColumnarExchange Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] @@ -138,9 +138,9 @@ Results [3]: [l_shipmode#X, sum(CASE WHEN ((o_orderpriority#X = 1-URGENT) OR (o_ Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/13.txt index e44b10042401e..614098d876a41 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/13.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (36) +- ShuffleQueryStage (34), Statistics(X) +- ColumnarExchange (33) - +- VeloxAppendBatches (32) + +- VeloxResizeBatches (32) +- ^ RegularHashAggregateExecTransformer (30) +- ^ InputIteratorTransformer (29) +- ShuffleQueryStage (27), Statistics(X) +- ColumnarExchange (26) - +- VeloxAppendBatches (25) + +- VeloxResizeBatches (25) +- ^ ProjectExecTransformer (23) +- ^ FlushableHashAggregateExecTransformer (22) +- ^ ProjectExecTransformer (21) @@ -19,7 +19,7 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ ProjectExecTransformer (13) +- ^ FlushableHashAggregateExecTransformer (12) +- ^ ProjectExecTransformer (11) @@ -113,9 +113,9 @@ Input [2]: [c_custkey#X, count#X] Input [3]: [hash_partition_key#X, c_custkey#X, count#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, count#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, count#X] @@ -157,9 +157,9 @@ Input [2]: [c_count#X, count#X] Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(25) VeloxAppendBatches +(25) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] -Arguments: X +Arguments: X, X (26) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] @@ -186,9 +186,9 @@ Results [2]: [c_count#X, count(1)#X AS custdist#X] Input [2]: [c_count#X, custdist#X] Arguments: false -(32) VeloxAppendBatches +(32) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] -Arguments: X +Arguments: X, X (33) ColumnarExchange Input [2]: [c_count#X, custdist#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/14.txt index 8f5ae0f52e634..df48e66cf0ab1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/14.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (35) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) @@ -98,9 +98,9 @@ Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/15.txt index d616981e57d21..dfdcc6a1d8c76 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/15.txt @@ -5,7 +5,7 @@ AdaptiveSparkPlan (43) +- AQEShuffleRead (27) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ BroadcastHashJoinExecTransformer Inner BuildLeft (21) :- ^ InputIteratorTransformer (7) @@ -18,7 +18,7 @@ AdaptiveSparkPlan (43) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) @@ -100,9 +100,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] @@ -143,9 +143,9 @@ Input [6]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, supplier_no#X, total_ Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] @@ -249,7 +249,7 @@ AdaptiveSparkPlan (69) +- ^ InputIteratorTransformer (54) +- ShuffleQueryStage (52), Statistics(X) +- ColumnarExchange (51) - +- VeloxAppendBatches (50) + +- VeloxResizeBatches (50) +- ^ ProjectExecTransformer (48) +- ^ FlushableHashAggregateExecTransformer (47) +- ^ ProjectExecTransformer (46) @@ -296,9 +296,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(50) VeloxAppendBatches +(50) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (51) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/16.txt index 3b4da47fc25f4..73bf50f8f6f06 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/16.txt @@ -6,19 +6,19 @@ AdaptiveSparkPlan (59) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ RegularHashAggregateExecTransformer (20) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ ProjectExecTransformer (13) +- ^ FlushableHashAggregateExecTransformer (12) +- ^ ProjectExecTransformer (11) @@ -118,9 +118,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] @@ -158,9 +158,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, count#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] @@ -187,9 +187,9 @@ Results [4]: [p_brand#X, p_type#X, p_size#X, count(ps_suppkey#X)#X AS supplier_c Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/17.txt index 77733ad6f8a5b..77c18798faa24 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/17.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (36) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9), Statistics(X) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ FlushableHashAggregateExecTransformer (5) +- ^ InputIteratorTransformer (4) +- RowToVeloxColumnar (2) @@ -59,9 +59,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/18.txt index 0f5e82ec54c16..db32a9be29f10 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/18.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (88) +- ^ InputIteratorTransformer (51) +- ShuffleQueryStage (49), Statistics(X) +- ColumnarExchange (48) - +- VeloxAppendBatches (47) + +- VeloxResizeBatches (47) +- ^ ProjectExecTransformer (45) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) @@ -31,7 +31,7 @@ AdaptiveSparkPlan (88) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FlushableHashAggregateExecTransformer (11) : +- ^ Scan parquet (10) @@ -140,9 +140,9 @@ Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] @@ -280,9 +280,9 @@ Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] Arguments: false -(47) VeloxAppendBatches +(47) VeloxResizeBatches Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (48) ColumnarExchange Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/19.txt index 3bafdb994153b..0a48f3fd81173 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/19.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (34) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) @@ -97,9 +97,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/20.txt index 945cc79de1ac2..d54ea7dc1c138 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/20.txt @@ -5,7 +5,7 @@ AdaptiveSparkPlan (98) +- AQEShuffleRead (61) +- ShuffleQueryStage (60), Statistics(X) +- ColumnarExchange (59) - +- VeloxAppendBatches (58) + +- VeloxResizeBatches (58) +- ^ ProjectExecTransformer (56) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (55) :- ^ ProjectExecTransformer (46) @@ -35,7 +35,7 @@ AdaptiveSparkPlan (98) : +- ^ InputIteratorTransformer (34) : +- ShuffleQueryStage (32), Statistics(X) : +- ColumnarExchange (31) - : +- VeloxAppendBatches (30) + : +- VeloxResizeBatches (30) : +- ^ ProjectExecTransformer (28) : +- ^ FlushableHashAggregateExecTransformer (27) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (26) @@ -217,9 +217,9 @@ Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(30) VeloxAppendBatches +(30) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (31) ColumnarExchange Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] @@ -335,9 +335,9 @@ Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] Input [2]: [s_name#X, s_address#X] Arguments: false -(58) VeloxAppendBatches +(58) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] -Arguments: X +Arguments: X, X (59) ColumnarExchange Input [2]: [s_name#X, s_address#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/21.txt index a7d9f42063a09..162a7db4d2127 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/21.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (92) +- ^ InputIteratorTransformer (56) +- ShuffleQueryStage (54), Statistics(X) +- ColumnarExchange (53) - +- VeloxAppendBatches (52) + +- VeloxResizeBatches (52) +- ^ ProjectExecTransformer (50) +- ^ FlushableHashAggregateExecTransformer (49) +- ^ ProjectExecTransformer (48) @@ -306,9 +306,9 @@ Input [2]: [s_name#X, count#X] Input [3]: [hash_partition_key#X, s_name#X, count#X] Arguments: false -(52) VeloxAppendBatches +(52) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, count#X] -Arguments: X +Arguments: X, X (53) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/22.txt index 0d9139bcd990e..b1ed2c89b6eb7 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/22.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (40) +- ^ InputIteratorTransformer (25) +- ShuffleQueryStage (23), Statistics(X) +- ColumnarExchange (22) - +- VeloxAppendBatches (21) + +- VeloxResizeBatches (21) +- ^ RegularHashAggregateExecTransformer (19) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) @@ -96,9 +96,9 @@ Input [4]: [cntrycode#X, count#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] @@ -125,9 +125,9 @@ Results [3]: [cntrycode#X, count(1)#X AS numcust#X, sum(c_acctbal#X)#X AS totacc Input [3]: [cntrycode#X, numcust#X, totacctbal#X] Arguments: false -(21) VeloxAppendBatches +(21) VeloxResizeBatches Input [3]: [cntrycode#X, numcust#X, totacctbal#X] -Arguments: X +Arguments: X, X (22) ColumnarExchange Input [3]: [cntrycode#X, numcust#X, totacctbal#X] @@ -225,7 +225,7 @@ AdaptiveSparkPlan (60) +- ^ InputIteratorTransformer (50) +- ShuffleQueryStage (48), Statistics(X) +- ColumnarExchange (47) - +- VeloxAppendBatches (46) + +- VeloxResizeBatches (46) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) +- ^ NoopFilter (42) @@ -265,9 +265,9 @@ Results [2]: [sum#X, count#X] Input [2]: [sum#X, count#X] Arguments: false -(46) VeloxAppendBatches +(46) VeloxResizeBatches Input [2]: [sum#X, count#X] -Arguments: X +Arguments: X, X (47) ColumnarExchange Input [2]: [sum#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/3.txt index 988bdc6c26ca0..9673efca9bd60 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/3.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (54) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ ProjectExecTransformer (22) @@ -162,9 +162,9 @@ Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/4.txt index 0b73608fbe4b7..ce4180123f4c0 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/4.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (46) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ RegularHashAggregateExecTransformer (22) +- ^ InputIteratorTransformer (21) +- ShuffleQueryStage (19), Statistics(X) +- ColumnarExchange (18) - +- VeloxAppendBatches (17) + +- VeloxResizeBatches (17) +- ^ ProjectExecTransformer (15) +- ^ FlushableHashAggregateExecTransformer (14) +- ^ ProjectExecTransformer (13) @@ -115,9 +115,9 @@ Input [2]: [o_orderpriority#X, count#X] Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] Arguments: false -(17) VeloxAppendBatches +(17) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] -Arguments: X +Arguments: X, X (18) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] @@ -144,9 +144,9 @@ Results [2]: [o_orderpriority#X, count(1)#X AS order_count#X] Input [2]: [o_orderpriority#X, order_count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [2]: [o_orderpriority#X, order_count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [2]: [o_orderpriority#X, order_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/5.txt index 8891188e8a08b..742f9db1a0545 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/5.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (102) +- ^ InputIteratorTransformer (64) +- ShuffleQueryStage (62), Statistics(X) +- ColumnarExchange (61) - +- VeloxAppendBatches (60) + +- VeloxResizeBatches (60) +- ^ RegularHashAggregateExecTransformer (58) +- ^ InputIteratorTransformer (57) +- ShuffleQueryStage (55), Statistics(X) +- ColumnarExchange (54) - +- VeloxAppendBatches (53) + +- VeloxResizeBatches (53) +- ^ ProjectExecTransformer (51) +- ^ FlushableHashAggregateExecTransformer (50) +- ^ ProjectExecTransformer (49) @@ -319,9 +319,9 @@ Input [3]: [n_name#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] Arguments: false -(53) VeloxAppendBatches +(53) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (54) ColumnarExchange Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] @@ -348,9 +348,9 @@ Results [2]: [n_name#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X AS reven Input [2]: [n_name#X, revenue#X] Arguments: false -(60) VeloxAppendBatches +(60) VeloxResizeBatches Input [2]: [n_name#X, revenue#X] -Arguments: X +Arguments: X, X (61) ColumnarExchange Input [2]: [n_name#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/6.txt index 64624c791f726..89cf24d874744 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/6.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (20) +- ^ InputIteratorTransformer (10) +- ShuffleQueryStage (8), Statistics(X) +- ColumnarExchange (7) - +- VeloxAppendBatches (6) + +- VeloxResizeBatches (6) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ NoopFilter (2) @@ -46,9 +46,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(6) VeloxAppendBatches +(6) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (7) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/7.txt index ae1ab637f8050..7b0e80cded477 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/7.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (95) +- ^ InputIteratorTransformer (59) +- ShuffleQueryStage (57), Statistics(X) +- ColumnarExchange (56) - +- VeloxAppendBatches (55) + +- VeloxResizeBatches (55) +- ^ RegularHashAggregateExecTransformer (53) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50), Statistics(X) +- ColumnarExchange (49) - +- VeloxAppendBatches (48) + +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ FlushableHashAggregateExecTransformer (45) +- ^ ProjectExecTransformer (44) @@ -289,9 +289,9 @@ Input [5]: [supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] @@ -318,9 +318,9 @@ Results [4]: [supp_nation#X, cust_nation#X, l_year#X, sum(volume#X)#X AS revenue Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] Arguments: false -(55) VeloxAppendBatches +(55) VeloxResizeBatches Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] -Arguments: X +Arguments: X, X (56) ColumnarExchange Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/8.txt index 4bbf7967744ca..394c10ccb0123 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/8.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (131) +- ^ InputIteratorTransformer (83) +- ShuffleQueryStage (81), Statistics(X) +- ColumnarExchange (80) - +- VeloxAppendBatches (79) + +- VeloxResizeBatches (79) +- ^ ProjectExecTransformer (77) +- ^ RegularHashAggregateExecTransformer (76) +- ^ InputIteratorTransformer (75) +- ShuffleQueryStage (73), Statistics(X) +- ColumnarExchange (72) - +- VeloxAppendBatches (71) + +- VeloxResizeBatches (71) +- ^ ProjectExecTransformer (69) +- ^ FlushableHashAggregateExecTransformer (68) +- ^ ProjectExecTransformer (67) @@ -422,9 +422,9 @@ Input [5]: [o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(71) VeloxAppendBatches +(71) VeloxResizeBatches Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (72) ColumnarExchange Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] @@ -455,9 +455,9 @@ Input [3]: [o_year#X, sum(CASE WHEN (nation#X = BRAZIL) THEN volume#X ELSE 0.000 Input [2]: [o_year#X, mkt_share#X] Arguments: false -(79) VeloxAppendBatches +(79) VeloxResizeBatches Input [2]: [o_year#X, mkt_share#X] -Arguments: X +Arguments: X, X (80) ColumnarExchange Input [2]: [o_year#X, mkt_share#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/9.txt index 301c001eb7938..f9fe0dbc9ac17 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/9.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (100) +- ^ InputIteratorTransformer (63) +- ShuffleQueryStage (61), Statistics(X) +- ColumnarExchange (60) - +- VeloxAppendBatches (59) + +- VeloxResizeBatches (59) +- ^ RegularHashAggregateExecTransformer (57) +- ^ InputIteratorTransformer (56) +- ShuffleQueryStage (54), Statistics(X) +- ColumnarExchange (53) - +- VeloxAppendBatches (52) + +- VeloxResizeBatches (52) +- ^ ProjectExecTransformer (50) +- ^ FlushableHashAggregateExecTransformer (49) +- ^ ProjectExecTransformer (48) @@ -313,9 +313,9 @@ Input [4]: [nation#X, o_year#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] Arguments: false -(52) VeloxAppendBatches +(52) VeloxResizeBatches Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (53) ColumnarExchange Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] @@ -342,9 +342,9 @@ Results [3]: [nation#X, o_year#X, sum(amount#X)#X AS sum_profit#X] Input [3]: [nation#X, o_year#X, sum_profit#X] Arguments: false -(59) VeloxAppendBatches +(59) VeloxResizeBatches Input [3]: [nation#X, o_year#X, sum_profit#X] -Arguments: X +Arguments: X, X (60) ColumnarExchange Input [3]: [nation#X, o_year#X, sum_profit#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/1.txt index 63b7d317f3cf2..5ceb73b301db8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/1.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (30) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ RegularHashAggregateExecTransformer (12) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ ProjectExecTransformer (5) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) @@ -58,9 +58,9 @@ Input [17]: [l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] @@ -87,9 +87,9 @@ Results [10]: [l_returnflag#X, l_linestatus#X, sum(l_quantity#X)#X AS sum_qty#X, Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/10.txt index db7c1cb796676..5aeddac49db50 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/10.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (68) +- ^ InputIteratorTransformer (39) +- ShuffleQueryStage (37) +- ColumnarExchange (36) - +- VeloxAppendBatches (35) + +- VeloxResizeBatches (35) +- ^ ProjectExecTransformer (33) +- ^ FlushableHashAggregateExecTransformer (32) +- ^ ProjectExecTransformer (31) @@ -210,9 +210,9 @@ Input [9]: [c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] Arguments: false -(35) VeloxAppendBatches +(35) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (36) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/11.txt index 33c7971d37492..7d95202ba98fc 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/11.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (60) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ FilterExecTransformer (31) +- ^ RegularHashAggregateExecTransformer (30) +- ^ InputIteratorTransformer (29) +- ShuffleQueryStage (27) +- ColumnarExchange (26) - +- VeloxAppendBatches (25) + +- VeloxResizeBatches (25) +- ^ ProjectExecTransformer (23) +- ^ FlushableHashAggregateExecTransformer (22) +- ^ ProjectExecTransformer (21) @@ -160,9 +160,9 @@ Input [3]: [ps_partkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] Arguments: false -(25) VeloxAppendBatches +(25) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (26) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] @@ -193,9 +193,9 @@ Arguments: (isnotnull(value#X) AND (cast(value#X as decimal(38,6)) > Subquery su Input [2]: [ps_partkey#X, value#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [2]: [ps_partkey#X, value#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [2]: [ps_partkey#X, value#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/12.txt index f1f3a9234354e..39d1eb13754eb 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/12.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (44) +- ^ InputIteratorTransformer (27) +- ShuffleQueryStage (25) +- ColumnarExchange (24) - +- VeloxAppendBatches (23) + +- VeloxResizeBatches (23) +- ^ RegularHashAggregateExecTransformer (21) +- ^ InputIteratorTransformer (20) +- ShuffleQueryStage (18) +- ColumnarExchange (17) - +- VeloxAppendBatches (16) + +- VeloxResizeBatches (16) +- ^ ProjectExecTransformer (14) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) @@ -108,9 +108,9 @@ Input [3]: [l_shipmode#X, sum#X, sum#X] Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] Arguments: false -(16) VeloxAppendBatches +(16) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] -Arguments: X +Arguments: X, X (17) ColumnarExchange Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] @@ -137,9 +137,9 @@ Results [3]: [l_shipmode#X, sum(CASE WHEN ((o_orderpriority#X = 1-URGENT) OR (o_ Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/13.txt index 13ef25b681c0d..1033f582f2b91 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/13.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (36) +- ShuffleQueryStage (34) +- ColumnarExchange (33) - +- VeloxAppendBatches (32) + +- VeloxResizeBatches (32) +- ^ RegularHashAggregateExecTransformer (30) +- ^ InputIteratorTransformer (29) +- ShuffleQueryStage (27) +- ColumnarExchange (26) - +- VeloxAppendBatches (25) + +- VeloxResizeBatches (25) +- ^ ProjectExecTransformer (23) +- ^ FlushableHashAggregateExecTransformer (22) +- ^ ProjectExecTransformer (21) @@ -19,7 +19,7 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ ProjectExecTransformer (13) +- ^ FlushableHashAggregateExecTransformer (12) +- ^ ProjectExecTransformer (11) @@ -112,9 +112,9 @@ Input [2]: [c_custkey#X, count#X] Input [3]: [hash_partition_key#X, c_custkey#X, count#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, count#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, count#X] @@ -156,9 +156,9 @@ Input [2]: [c_count#X, count#X] Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(25) VeloxAppendBatches +(25) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] -Arguments: X +Arguments: X, X (26) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] @@ -185,9 +185,9 @@ Results [2]: [c_count#X, count(1)#X AS custdist#X] Input [2]: [c_count#X, custdist#X] Arguments: false -(32) VeloxAppendBatches +(32) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] -Arguments: X +Arguments: X, X (33) ColumnarExchange Input [2]: [c_count#X, custdist#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/14.txt index c86f2215cda5b..b9231f934ea8a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/14.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (35) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) @@ -97,9 +97,9 @@ Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/15.txt index 8edb179e592fc..f3b74c2c51045 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/15.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (46) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ BroadcastHashJoinExecTransformer Inner BuildLeft (21) :- ^ InputIteratorTransformer (7) @@ -19,7 +19,7 @@ AdaptiveSparkPlan (46) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) @@ -101,9 +101,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] @@ -143,9 +143,9 @@ Input [6]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, supplier_no#X, total_ Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/16.txt index 4d4d52c7d2e61..440e56487b6f3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/16.txt @@ -6,19 +6,19 @@ AdaptiveSparkPlan (59) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ RegularHashAggregateExecTransformer (20) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ ProjectExecTransformer (13) +- ^ FlushableHashAggregateExecTransformer (12) +- ^ ProjectExecTransformer (11) @@ -117,9 +117,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] @@ -157,9 +157,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, count#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] @@ -186,9 +186,9 @@ Results [4]: [p_brand#X, p_type#X, p_size#X, count(ps_suppkey#X)#X AS supplier_c Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/17.txt index 7c5359849d4e8..7f4e4e666c0d6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/17.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (36) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ FlushableHashAggregateExecTransformer (5) +- ^ InputIteratorTransformer (4) +- RowToVeloxColumnar (2) @@ -59,9 +59,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/18.txt index a4aaf08ff8bb3..6da312afa5f9a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/18.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (88) +- ^ InputIteratorTransformer (51) +- ShuffleQueryStage (49) +- ColumnarExchange (48) - +- VeloxAppendBatches (47) + +- VeloxResizeBatches (47) +- ^ ProjectExecTransformer (45) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) @@ -31,7 +31,7 @@ AdaptiveSparkPlan (88) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FlushableHashAggregateExecTransformer (11) : +- ^ Scan parquet (10) @@ -140,9 +140,9 @@ Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] @@ -276,9 +276,9 @@ Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] Arguments: false -(47) VeloxAppendBatches +(47) VeloxResizeBatches Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (48) ColumnarExchange Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/19.txt index 87acf3c4b28e6..835386f95fdc7 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/19.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (34) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) @@ -96,9 +96,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/20.txt index 9e03d83195371..51eaaa2a51b9b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/20.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (112) +- ^ InputIteratorTransformer (70) +- ShuffleQueryStage (68) +- ColumnarExchange (67) - +- VeloxAppendBatches (66) + +- VeloxResizeBatches (66) +- ^ ProjectExecTransformer (64) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (63) :- ^ ProjectExecTransformer (54) @@ -15,7 +15,7 @@ AdaptiveSparkPlan (112) : : +- AQEShuffleRead (8) : : +- ShuffleQueryStage (7) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) @@ -42,7 +42,7 @@ AdaptiveSparkPlan (112) : +- ^ InputIteratorTransformer (42) : +- ShuffleQueryStage (40) : +- ColumnarExchange (39) - : +- VeloxAppendBatches (38) + : +- VeloxResizeBatches (38) : +- ^ ProjectExecTransformer (36) : +- ^ FlushableHashAggregateExecTransformer (35) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (34) @@ -118,9 +118,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] @@ -255,9 +255,9 @@ Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(38) VeloxAppendBatches +(38) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (39) ColumnarExchange Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] @@ -370,9 +370,9 @@ Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] Input [2]: [s_name#X, s_address#X] Arguments: false -(66) VeloxAppendBatches +(66) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] -Arguments: X +Arguments: X, X (67) ColumnarExchange Input [2]: [s_name#X, s_address#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/21.txt index ee28d7b592e87..875f05406e856 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/21.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (93) +- ^ InputIteratorTransformer (56) +- ShuffleQueryStage (54) +- ColumnarExchange (53) - +- VeloxAppendBatches (52) + +- VeloxResizeBatches (52) +- ^ ProjectExecTransformer (50) +- ^ FlushableHashAggregateExecTransformer (49) +- ^ ProjectExecTransformer (48) @@ -302,9 +302,9 @@ Input [2]: [s_name#X, count#X] Input [3]: [hash_partition_key#X, s_name#X, count#X] Arguments: false -(52) VeloxAppendBatches +(52) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, count#X] -Arguments: X +Arguments: X, X (53) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/22.txt index d578b43f3d6a3..e0b3b12c7b964 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/22.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (40) +- ^ InputIteratorTransformer (25) +- ShuffleQueryStage (23) +- ColumnarExchange (22) - +- VeloxAppendBatches (21) + +- VeloxResizeBatches (21) +- ^ RegularHashAggregateExecTransformer (19) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) @@ -95,9 +95,9 @@ Input [4]: [cntrycode#X, count#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] @@ -124,9 +124,9 @@ Results [3]: [cntrycode#X, count(1)#X AS numcust#X, sum(c_acctbal#X)#X AS totacc Input [3]: [cntrycode#X, numcust#X, totacctbal#X] Arguments: false -(21) VeloxAppendBatches +(21) VeloxResizeBatches Input [3]: [cntrycode#X, numcust#X, totacctbal#X] -Arguments: X +Arguments: X, X (22) ColumnarExchange Input [3]: [cntrycode#X, numcust#X, totacctbal#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/3.txt index 2ffd75b92964a..ed5f34e91889e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/3.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (54) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ ProjectExecTransformer (22) @@ -160,9 +160,9 @@ Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/4.txt index 36c6de5374ba2..a24a32699b79f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/4.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (46) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ RegularHashAggregateExecTransformer (22) +- ^ InputIteratorTransformer (21) +- ShuffleQueryStage (19) +- ColumnarExchange (18) - +- VeloxAppendBatches (17) + +- VeloxResizeBatches (17) +- ^ ProjectExecTransformer (15) +- ^ FlushableHashAggregateExecTransformer (14) +- ^ ProjectExecTransformer (13) @@ -114,9 +114,9 @@ Input [2]: [o_orderpriority#X, count#X] Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] Arguments: false -(17) VeloxAppendBatches +(17) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] -Arguments: X +Arguments: X, X (18) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] @@ -143,9 +143,9 @@ Results [2]: [o_orderpriority#X, count(1)#X AS order_count#X] Input [2]: [o_orderpriority#X, order_count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [2]: [o_orderpriority#X, order_count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [2]: [o_orderpriority#X, order_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/5.txt index 49ee27485b4ac..178ee7c7e2f38 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/5.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (102) +- ^ InputIteratorTransformer (64) +- ShuffleQueryStage (62) +- ColumnarExchange (61) - +- VeloxAppendBatches (60) + +- VeloxResizeBatches (60) +- ^ RegularHashAggregateExecTransformer (58) +- ^ InputIteratorTransformer (57) +- ShuffleQueryStage (55) +- ColumnarExchange (54) - +- VeloxAppendBatches (53) + +- VeloxResizeBatches (53) +- ^ ProjectExecTransformer (51) +- ^ FlushableHashAggregateExecTransformer (50) +- ^ ProjectExecTransformer (49) @@ -314,9 +314,9 @@ Input [3]: [n_name#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] Arguments: false -(53) VeloxAppendBatches +(53) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (54) ColumnarExchange Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] @@ -343,9 +343,9 @@ Results [2]: [n_name#X, sum(CheckOverflow((promote_precision(cast(l_extendedpric Input [2]: [n_name#X, revenue#X] Arguments: false -(60) VeloxAppendBatches +(60) VeloxResizeBatches Input [2]: [n_name#X, revenue#X] -Arguments: X +Arguments: X, X (61) ColumnarExchange Input [2]: [n_name#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/6.txt index 786a89fe715ae..ee8c494f56a30 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/6.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (20) +- ^ InputIteratorTransformer (10) +- ShuffleQueryStage (8) +- ColumnarExchange (7) - +- VeloxAppendBatches (6) + +- VeloxResizeBatches (6) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) @@ -46,9 +46,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(6) VeloxAppendBatches +(6) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (7) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/7.txt index 2ba42f806f3e5..f0e7e37d3887e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/7.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (95) +- ^ InputIteratorTransformer (59) +- ShuffleQueryStage (57) +- ColumnarExchange (56) - +- VeloxAppendBatches (55) + +- VeloxResizeBatches (55) +- ^ RegularHashAggregateExecTransformer (53) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50) +- ColumnarExchange (49) - +- VeloxAppendBatches (48) + +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ FlushableHashAggregateExecTransformer (45) +- ^ ProjectExecTransformer (44) @@ -284,9 +284,9 @@ Input [5]: [supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] @@ -313,9 +313,9 @@ Results [4]: [supp_nation#X, cust_nation#X, l_year#X, sum(volume#X)#X AS revenue Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] Arguments: false -(55) VeloxAppendBatches +(55) VeloxResizeBatches Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] -Arguments: X +Arguments: X, X (56) ColumnarExchange Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/8.txt index 64a40563c2381..9d9abd6b04bf3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/8.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (131) +- ^ InputIteratorTransformer (83) +- ShuffleQueryStage (81) +- ColumnarExchange (80) - +- VeloxAppendBatches (79) + +- VeloxResizeBatches (79) +- ^ ProjectExecTransformer (77) +- ^ RegularHashAggregateExecTransformer (76) +- ^ InputIteratorTransformer (75) +- ShuffleQueryStage (73) +- ColumnarExchange (72) - +- VeloxAppendBatches (71) + +- VeloxResizeBatches (71) +- ^ ProjectExecTransformer (69) +- ^ FlushableHashAggregateExecTransformer (68) +- ^ ProjectExecTransformer (67) @@ -415,9 +415,9 @@ Input [5]: [o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(71) VeloxAppendBatches +(71) VeloxResizeBatches Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (72) ColumnarExchange Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] @@ -448,9 +448,9 @@ Input [3]: [o_year#X, sum(CASE WHEN (nation#X = BRAZIL) THEN volume#X ELSE 0.000 Input [2]: [o_year#X, mkt_share#X] Arguments: false -(79) VeloxAppendBatches +(79) VeloxResizeBatches Input [2]: [o_year#X, mkt_share#X] -Arguments: X +Arguments: X, X (80) ColumnarExchange Input [2]: [o_year#X, mkt_share#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/9.txt index b5a21a0261d2e..2b213b681a052 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/9.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (100) +- ^ InputIteratorTransformer (63) +- ShuffleQueryStage (61) +- ColumnarExchange (60) - +- VeloxAppendBatches (59) + +- VeloxResizeBatches (59) +- ^ RegularHashAggregateExecTransformer (57) +- ^ InputIteratorTransformer (56) +- ShuffleQueryStage (54) +- ColumnarExchange (53) - +- VeloxAppendBatches (52) + +- VeloxResizeBatches (52) +- ^ ProjectExecTransformer (50) +- ^ FlushableHashAggregateExecTransformer (49) +- ^ ProjectExecTransformer (48) @@ -308,9 +308,9 @@ Input [4]: [nation#X, o_year#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] Arguments: false -(52) VeloxAppendBatches +(52) VeloxResizeBatches Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (53) ColumnarExchange Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] @@ -337,9 +337,9 @@ Results [3]: [nation#X, o_year#X, sum(amount#X)#X AS sum_profit#X] Input [3]: [nation#X, o_year#X, sum_profit#X] Arguments: false -(59) VeloxAppendBatches +(59) VeloxResizeBatches Input [3]: [nation#X, o_year#X, sum_profit#X] -Arguments: X +Arguments: X, X (60) ColumnarExchange Input [3]: [nation#X, o_year#X, sum_profit#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/1.txt index 22dd5100c4fb3..12c19c45e38d1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/1.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (30) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ RegularHashAggregateExecTransformer (12) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9), Statistics(X) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ ProjectExecTransformer (5) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) @@ -58,9 +58,9 @@ Input [17]: [l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] @@ -87,9 +87,9 @@ Results [10]: [l_returnflag#X, l_linestatus#X, sum(l_quantity#X)#X AS sum_qty#X, Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/10.txt index 8681ecf5f93f6..adaeaf49efeeb 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/10.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (68) +- ^ InputIteratorTransformer (39) +- ShuffleQueryStage (37), Statistics(X) +- ColumnarExchange (36) - +- VeloxAppendBatches (35) + +- VeloxResizeBatches (35) +- ^ ProjectExecTransformer (33) +- ^ FlushableHashAggregateExecTransformer (32) +- ^ ProjectExecTransformer (31) @@ -210,9 +210,9 @@ Input [9]: [c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] Arguments: false -(35) VeloxAppendBatches +(35) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (36) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/11.txt index 6dfc1d00f4cc1..25d3c4ac2f75b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/11.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (60) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35), Statistics(X) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ FilterExecTransformer (31) +- ^ RegularHashAggregateExecTransformer (30) +- ^ InputIteratorTransformer (29) +- ShuffleQueryStage (27), Statistics(X) +- ColumnarExchange (26) - +- VeloxAppendBatches (25) + +- VeloxResizeBatches (25) +- ^ ProjectExecTransformer (23) +- ^ FlushableHashAggregateExecTransformer (22) +- ^ ProjectExecTransformer (21) @@ -160,9 +160,9 @@ Input [3]: [ps_partkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] Arguments: false -(25) VeloxAppendBatches +(25) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (26) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] @@ -193,9 +193,9 @@ Arguments: (isnotnull(value#X) AND (cast(value#X as decimal(38,6)) > Subquery su Input [2]: [ps_partkey#X, value#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [2]: [ps_partkey#X, value#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [2]: [ps_partkey#X, value#X] @@ -330,7 +330,7 @@ AdaptiveSparkPlan (102) +- ^ InputIteratorTransformer (81) +- ShuffleQueryStage (79), Statistics(X) +- ColumnarExchange (78) - +- VeloxAppendBatches (77) + +- VeloxResizeBatches (77) +- ^ FlushableHashAggregateExecTransformer (75) +- ^ ProjectExecTransformer (74) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (73) @@ -429,9 +429,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(77) VeloxAppendBatches +(77) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (78) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/12.txt index 1e1ad6d497a4a..21681feeacc14 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/12.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (44) +- ^ InputIteratorTransformer (27) +- ShuffleQueryStage (25), Statistics(X) +- ColumnarExchange (24) - +- VeloxAppendBatches (23) + +- VeloxResizeBatches (23) +- ^ RegularHashAggregateExecTransformer (21) +- ^ InputIteratorTransformer (20) +- ShuffleQueryStage (18), Statistics(X) +- ColumnarExchange (17) - +- VeloxAppendBatches (16) + +- VeloxResizeBatches (16) +- ^ ProjectExecTransformer (14) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) @@ -108,9 +108,9 @@ Input [3]: [l_shipmode#X, sum#X, sum#X] Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] Arguments: false -(16) VeloxAppendBatches +(16) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] -Arguments: X +Arguments: X, X (17) ColumnarExchange Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] @@ -137,9 +137,9 @@ Results [3]: [l_shipmode#X, sum(CASE WHEN ((o_orderpriority#X = 1-URGENT) OR (o_ Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/13.txt index ff6532aa65791..808e8189cca4f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/13.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (36) +- ShuffleQueryStage (34), Statistics(X) +- ColumnarExchange (33) - +- VeloxAppendBatches (32) + +- VeloxResizeBatches (32) +- ^ RegularHashAggregateExecTransformer (30) +- ^ InputIteratorTransformer (29) +- ShuffleQueryStage (27), Statistics(X) +- ColumnarExchange (26) - +- VeloxAppendBatches (25) + +- VeloxResizeBatches (25) +- ^ ProjectExecTransformer (23) +- ^ FlushableHashAggregateExecTransformer (22) +- ^ ProjectExecTransformer (21) @@ -19,7 +19,7 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ ProjectExecTransformer (13) +- ^ FlushableHashAggregateExecTransformer (12) +- ^ ProjectExecTransformer (11) @@ -112,9 +112,9 @@ Input [2]: [c_custkey#X, count#X] Input [3]: [hash_partition_key#X, c_custkey#X, count#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, count#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, count#X] @@ -156,9 +156,9 @@ Input [2]: [c_count#X, count#X] Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(25) VeloxAppendBatches +(25) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] -Arguments: X +Arguments: X, X (26) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] @@ -185,9 +185,9 @@ Results [2]: [c_count#X, count(1)#X AS custdist#X] Input [2]: [c_count#X, custdist#X] Arguments: false -(32) VeloxAppendBatches +(32) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] -Arguments: X +Arguments: X, X (33) ColumnarExchange Input [2]: [c_count#X, custdist#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/14.txt index d144dd39ca1d0..6ac8b1cd94d36 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/14.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (35) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) @@ -97,9 +97,9 @@ Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/15.txt index c4f825f5ca533..671000b2f28d2 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/15.txt @@ -5,7 +5,7 @@ AdaptiveSparkPlan (43) +- AQEShuffleRead (27) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ BroadcastHashJoinExecTransformer Inner BuildLeft (21) :- ^ InputIteratorTransformer (7) @@ -18,7 +18,7 @@ AdaptiveSparkPlan (43) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) @@ -100,9 +100,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] @@ -142,9 +142,9 @@ Input [6]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, supplier_no#X, total_ Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] @@ -247,7 +247,7 @@ AdaptiveSparkPlan (69) +- ^ InputIteratorTransformer (54) +- ShuffleQueryStage (52), Statistics(X) +- ColumnarExchange (51) - +- VeloxAppendBatches (50) + +- VeloxResizeBatches (50) +- ^ ProjectExecTransformer (48) +- ^ FlushableHashAggregateExecTransformer (47) +- ^ ProjectExecTransformer (46) @@ -294,9 +294,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(50) VeloxAppendBatches +(50) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (51) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/16.txt index 741b837976a42..19e334e888ec5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/16.txt @@ -6,19 +6,19 @@ AdaptiveSparkPlan (59) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ RegularHashAggregateExecTransformer (20) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ ProjectExecTransformer (13) +- ^ FlushableHashAggregateExecTransformer (12) +- ^ ProjectExecTransformer (11) @@ -117,9 +117,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] @@ -157,9 +157,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, count#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] @@ -186,9 +186,9 @@ Results [4]: [p_brand#X, p_type#X, p_size#X, count(ps_suppkey#X)#X AS supplier_c Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/17.txt index 1e2ed970aef4f..38801e9f690a8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/17.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (36) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9), Statistics(X) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ FlushableHashAggregateExecTransformer (5) +- ^ InputIteratorTransformer (4) +- RowToVeloxColumnar (2) @@ -59,9 +59,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/18.txt index 05884fea78851..26995352c1cfe 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/18.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (88) +- ^ InputIteratorTransformer (51) +- ShuffleQueryStage (49), Statistics(X) +- ColumnarExchange (48) - +- VeloxAppendBatches (47) + +- VeloxResizeBatches (47) +- ^ ProjectExecTransformer (45) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) @@ -31,7 +31,7 @@ AdaptiveSparkPlan (88) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FlushableHashAggregateExecTransformer (11) : +- ^ Scan parquet (10) @@ -140,9 +140,9 @@ Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] @@ -276,9 +276,9 @@ Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] Arguments: false -(47) VeloxAppendBatches +(47) VeloxResizeBatches Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (48) ColumnarExchange Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/19.txt index f044660313529..ce6a3f10da038 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/19.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (34) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) @@ -96,9 +96,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/20.txt index 5cd3c9d35c2c7..125e495397e8f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/20.txt @@ -5,7 +5,7 @@ AdaptiveSparkPlan (109) +- AQEShuffleRead (69) +- ShuffleQueryStage (68), Statistics(X) +- ColumnarExchange (67) - +- VeloxAppendBatches (66) + +- VeloxResizeBatches (66) +- ^ ProjectExecTransformer (64) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (63) :- ^ ProjectExecTransformer (54) @@ -14,7 +14,7 @@ AdaptiveSparkPlan (109) : : +- AQEShuffleRead (8) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) @@ -41,7 +41,7 @@ AdaptiveSparkPlan (109) : +- ^ InputIteratorTransformer (42) : +- ShuffleQueryStage (40), Statistics(X) : +- ColumnarExchange (39) - : +- VeloxAppendBatches (38) + : +- VeloxResizeBatches (38) : +- ^ ProjectExecTransformer (36) : +- ^ FlushableHashAggregateExecTransformer (35) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (34) @@ -117,9 +117,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] @@ -254,9 +254,9 @@ Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(38) VeloxAppendBatches +(38) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (39) ColumnarExchange Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] @@ -369,9 +369,9 @@ Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] Input [2]: [s_name#X, s_address#X] Arguments: false -(66) VeloxAppendBatches +(66) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] -Arguments: X +Arguments: X, X (67) ColumnarExchange Input [2]: [s_name#X, s_address#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/21.txt index 3ffdec004af2d..6e411317dd33f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/21.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (92) +- ^ InputIteratorTransformer (56) +- ShuffleQueryStage (54), Statistics(X) +- ColumnarExchange (53) - +- VeloxAppendBatches (52) + +- VeloxResizeBatches (52) +- ^ ProjectExecTransformer (50) +- ^ FlushableHashAggregateExecTransformer (49) +- ^ ProjectExecTransformer (48) @@ -301,9 +301,9 @@ Input [2]: [s_name#X, count#X] Input [3]: [hash_partition_key#X, s_name#X, count#X] Arguments: false -(52) VeloxAppendBatches +(52) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, count#X] -Arguments: X +Arguments: X, X (53) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/22.txt index f30752effaa42..a4883c6228f4a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/22.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (40) +- ^ InputIteratorTransformer (25) +- ShuffleQueryStage (23), Statistics(X) +- ColumnarExchange (22) - +- VeloxAppendBatches (21) + +- VeloxResizeBatches (21) +- ^ RegularHashAggregateExecTransformer (19) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) @@ -95,9 +95,9 @@ Input [4]: [cntrycode#X, count#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] @@ -124,9 +124,9 @@ Results [3]: [cntrycode#X, count(1)#X AS numcust#X, sum(c_acctbal#X)#X AS totacc Input [3]: [cntrycode#X, numcust#X, totacctbal#X] Arguments: false -(21) VeloxAppendBatches +(21) VeloxResizeBatches Input [3]: [cntrycode#X, numcust#X, totacctbal#X] -Arguments: X +Arguments: X, X (22) ColumnarExchange Input [3]: [cntrycode#X, numcust#X, totacctbal#X] @@ -223,7 +223,7 @@ AdaptiveSparkPlan (60) +- ^ InputIteratorTransformer (50) +- ShuffleQueryStage (48), Statistics(X) +- ColumnarExchange (47) - +- VeloxAppendBatches (46) + +- VeloxResizeBatches (46) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) +- ^ FilterExecTransformer (42) @@ -263,9 +263,9 @@ Results [2]: [sum#X, count#X] Input [2]: [sum#X, count#X] Arguments: false -(46) VeloxAppendBatches +(46) VeloxResizeBatches Input [2]: [sum#X, count#X] -Arguments: X +Arguments: X, X (47) ColumnarExchange Input [2]: [sum#X, count#X] @@ -340,7 +340,7 @@ AdaptiveSparkPlan (60) +- ^ InputIteratorTransformer (50) +- ShuffleQueryStage (48), Statistics(X) +- ColumnarExchange (47) - +- VeloxAppendBatches (46) + +- VeloxResizeBatches (46) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) +- ^ FilterExecTransformer (42) diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/3.txt index d9f87ca9b5381..41aa9b576a9d8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/3.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (54) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ ProjectExecTransformer (22) @@ -160,9 +160,9 @@ Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/4.txt index db9f7716a365c..7f87abe3e7a28 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/4.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (46) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ RegularHashAggregateExecTransformer (22) +- ^ InputIteratorTransformer (21) +- ShuffleQueryStage (19), Statistics(X) +- ColumnarExchange (18) - +- VeloxAppendBatches (17) + +- VeloxResizeBatches (17) +- ^ ProjectExecTransformer (15) +- ^ FlushableHashAggregateExecTransformer (14) +- ^ ProjectExecTransformer (13) @@ -114,9 +114,9 @@ Input [2]: [o_orderpriority#X, count#X] Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] Arguments: false -(17) VeloxAppendBatches +(17) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] -Arguments: X +Arguments: X, X (18) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] @@ -143,9 +143,9 @@ Results [2]: [o_orderpriority#X, count(1)#X AS order_count#X] Input [2]: [o_orderpriority#X, order_count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [2]: [o_orderpriority#X, order_count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [2]: [o_orderpriority#X, order_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/5.txt index 67e9d847aefad..6780a88b37da3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/5.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (102) +- ^ InputIteratorTransformer (64) +- ShuffleQueryStage (62), Statistics(X) +- ColumnarExchange (61) - +- VeloxAppendBatches (60) + +- VeloxResizeBatches (60) +- ^ RegularHashAggregateExecTransformer (58) +- ^ InputIteratorTransformer (57) +- ShuffleQueryStage (55), Statistics(X) +- ColumnarExchange (54) - +- VeloxAppendBatches (53) + +- VeloxResizeBatches (53) +- ^ ProjectExecTransformer (51) +- ^ FlushableHashAggregateExecTransformer (50) +- ^ ProjectExecTransformer (49) @@ -314,9 +314,9 @@ Input [3]: [n_name#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] Arguments: false -(53) VeloxAppendBatches +(53) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (54) ColumnarExchange Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] @@ -343,9 +343,9 @@ Results [2]: [n_name#X, sum(CheckOverflow((promote_precision(cast(l_extendedpric Input [2]: [n_name#X, revenue#X] Arguments: false -(60) VeloxAppendBatches +(60) VeloxResizeBatches Input [2]: [n_name#X, revenue#X] -Arguments: X +Arguments: X, X (61) ColumnarExchange Input [2]: [n_name#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/6.txt index 3432579a0de06..864cdf3832e13 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/6.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (20) +- ^ InputIteratorTransformer (10) +- ShuffleQueryStage (8), Statistics(X) +- ColumnarExchange (7) - +- VeloxAppendBatches (6) + +- VeloxResizeBatches (6) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) @@ -46,9 +46,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(6) VeloxAppendBatches +(6) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (7) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/7.txt index 3db2d7669f149..5f81d7e4298bd 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/7.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (95) +- ^ InputIteratorTransformer (59) +- ShuffleQueryStage (57), Statistics(X) +- ColumnarExchange (56) - +- VeloxAppendBatches (55) + +- VeloxResizeBatches (55) +- ^ RegularHashAggregateExecTransformer (53) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50), Statistics(X) +- ColumnarExchange (49) - +- VeloxAppendBatches (48) + +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ FlushableHashAggregateExecTransformer (45) +- ^ ProjectExecTransformer (44) @@ -284,9 +284,9 @@ Input [5]: [supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] @@ -313,9 +313,9 @@ Results [4]: [supp_nation#X, cust_nation#X, l_year#X, sum(volume#X)#X AS revenue Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] Arguments: false -(55) VeloxAppendBatches +(55) VeloxResizeBatches Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] -Arguments: X +Arguments: X, X (56) ColumnarExchange Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/8.txt index f77b5fca35c8e..292fc5bfd4a84 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/8.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (131) +- ^ InputIteratorTransformer (83) +- ShuffleQueryStage (81), Statistics(X) +- ColumnarExchange (80) - +- VeloxAppendBatches (79) + +- VeloxResizeBatches (79) +- ^ ProjectExecTransformer (77) +- ^ RegularHashAggregateExecTransformer (76) +- ^ InputIteratorTransformer (75) +- ShuffleQueryStage (73), Statistics(X) +- ColumnarExchange (72) - +- VeloxAppendBatches (71) + +- VeloxResizeBatches (71) +- ^ ProjectExecTransformer (69) +- ^ FlushableHashAggregateExecTransformer (68) +- ^ ProjectExecTransformer (67) @@ -415,9 +415,9 @@ Input [5]: [o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(71) VeloxAppendBatches +(71) VeloxResizeBatches Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (72) ColumnarExchange Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] @@ -448,9 +448,9 @@ Input [3]: [o_year#X, sum(CASE WHEN (nation#X = BRAZIL) THEN volume#X ELSE 0.000 Input [2]: [o_year#X, mkt_share#X] Arguments: false -(79) VeloxAppendBatches +(79) VeloxResizeBatches Input [2]: [o_year#X, mkt_share#X] -Arguments: X +Arguments: X, X (80) ColumnarExchange Input [2]: [o_year#X, mkt_share#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/9.txt index 1be487ed123ae..b5ed852a37e44 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/9.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (100) +- ^ InputIteratorTransformer (63) +- ShuffleQueryStage (61), Statistics(X) +- ColumnarExchange (60) - +- VeloxAppendBatches (59) + +- VeloxResizeBatches (59) +- ^ RegularHashAggregateExecTransformer (57) +- ^ InputIteratorTransformer (56) +- ShuffleQueryStage (54), Statistics(X) +- ColumnarExchange (53) - +- VeloxAppendBatches (52) + +- VeloxResizeBatches (52) +- ^ ProjectExecTransformer (50) +- ^ FlushableHashAggregateExecTransformer (49) +- ^ ProjectExecTransformer (48) @@ -308,9 +308,9 @@ Input [4]: [nation#X, o_year#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] Arguments: false -(52) VeloxAppendBatches +(52) VeloxResizeBatches Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (53) ColumnarExchange Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] @@ -337,9 +337,9 @@ Results [3]: [nation#X, o_year#X, sum(amount#X)#X AS sum_profit#X] Input [3]: [nation#X, o_year#X, sum_profit#X] Arguments: false -(59) VeloxAppendBatches +(59) VeloxResizeBatches Input [3]: [nation#X, o_year#X, sum_profit#X] -Arguments: X +Arguments: X, X (60) ColumnarExchange Input [3]: [nation#X, o_year#X, sum_profit#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/1.txt index 1e53cd90e1b35..0c773785eafce 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/1.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (30) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ RegularHashAggregateExecTransformer (12) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9), Statistics(X) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ ProjectExecTransformer (5) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) @@ -58,9 +58,9 @@ Input [17]: [l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] @@ -87,9 +87,9 @@ Results [10]: [l_returnflag#X, l_linestatus#X, sum(l_quantity#X)#X AS sum_qty#X, Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/10.txt index 098b6610a2e12..c4f94275a4408 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/10.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (68) +- ^ InputIteratorTransformer (39) +- ShuffleQueryStage (37), Statistics(X) +- ColumnarExchange (36) - +- VeloxAppendBatches (35) + +- VeloxResizeBatches (35) +- ^ ProjectExecTransformer (33) +- ^ FlushableHashAggregateExecTransformer (32) +- ^ ProjectExecTransformer (31) @@ -213,9 +213,9 @@ Input [9]: [c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] Arguments: false -(35) VeloxAppendBatches +(35) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (36) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/11.txt index 725debe00d5e3..cb4bff2a0ea33 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/11.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (60) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35), Statistics(X) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ FilterExecTransformer (31) +- ^ RegularHashAggregateExecTransformer (30) +- ^ InputIteratorTransformer (29) +- ShuffleQueryStage (27), Statistics(X) +- ColumnarExchange (26) - +- VeloxAppendBatches (25) + +- VeloxResizeBatches (25) +- ^ ProjectExecTransformer (23) +- ^ FlushableHashAggregateExecTransformer (22) +- ^ ProjectExecTransformer (21) @@ -162,9 +162,9 @@ Input [3]: [ps_partkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] Arguments: false -(25) VeloxAppendBatches +(25) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (26) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] @@ -195,9 +195,9 @@ Arguments: (isnotnull(value#X) AND (cast(value#X as decimal(38,6)) > Subquery su Input [2]: [ps_partkey#X, value#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [2]: [ps_partkey#X, value#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [2]: [ps_partkey#X, value#X] @@ -334,7 +334,7 @@ AdaptiveSparkPlan (102) +- ^ InputIteratorTransformer (81) +- ShuffleQueryStage (79), Statistics(X) +- ColumnarExchange (78) - +- VeloxAppendBatches (77) + +- VeloxResizeBatches (77) +- ^ FlushableHashAggregateExecTransformer (75) +- ^ ProjectExecTransformer (74) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (73) @@ -435,9 +435,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(77) VeloxAppendBatches +(77) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (78) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/12.txt index e2b9e5153009f..099fdbce656a3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/12.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (44) +- ^ InputIteratorTransformer (27) +- ShuffleQueryStage (25), Statistics(X) +- ColumnarExchange (24) - +- VeloxAppendBatches (23) + +- VeloxResizeBatches (23) +- ^ RegularHashAggregateExecTransformer (21) +- ^ InputIteratorTransformer (20) +- ShuffleQueryStage (18), Statistics(X) +- ColumnarExchange (17) - +- VeloxAppendBatches (16) + +- VeloxResizeBatches (16) +- ^ ProjectExecTransformer (14) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) @@ -109,9 +109,9 @@ Input [3]: [l_shipmode#X, sum#X, sum#X] Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] Arguments: false -(16) VeloxAppendBatches +(16) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] -Arguments: X +Arguments: X, X (17) ColumnarExchange Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] @@ -138,9 +138,9 @@ Results [3]: [l_shipmode#X, sum(CASE WHEN ((o_orderpriority#X = 1-URGENT) OR (o_ Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/13.txt index f64de4dee4b20..3a2fb9b787603 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/13.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (36) +- ShuffleQueryStage (34), Statistics(X) +- ColumnarExchange (33) - +- VeloxAppendBatches (32) + +- VeloxResizeBatches (32) +- ^ RegularHashAggregateExecTransformer (30) +- ^ InputIteratorTransformer (29) +- ShuffleQueryStage (27), Statistics(X) +- ColumnarExchange (26) - +- VeloxAppendBatches (25) + +- VeloxResizeBatches (25) +- ^ ProjectExecTransformer (23) +- ^ FlushableHashAggregateExecTransformer (22) +- ^ ProjectExecTransformer (21) @@ -19,7 +19,7 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ ProjectExecTransformer (13) +- ^ FlushableHashAggregateExecTransformer (12) +- ^ ProjectExecTransformer (11) @@ -113,9 +113,9 @@ Input [2]: [c_custkey#X, count#X] Input [3]: [hash_partition_key#X, c_custkey#X, count#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, count#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, count#X] @@ -157,9 +157,9 @@ Input [2]: [c_count#X, count#X] Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(25) VeloxAppendBatches +(25) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] -Arguments: X +Arguments: X, X (26) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] @@ -186,9 +186,9 @@ Results [2]: [c_count#X, count(1)#X AS custdist#X] Input [2]: [c_count#X, custdist#X] Arguments: false -(32) VeloxAppendBatches +(32) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] -Arguments: X +Arguments: X, X (33) ColumnarExchange Input [2]: [c_count#X, custdist#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/14.txt index 901c481fac69a..ed116936e3e2f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/14.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (35) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) @@ -98,9 +98,9 @@ Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/15.txt index a87fdff4537d9..e7ac9b2efce16 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/15.txt @@ -5,7 +5,7 @@ AdaptiveSparkPlan (43) +- AQEShuffleRead (27) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ BroadcastHashJoinExecTransformer Inner BuildLeft (21) :- ^ InputIteratorTransformer (7) @@ -18,7 +18,7 @@ AdaptiveSparkPlan (43) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) @@ -100,9 +100,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] @@ -143,9 +143,9 @@ Input [6]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, supplier_no#X, total_ Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] @@ -249,7 +249,7 @@ AdaptiveSparkPlan (69) +- ^ InputIteratorTransformer (54) +- ShuffleQueryStage (52), Statistics(X) +- ColumnarExchange (51) - +- VeloxAppendBatches (50) + +- VeloxResizeBatches (50) +- ^ ProjectExecTransformer (48) +- ^ FlushableHashAggregateExecTransformer (47) +- ^ ProjectExecTransformer (46) @@ -296,9 +296,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(50) VeloxAppendBatches +(50) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (51) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/16.txt index e2f073b8908f0..2cc727f9782b6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/16.txt @@ -6,19 +6,19 @@ AdaptiveSparkPlan (59) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ RegularHashAggregateExecTransformer (20) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ ProjectExecTransformer (13) +- ^ FlushableHashAggregateExecTransformer (12) +- ^ ProjectExecTransformer (11) @@ -118,9 +118,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] @@ -158,9 +158,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, count#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] @@ -187,9 +187,9 @@ Results [4]: [p_brand#X, p_type#X, p_size#X, count(ps_suppkey#X)#X AS supplier_c Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/17.txt index 77733ad6f8a5b..77c18798faa24 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/17.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (36) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9), Statistics(X) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ FlushableHashAggregateExecTransformer (5) +- ^ InputIteratorTransformer (4) +- RowToVeloxColumnar (2) @@ -59,9 +59,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/18.txt index 4f36a185da726..88e9352841ac7 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/18.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (88) +- ^ InputIteratorTransformer (51) +- ShuffleQueryStage (49), Statistics(X) +- ColumnarExchange (48) - +- VeloxAppendBatches (47) + +- VeloxResizeBatches (47) +- ^ ProjectExecTransformer (45) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) @@ -31,7 +31,7 @@ AdaptiveSparkPlan (88) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FlushableHashAggregateExecTransformer (11) : +- ^ Scan parquet (10) @@ -140,9 +140,9 @@ Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] @@ -280,9 +280,9 @@ Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] Arguments: false -(47) VeloxAppendBatches +(47) VeloxResizeBatches Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (48) ColumnarExchange Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/19.txt index 7854a767b26bf..01b07807e5576 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/19.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (34) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) @@ -97,9 +97,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/20.txt index 981017da501e3..f4eea85b7b425 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/20.txt @@ -5,7 +5,7 @@ AdaptiveSparkPlan (98) +- AQEShuffleRead (61) +- ShuffleQueryStage (60), Statistics(X) +- ColumnarExchange (59) - +- VeloxAppendBatches (58) + +- VeloxResizeBatches (58) +- ^ ProjectExecTransformer (56) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (55) :- ^ ProjectExecTransformer (46) @@ -35,7 +35,7 @@ AdaptiveSparkPlan (98) : +- ^ InputIteratorTransformer (34) : +- ShuffleQueryStage (32), Statistics(X) : +- ColumnarExchange (31) - : +- VeloxAppendBatches (30) + : +- VeloxResizeBatches (30) : +- ^ ProjectExecTransformer (28) : +- ^ FlushableHashAggregateExecTransformer (27) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (26) @@ -217,9 +217,9 @@ Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(30) VeloxAppendBatches +(30) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (31) ColumnarExchange Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] @@ -335,9 +335,9 @@ Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] Input [2]: [s_name#X, s_address#X] Arguments: false -(58) VeloxAppendBatches +(58) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] -Arguments: X +Arguments: X, X (59) ColumnarExchange Input [2]: [s_name#X, s_address#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/21.txt index 5aef62e337655..ae73b6fd04232 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/21.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (92) +- ^ InputIteratorTransformer (56) +- ShuffleQueryStage (54), Statistics(X) +- ColumnarExchange (53) - +- VeloxAppendBatches (52) + +- VeloxResizeBatches (52) +- ^ ProjectExecTransformer (50) +- ^ FlushableHashAggregateExecTransformer (49) +- ^ ProjectExecTransformer (48) @@ -306,9 +306,9 @@ Input [2]: [s_name#X, count#X] Input [3]: [hash_partition_key#X, s_name#X, count#X] Arguments: false -(52) VeloxAppendBatches +(52) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, count#X] -Arguments: X +Arguments: X, X (53) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/22.txt index fbda7224d6424..5526710f89e17 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/22.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (40) +- ^ InputIteratorTransformer (25) +- ShuffleQueryStage (23), Statistics(X) +- ColumnarExchange (22) - +- VeloxAppendBatches (21) + +- VeloxResizeBatches (21) +- ^ RegularHashAggregateExecTransformer (19) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) @@ -96,9 +96,9 @@ Input [4]: [cntrycode#X, count#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] @@ -125,9 +125,9 @@ Results [3]: [cntrycode#X, count(1)#X AS numcust#X, sum(c_acctbal#X)#X AS totacc Input [3]: [cntrycode#X, numcust#X, totacctbal#X] Arguments: false -(21) VeloxAppendBatches +(21) VeloxResizeBatches Input [3]: [cntrycode#X, numcust#X, totacctbal#X] -Arguments: X +Arguments: X, X (22) ColumnarExchange Input [3]: [cntrycode#X, numcust#X, totacctbal#X] @@ -225,7 +225,7 @@ AdaptiveSparkPlan (60) +- ^ InputIteratorTransformer (50) +- ShuffleQueryStage (48), Statistics(X) +- ColumnarExchange (47) - +- VeloxAppendBatches (46) + +- VeloxResizeBatches (46) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) +- ^ FilterExecTransformer (42) @@ -265,9 +265,9 @@ Results [2]: [sum#X, count#X] Input [2]: [sum#X, count#X] Arguments: false -(46) VeloxAppendBatches +(46) VeloxResizeBatches Input [2]: [sum#X, count#X] -Arguments: X +Arguments: X, X (47) ColumnarExchange Input [2]: [sum#X, count#X] @@ -342,7 +342,7 @@ AdaptiveSparkPlan (60) +- ^ InputIteratorTransformer (50) +- ShuffleQueryStage (48), Statistics(X) +- ColumnarExchange (47) - +- VeloxAppendBatches (46) + +- VeloxResizeBatches (46) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) +- ^ FilterExecTransformer (42) diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/3.txt index 6d518ac272144..4a37e3343dfac 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/3.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (54) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ ProjectExecTransformer (22) @@ -162,9 +162,9 @@ Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/4.txt index bb6c149c39e1d..0bff0b16383f5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/4.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (46) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ RegularHashAggregateExecTransformer (22) +- ^ InputIteratorTransformer (21) +- ShuffleQueryStage (19), Statistics(X) +- ColumnarExchange (18) - +- VeloxAppendBatches (17) + +- VeloxResizeBatches (17) +- ^ ProjectExecTransformer (15) +- ^ FlushableHashAggregateExecTransformer (14) +- ^ ProjectExecTransformer (13) @@ -115,9 +115,9 @@ Input [2]: [o_orderpriority#X, count#X] Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] Arguments: false -(17) VeloxAppendBatches +(17) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] -Arguments: X +Arguments: X, X (18) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] @@ -144,9 +144,9 @@ Results [2]: [o_orderpriority#X, count(1)#X AS order_count#X] Input [2]: [o_orderpriority#X, order_count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [2]: [o_orderpriority#X, order_count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [2]: [o_orderpriority#X, order_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/5.txt index afb9aa3699667..ff4a7828c4b4e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/5.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (102) +- ^ InputIteratorTransformer (64) +- ShuffleQueryStage (62), Statistics(X) +- ColumnarExchange (61) - +- VeloxAppendBatches (60) + +- VeloxResizeBatches (60) +- ^ RegularHashAggregateExecTransformer (58) +- ^ InputIteratorTransformer (57) +- ShuffleQueryStage (55), Statistics(X) +- ColumnarExchange (54) - +- VeloxAppendBatches (53) + +- VeloxResizeBatches (53) +- ^ ProjectExecTransformer (51) +- ^ FlushableHashAggregateExecTransformer (50) +- ^ ProjectExecTransformer (49) @@ -319,9 +319,9 @@ Input [3]: [n_name#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] Arguments: false -(53) VeloxAppendBatches +(53) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (54) ColumnarExchange Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] @@ -348,9 +348,9 @@ Results [2]: [n_name#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X AS reven Input [2]: [n_name#X, revenue#X] Arguments: false -(60) VeloxAppendBatches +(60) VeloxResizeBatches Input [2]: [n_name#X, revenue#X] -Arguments: X +Arguments: X, X (61) ColumnarExchange Input [2]: [n_name#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/6.txt index ddc921e22d0f6..9463fa1da9d93 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/6.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (20) +- ^ InputIteratorTransformer (10) +- ShuffleQueryStage (8), Statistics(X) +- ColumnarExchange (7) - +- VeloxAppendBatches (6) + +- VeloxResizeBatches (6) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) @@ -46,9 +46,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(6) VeloxAppendBatches +(6) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (7) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/7.txt index 5ca9bbe39ef2f..0925850d92375 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/7.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (95) +- ^ InputIteratorTransformer (59) +- ShuffleQueryStage (57), Statistics(X) +- ColumnarExchange (56) - +- VeloxAppendBatches (55) + +- VeloxResizeBatches (55) +- ^ RegularHashAggregateExecTransformer (53) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50), Statistics(X) +- ColumnarExchange (49) - +- VeloxAppendBatches (48) + +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ FlushableHashAggregateExecTransformer (45) +- ^ ProjectExecTransformer (44) @@ -289,9 +289,9 @@ Input [5]: [supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] @@ -318,9 +318,9 @@ Results [4]: [supp_nation#X, cust_nation#X, l_year#X, sum(volume#X)#X AS revenue Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] Arguments: false -(55) VeloxAppendBatches +(55) VeloxResizeBatches Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] -Arguments: X +Arguments: X, X (56) ColumnarExchange Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/8.txt index 6c2c8eb46bc97..56fd86d255b44 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/8.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (131) +- ^ InputIteratorTransformer (83) +- ShuffleQueryStage (81), Statistics(X) +- ColumnarExchange (80) - +- VeloxAppendBatches (79) + +- VeloxResizeBatches (79) +- ^ ProjectExecTransformer (77) +- ^ RegularHashAggregateExecTransformer (76) +- ^ InputIteratorTransformer (75) +- ShuffleQueryStage (73), Statistics(X) +- ColumnarExchange (72) - +- VeloxAppendBatches (71) + +- VeloxResizeBatches (71) +- ^ ProjectExecTransformer (69) +- ^ FlushableHashAggregateExecTransformer (68) +- ^ ProjectExecTransformer (67) @@ -422,9 +422,9 @@ Input [5]: [o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(71) VeloxAppendBatches +(71) VeloxResizeBatches Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (72) ColumnarExchange Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] @@ -455,9 +455,9 @@ Input [3]: [o_year#X, sum(CASE WHEN (nation#X = BRAZIL) THEN volume#X ELSE 0.000 Input [2]: [o_year#X, mkt_share#X] Arguments: false -(79) VeloxAppendBatches +(79) VeloxResizeBatches Input [2]: [o_year#X, mkt_share#X] -Arguments: X +Arguments: X, X (80) ColumnarExchange Input [2]: [o_year#X, mkt_share#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/9.txt index 6a4faab249c87..1f25540dd1363 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/9.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (100) +- ^ InputIteratorTransformer (63) +- ShuffleQueryStage (61), Statistics(X) +- ColumnarExchange (60) - +- VeloxAppendBatches (59) + +- VeloxResizeBatches (59) +- ^ RegularHashAggregateExecTransformer (57) +- ^ InputIteratorTransformer (56) +- ShuffleQueryStage (54), Statistics(X) +- ColumnarExchange (53) - +- VeloxAppendBatches (52) + +- VeloxResizeBatches (52) +- ^ ProjectExecTransformer (50) +- ^ FlushableHashAggregateExecTransformer (49) +- ^ ProjectExecTransformer (48) @@ -313,9 +313,9 @@ Input [4]: [nation#X, o_year#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] Arguments: false -(52) VeloxAppendBatches +(52) VeloxResizeBatches Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (53) ColumnarExchange Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] @@ -342,9 +342,9 @@ Results [3]: [nation#X, o_year#X, sum(amount#X)#X AS sum_profit#X] Input [3]: [nation#X, o_year#X, sum_profit#X] Arguments: false -(59) VeloxAppendBatches +(59) VeloxResizeBatches Input [3]: [nation#X, o_year#X, sum_profit#X] -Arguments: X +Arguments: X, X (60) ColumnarExchange Input [3]: [nation#X, o_year#X, sum_profit#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/1.txt index 53edb933c1fb4..deb09c6c1c0f6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/1.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (30) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ RegularHashAggregateExecTransformer (12) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ ProjectExecTransformer (5) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) @@ -58,9 +58,9 @@ Input [17]: [l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] @@ -87,9 +87,9 @@ Results [10]: [l_returnflag#X, l_linestatus#X, sum(l_quantity#X)#X AS sum_qty#X, Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/10.txt index ec46bfd07b911..49e18f05aa63f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/10.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (100) +- ^ InputIteratorTransformer (62) +- ShuffleQueryStage (60) +- ColumnarExchange (59) - +- VeloxAppendBatches (58) + +- VeloxResizeBatches (58) +- ^ ProjectExecTransformer (56) +- ^ FlushableHashAggregateExecTransformer (55) +- ^ ProjectExecTransformer (54) @@ -16,40 +16,40 @@ AdaptiveSparkPlan (100) :- ^ InputIteratorTransformer (43) : +- ShuffleQueryStage (41) : +- ColumnarExchange (40) - : +- VeloxAppendBatches (39) + : +- VeloxResizeBatches (39) : +- ^ ProjectExecTransformer (37) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) : :- ^ InputIteratorTransformer (26) : : +- ShuffleQueryStage (24) : : +- ColumnarExchange (23) - : : +- VeloxAppendBatches (22) + : : +- VeloxResizeBatches (22) : : +- ^ ProjectExecTransformer (20) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) : : :- ^ InputIteratorTransformer (9) : : : +- ShuffleQueryStage (7) : : : +- ColumnarExchange (6) - : : : +- VeloxAppendBatches (5) + : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ NoopFilter (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16) : : +- ColumnarExchange (15) - : : +- VeloxAppendBatches (14) + : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ NoopFilter (11) : : +- ^ Scan parquet (10) : +- ^ InputIteratorTransformer (35) : +- ShuffleQueryStage (33) : +- ColumnarExchange (32) - : +- VeloxAppendBatches (31) + : +- VeloxResizeBatches (31) : +- ^ ProjectExecTransformer (29) : +- ^ NoopFilter (28) : +- ^ Scan parquet (27) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50) +- ColumnarExchange (49) - +- VeloxAppendBatches (48) + +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ NoopFilter (45) +- ^ Scan parquet (44) @@ -107,9 +107,9 @@ Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] @@ -144,9 +144,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -175,9 +175,9 @@ Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] @@ -212,9 +212,9 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] @@ -243,9 +243,9 @@ Input [11]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acc Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] @@ -280,9 +280,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -322,9 +322,9 @@ Input [9]: [c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] Arguments: false -(58) VeloxAppendBatches +(58) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (59) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/11.txt index cccf1408bea99..7aff321e6ae69 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/11.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (82) +- ^ InputIteratorTransformer (53) +- ShuffleQueryStage (51) +- ColumnarExchange (50) - +- VeloxAppendBatches (49) + +- VeloxResizeBatches (49) +- ^ FilterExecTransformer (47) +- ^ RegularHashAggregateExecTransformer (46) +- ^ InputIteratorTransformer (45) +- ShuffleQueryStage (43) +- ColumnarExchange (42) - +- VeloxAppendBatches (41) + +- VeloxResizeBatches (41) +- ^ ProjectExecTransformer (39) +- ^ FlushableHashAggregateExecTransformer (38) +- ^ ProjectExecTransformer (37) @@ -20,27 +20,27 @@ AdaptiveSparkPlan (82) :- ^ InputIteratorTransformer (26) : +- ShuffleQueryStage (24) : +- ColumnarExchange (23) - : +- VeloxAppendBatches (22) + : +- VeloxResizeBatches (22) : +- ^ ProjectExecTransformer (20) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ NoopFilter (11) : +- ^ Scan parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ NoopFilter (28) +- ^ Scan parquet (27) @@ -91,9 +91,9 @@ Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] @@ -128,9 +128,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -159,9 +159,9 @@ Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppke Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] @@ -196,9 +196,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -238,9 +238,9 @@ Input [3]: [ps_partkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] Arguments: false -(41) VeloxAppendBatches +(41) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (42) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] @@ -271,9 +271,9 @@ Arguments: (isnotnull(value#X) AND (cast(value#X as decimal(38,6)) > Subquery su Input [2]: [ps_partkey#X, value#X] Arguments: false -(49) VeloxAppendBatches +(49) VeloxResizeBatches Input [2]: [ps_partkey#X, value#X] -Arguments: X +Arguments: X, X (50) ColumnarExchange Input [2]: [ps_partkey#X, value#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/12.txt index 17cdf62608cc5..0ac613d3792b5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/12.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (55) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ NoopFilter (11) +- ^ Scan parquet (10) @@ -68,9 +68,9 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] @@ -105,9 +105,9 @@ Input [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipm Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] @@ -147,9 +147,9 @@ Input [3]: [l_shipmode#X, sum#X, sum#X] Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] @@ -176,9 +176,9 @@ Results [3]: [l_shipmode#X, sum(CASE WHEN ((o_orderpriority#X = 1-URGENT) OR (o_ Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/13.txt index 730f0e0a438a5..de18e59ae904b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/13.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (58) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ RegularHashAggregateExecTransformer (31) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ ProjectExecTransformer (22) @@ -22,13 +22,13 @@ AdaptiveSparkPlan (58) :- ^ InputIteratorTransformer (8) : +- ShuffleQueryStage (6) : +- ColumnarExchange (5) - : +- VeloxAppendBatches (4) + : +- VeloxResizeBatches (4) : +- ^ ProjectExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15) +- ColumnarExchange (14) - +- VeloxAppendBatches (13) + +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) +- ^ NoopFilter (10) +- ^ Scan parquet (9) @@ -66,9 +66,9 @@ Input [1]: [c_custkey#X] Input [2]: [hash_partition_key#X, c_custkey#X] Arguments: false -(4) VeloxAppendBatches +(4) VeloxResizeBatches Input [2]: [hash_partition_key#X, c_custkey#X] -Arguments: X +Arguments: X, X (5) ColumnarExchange Input [2]: [hash_partition_key#X, c_custkey#X] @@ -103,9 +103,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(13) VeloxAppendBatches +(13) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (14) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -163,9 +163,9 @@ Input [2]: [c_count#X, count#X] Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] @@ -192,9 +192,9 @@ Results [2]: [c_count#X, count(1)#X AS custdist#X] Input [2]: [c_count#X, custdist#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [2]: [c_count#X, custdist#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/14.txt index 55111a31f8740..66c324638566c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/14.txt @@ -10,14 +10,14 @@ AdaptiveSparkPlan (39) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ NoopFilter (11) +- ^ Scan parquet (10) @@ -56,9 +56,9 @@ Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] @@ -93,9 +93,9 @@ Input [2]: [p_partkey#X, p_type#X] Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/15.txt index db2df6c875443..3aac8d913ac72 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/15.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (50) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (23) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) @@ -21,7 +21,7 @@ AdaptiveSparkPlan (50) +- ^ InputIteratorTransformer (20) +- ShuffleQueryStage (18) +- ColumnarExchange (17) - +- VeloxAppendBatches (16) + +- VeloxResizeBatches (16) +- ^ ProjectExecTransformer (14) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) @@ -65,9 +65,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] @@ -113,9 +113,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(16) VeloxAppendBatches +(16) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (17) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] @@ -155,9 +155,9 @@ Input [6]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, supplier_no#X, total_ Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/16.txt index 2eb5668906ba9..abf71575d8541 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/16.txt @@ -6,19 +6,19 @@ AdaptiveSparkPlan (71) +- ^ InputIteratorTransformer (44) +- ShuffleQueryStage (42) +- ColumnarExchange (41) - +- VeloxAppendBatches (40) + +- VeloxResizeBatches (40) +- ^ RegularHashAggregateExecTransformer (38) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ ProjectExecTransformer (31) +- ^ FlushableHashAggregateExecTransformer (30) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -26,14 +26,14 @@ AdaptiveSparkPlan (71) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ NoopFilter (11) +- ^ Scan parquet (10) @@ -82,9 +82,9 @@ Input [2]: [ps_partkey#X, ps_suppkey#X] Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] @@ -119,9 +119,9 @@ Input [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] @@ -161,9 +161,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] @@ -201,9 +201,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, count#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] @@ -230,9 +230,9 @@ Results [4]: [p_brand#X, p_type#X, p_size#X, count(ps_suppkey#X)#X AS supplier_c Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/17.txt index 5226aacff7531..b3b530a92eb6a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/17.txt @@ -12,14 +12,14 @@ AdaptiveSparkPlan (63) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ NoopFilter (11) : +- ^ Scan parquet (10) @@ -29,7 +29,7 @@ AdaptiveSparkPlan (63) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ NoopFilter (22) @@ -78,9 +78,9 @@ Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] @@ -115,9 +115,9 @@ Input [3]: [p_partkey#X, p_brand#X, p_container#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -168,9 +168,9 @@ Input [3]: [l_partkey#X, sum#X, count#X] Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/18.txt index c1287b2d685a4..7845c0868dc36 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/18.txt @@ -10,26 +10,26 @@ AdaptiveSparkPlan (110) :- ^ InputIteratorTransformer (46) : +- ShuffleQueryStage (44) : +- ColumnarExchange (43) - : +- VeloxAppendBatches (42) + : +- VeloxResizeBatches (42) : +- ^ ProjectExecTransformer (40) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (38) : +- ShuffleQueryStage (36) : +- ColumnarExchange (35) - : +- VeloxAppendBatches (34) + : +- VeloxResizeBatches (34) : +- ^ ProjectExecTransformer (32) : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) : :- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16) : : +- ColumnarExchange (15) - : : +- VeloxAppendBatches (14) + : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ NoopFilter (11) : : +- ^ Scan parquet (10) @@ -39,7 +39,7 @@ AdaptiveSparkPlan (110) : +- ^ InputIteratorTransformer (27) : +- ShuffleQueryStage (25) : +- ColumnarExchange (24) - : +- VeloxAppendBatches (23) + : +- VeloxResizeBatches (23) : +- ^ ProjectExecTransformer (21) : +- ^ FlushableHashAggregateExecTransformer (20) : +- ^ Scan parquet (19) @@ -47,7 +47,7 @@ AdaptiveSparkPlan (110) :- ^ InputIteratorTransformer (55) : +- ShuffleQueryStage (53) : +- ColumnarExchange (52) - : +- VeloxAppendBatches (51) + : +- VeloxResizeBatches (51) : +- ^ ProjectExecTransformer (49) : +- ^ NoopFilter (48) : +- ^ Scan parquet (47) @@ -118,9 +118,9 @@ Input [2]: [c_custkey#X, c_name#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] @@ -155,9 +155,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] @@ -194,9 +194,9 @@ Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] @@ -240,9 +240,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(34) VeloxAppendBatches +(34) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (35) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] @@ -271,9 +271,9 @@ Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_ Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(42) VeloxAppendBatches +(42) VeloxResizeBatches Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (43) ColumnarExchange Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] @@ -308,9 +308,9 @@ Input [2]: [l_orderkey#X, l_quantity#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] Arguments: false -(51) VeloxAppendBatches +(51) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] -Arguments: X +Arguments: X, X (52) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/19.txt index 21e4f472f3b3a..a6fc173e233bb 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/19.txt @@ -9,14 +9,14 @@ AdaptiveSparkPlan (38) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ NoopFilter (11) +- ^ Scan parquet (10) @@ -55,9 +55,9 @@ Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipin Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -92,9 +92,9 @@ Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/20.txt index 1ac0992834eb2..46cf1c79418f0 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/20.txt @@ -6,52 +6,52 @@ AdaptiveSparkPlan (146) +- ^ InputIteratorTransformer (93) +- ShuffleQueryStage (91) +- ColumnarExchange (90) - +- VeloxAppendBatches (89) + +- VeloxResizeBatches (89) +- ^ ProjectExecTransformer (87) +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (86) :- ^ InputIteratorTransformer (76) : +- ShuffleQueryStage (74) : +- ColumnarExchange (73) - : +- VeloxAppendBatches (72) + : +- VeloxResizeBatches (72) : +- ^ ProjectExecTransformer (70) : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (69) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (68) : +- ShuffleQueryStage (66) : +- ColumnarExchange (65) - : +- VeloxAppendBatches (64) + : +- VeloxResizeBatches (64) : +- ^ ProjectExecTransformer (62) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (61) : :- ^ InputIteratorTransformer (35) : : +- ShuffleQueryStage (33) : : +- ColumnarExchange (32) - : : +- VeloxAppendBatches (31) + : : +- VeloxResizeBatches (31) : : +- ^ ProjectExecTransformer (29) : : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (28) : : :- ^ InputIteratorTransformer (18) : : : +- ShuffleQueryStage (16) : : : +- ColumnarExchange (15) - : : : +- VeloxAppendBatches (14) + : : : +- VeloxResizeBatches (14) : : : +- ^ ProjectExecTransformer (12) : : : +- ^ NoopFilter (11) : : : +- ^ Scan parquet (10) : : +- ^ InputIteratorTransformer (27) : : +- ShuffleQueryStage (25) : : +- ColumnarExchange (24) - : : +- VeloxAppendBatches (23) + : : +- VeloxResizeBatches (23) : : +- ^ ProjectExecTransformer (21) : : +- ^ NoopFilter (20) : : +- ^ Scan parquet (19) : +- ^ InputIteratorTransformer (60) : +- ShuffleQueryStage (58) : +- ColumnarExchange (57) - : +- VeloxAppendBatches (56) + : +- VeloxResizeBatches (56) : +- ^ ProjectExecTransformer (54) : +- ^ FilterExecTransformer (53) : +- ^ ProjectExecTransformer (52) @@ -61,7 +61,7 @@ AdaptiveSparkPlan (146) : :- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42) : : +- ColumnarExchange (41) - : : +- VeloxAppendBatches (40) + : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) : : +- ^ NoopFilter (37) : : +- ^ Scan parquet (36) @@ -71,7 +71,7 @@ AdaptiveSparkPlan (146) +- ^ InputIteratorTransformer (85) +- ShuffleQueryStage (83) +- ColumnarExchange (82) - +- VeloxAppendBatches (81) + +- VeloxResizeBatches (81) +- ^ ProjectExecTransformer (79) +- ^ NoopFilter (78) +- ^ Scan parquet (77) @@ -146,9 +146,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] @@ -183,9 +183,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] @@ -220,9 +220,9 @@ Input [2]: [p_partkey#X, p_name#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -251,9 +251,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] @@ -288,9 +288,9 @@ Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] @@ -354,9 +354,9 @@ Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] @@ -385,9 +385,9 @@ Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: false -(64) VeloxAppendBatches +(64) VeloxResizeBatches Input [2]: [hash_partition_key#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (65) ColumnarExchange Input [2]: [hash_partition_key#X, ps_suppkey#X] @@ -416,9 +416,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(72) VeloxAppendBatches +(72) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (73) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] @@ -453,9 +453,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(81) VeloxAppendBatches +(81) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (82) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -484,9 +484,9 @@ Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] Input [2]: [s_name#X, s_address#X] Arguments: false -(89) VeloxAppendBatches +(89) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] -Arguments: X +Arguments: X, X (90) ColumnarExchange Input [2]: [s_name#X, s_address#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/21.txt index e2a72528c4ed4..e1e46cf28ffdd 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/21.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (138) +- ^ InputIteratorTransformer (88) +- ShuffleQueryStage (86) +- ColumnarExchange (85) - +- VeloxAppendBatches (84) + +- VeloxResizeBatches (84) +- ^ ProjectExecTransformer (82) +- ^ FlushableHashAggregateExecTransformer (81) +- ^ ProjectExecTransformer (80) @@ -15,60 +15,60 @@ AdaptiveSparkPlan (138) :- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (62) : :- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (45) : : :- ^ InputIteratorTransformer (9) : : : +- ShuffleQueryStage (7) : : : +- ColumnarExchange (6) - : : : +- VeloxAppendBatches (5) + : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ NoopFilter (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42) : : +- ColumnarExchange (41) - : : +- VeloxAppendBatches (40) + : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) : : +- ^ ShuffledHashJoinExecTransformer LeftAnti BuildRight (37) : : :- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (27) : : : :- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ NoopFilter (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (26) : : : +- ShuffleQueryStage (24) : : : +- ColumnarExchange (23) - : : : +- VeloxAppendBatches (22) + : : : +- VeloxResizeBatches (22) : : : +- ^ ProjectExecTransformer (20) : : : +- ^ Scan parquet (19) : : +- ^ InputIteratorTransformer (36) : : +- ShuffleQueryStage (34) : : +- ColumnarExchange (33) - : : +- VeloxAppendBatches (32) + : : +- VeloxResizeBatches (32) : : +- ^ ProjectExecTransformer (30) : : +- ^ NoopFilter (29) : : +- ^ Scan parquet (28) : +- ^ InputIteratorTransformer (61) : +- ShuffleQueryStage (59) : +- ColumnarExchange (58) - : +- VeloxAppendBatches (57) + : +- VeloxResizeBatches (57) : +- ^ ProjectExecTransformer (55) : +- ^ NoopFilter (54) : +- ^ Scan parquet (53) +- ^ InputIteratorTransformer (78) +- ShuffleQueryStage (76) +- ColumnarExchange (75) - +- VeloxAppendBatches (74) + +- VeloxResizeBatches (74) +- ^ ProjectExecTransformer (72) +- ^ NoopFilter (71) +- ^ Scan parquet (70) @@ -139,9 +139,9 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] @@ -176,9 +176,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -208,9 +208,9 @@ Input [2]: [l_orderkey#X, l_suppkey#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -250,9 +250,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(32) VeloxAppendBatches +(32) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (33) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -281,9 +281,9 @@ Input [2]: [l_orderkey#X, l_suppkey#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -312,9 +312,9 @@ Input [5]: [s_suppkey#X, s_name#X, s_nationkey#X, l_orderkey#X, l_suppkey#X] Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] @@ -349,9 +349,9 @@ Input [2]: [o_orderkey#X, o_orderstatus#X] Input [2]: [hash_partition_key#X, o_orderkey#X] Arguments: false -(57) VeloxAppendBatches +(57) VeloxResizeBatches Input [2]: [hash_partition_key#X, o_orderkey#X] -Arguments: X +Arguments: X, X (58) ColumnarExchange Input [2]: [hash_partition_key#X, o_orderkey#X] @@ -380,9 +380,9 @@ Input [4]: [s_name#X, s_nationkey#X, l_orderkey#X, o_orderkey#X] Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] @@ -417,9 +417,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(74) VeloxAppendBatches +(74) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (75) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -459,9 +459,9 @@ Input [2]: [s_name#X, count#X] Input [3]: [hash_partition_key#X, s_name#X, count#X] Arguments: false -(84) VeloxAppendBatches +(84) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, count#X] -Arguments: X +Arguments: X, X (85) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/22.txt index 984abd470378b..8582ecdb64d43 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/22.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (52) +- ^ InputIteratorTransformer (34) +- ShuffleQueryStage (32) +- ColumnarExchange (31) - +- VeloxAppendBatches (30) + +- VeloxResizeBatches (30) +- ^ RegularHashAggregateExecTransformer (28) +- ^ InputIteratorTransformer (27) +- ShuffleQueryStage (25) +- ColumnarExchange (24) - +- VeloxAppendBatches (23) + +- VeloxResizeBatches (23) +- ^ ProjectExecTransformer (21) +- ^ FlushableHashAggregateExecTransformer (20) +- ^ ProjectExecTransformer (19) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (52) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15) +- ColumnarExchange (14) - +- VeloxAppendBatches (13) + +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) +- ^ Scan parquet (10) +- == Initial Plan == @@ -65,9 +65,9 @@ Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] @@ -97,9 +97,9 @@ Input [1]: [o_custkey#X] Input [2]: [hash_partition_key#X, o_custkey#X] Arguments: false -(13) VeloxAppendBatches +(13) VeloxResizeBatches Input [2]: [hash_partition_key#X, o_custkey#X] -Arguments: X +Arguments: X, X (14) ColumnarExchange Input [2]: [hash_partition_key#X, o_custkey#X] @@ -139,9 +139,9 @@ Input [4]: [cntrycode#X, count#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] @@ -168,9 +168,9 @@ Results [3]: [cntrycode#X, count(1)#X AS numcust#X, sum(c_acctbal#X)#X AS totacc Input [3]: [cntrycode#X, numcust#X, totacctbal#X] Arguments: false -(30) VeloxAppendBatches +(30) VeloxResizeBatches Input [3]: [cntrycode#X, numcust#X, totacctbal#X] -Arguments: X +Arguments: X, X (31) ColumnarExchange Input [3]: [cntrycode#X, numcust#X, totacctbal#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/3.txt index 58484edaa685a..f1f128ce00fbf 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/3.txt @@ -11,27 +11,27 @@ AdaptiveSparkPlan (67) :- ^ InputIteratorTransformer (26) : +- ShuffleQueryStage (24) : +- ColumnarExchange (23) - : +- VeloxAppendBatches (22) + : +- VeloxResizeBatches (22) : +- ^ ProjectExecTransformer (20) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ NoopFilter (11) : +- ^ Scan parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ NoopFilter (28) +- ^ Scan parquet (27) @@ -80,9 +80,9 @@ Input [2]: [c_custkey#X, c_mktsegment#X] Input [2]: [hash_partition_key#X, c_custkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, c_custkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, c_custkey#X] @@ -117,9 +117,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] @@ -148,9 +148,9 @@ Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriorit Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] @@ -185,9 +185,9 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/4.txt index cb7a3c3a0955c..6eb069e562dec 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/4.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (56) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (56) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ NoopFilter (11) +- ^ Scan parquet (10) @@ -69,9 +69,9 @@ Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] @@ -106,9 +106,9 @@ Input [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Input [2]: [hash_partition_key#X, l_orderkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [2]: [hash_partition_key#X, l_orderkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [2]: [hash_partition_key#X, l_orderkey#X] @@ -148,9 +148,9 @@ Input [2]: [o_orderpriority#X, count#X] Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] @@ -177,9 +177,9 @@ Results [2]: [o_orderpriority#X, count(1)#X AS order_count#X] Input [2]: [o_orderpriority#X, order_count#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [2]: [o_orderpriority#X, order_count#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [2]: [o_orderpriority#X, order_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/5.txt index 930a5a0bf4886..1c71444a83d4f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/5.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (156) +- ^ InputIteratorTransformer (103) +- ShuffleQueryStage (101) +- ColumnarExchange (100) - +- VeloxAppendBatches (99) + +- VeloxResizeBatches (99) +- ^ RegularHashAggregateExecTransformer (97) +- ^ InputIteratorTransformer (96) +- ShuffleQueryStage (94) +- ColumnarExchange (93) - +- VeloxAppendBatches (92) + +- VeloxResizeBatches (92) +- ^ ProjectExecTransformer (90) +- ^ FlushableHashAggregateExecTransformer (89) +- ^ ProjectExecTransformer (88) @@ -19,66 +19,66 @@ AdaptiveSparkPlan (156) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ NoopFilter (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ NoopFilter (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ NoopFilter (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ NoopFilter (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ NoopFilter (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84) +- ColumnarExchange (83) - +- VeloxAppendBatches (82) + +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ NoopFilter (79) +- ^ Scan parquet (78) @@ -153,9 +153,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -190,9 +190,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -221,9 +221,9 @@ Input [4]: [c_custkey#X, c_nationkey#X, o_orderkey#X, o_custkey#X] Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] @@ -258,9 +258,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -289,9 +289,9 @@ Input [6]: [c_nationkey#X, o_orderkey#X, l_orderkey#X, l_suppkey#X, l_extendedpr Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -326,9 +326,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -357,9 +357,9 @@ Input [6]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppk Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -394,9 +394,9 @@ Input [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] @@ -425,9 +425,9 @@ Input [6]: [l_extendedprice#X, l_discount#X, s_nationkey#X, n_nationkey#X, n_nam Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] @@ -462,9 +462,9 @@ Input [2]: [r_regionkey#X, r_name#X] Input [2]: [hash_partition_key#X, r_regionkey#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [2]: [hash_partition_key#X, r_regionkey#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [2]: [hash_partition_key#X, r_regionkey#X] @@ -504,9 +504,9 @@ Input [3]: [n_name#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] Arguments: false -(92) VeloxAppendBatches +(92) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (93) ColumnarExchange Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] @@ -533,9 +533,9 @@ Results [2]: [n_name#X, sum(CheckOverflow((promote_precision(cast(l_extendedpric Input [2]: [n_name#X, revenue#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [2]: [n_name#X, revenue#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [2]: [n_name#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/6.txt index b6f876d48e5a0..f30bcb0da8014 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/6.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (20) +- ^ InputIteratorTransformer (10) +- ShuffleQueryStage (8) +- ColumnarExchange (7) - +- VeloxAppendBatches (6) + +- VeloxResizeBatches (6) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ NoopFilter (2) @@ -46,9 +46,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(6) VeloxAppendBatches +(6) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (7) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/7.txt index d9eb23cb737e7..0179c5ce37f1d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/7.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (149) +- ^ InputIteratorTransformer (98) +- ShuffleQueryStage (96) +- ColumnarExchange (95) - +- VeloxAppendBatches (94) + +- VeloxResizeBatches (94) +- ^ RegularHashAggregateExecTransformer (92) +- ^ InputIteratorTransformer (91) +- ShuffleQueryStage (89) +- ColumnarExchange (88) - +- VeloxAppendBatches (87) + +- VeloxResizeBatches (87) +- ^ ProjectExecTransformer (85) +- ^ FlushableHashAggregateExecTransformer (84) +- ^ ProjectExecTransformer (83) @@ -19,59 +19,59 @@ AdaptiveSparkPlan (149) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ NoopFilter (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ NoopFilter (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ NoopFilter (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ NoopFilter (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ NoopFilter (62) : +- ^ Scan parquet (61) @@ -147,9 +147,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -184,9 +184,9 @@ Input [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipda Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] @@ -215,9 +215,9 @@ Input [7]: [s_suppkey#X, s_nationkey#X, l_orderkey#X, l_suppkey#X, l_extendedpri Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] @@ -252,9 +252,9 @@ Input [2]: [o_orderkey#X, o_custkey#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -283,9 +283,9 @@ Input [7]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_ship Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] @@ -320,9 +320,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -351,9 +351,9 @@ Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_cust Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] @@ -388,9 +388,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -419,9 +419,9 @@ Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nati Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] @@ -474,9 +474,9 @@ Input [5]: [supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Arguments: false -(87) VeloxAppendBatches +(87) VeloxResizeBatches Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (88) ColumnarExchange Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] @@ -503,9 +503,9 @@ Results [4]: [supp_nation#X, cust_nation#X, l_year#X, sum(volume#X)#X AS revenue Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] Arguments: false -(94) VeloxAppendBatches +(94) VeloxResizeBatches Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] -Arguments: X +Arguments: X, X (95) ColumnarExchange Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/8.txt index 5c9e51b95c606..ce21c6ca2cae4 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/8.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (207) +- ^ InputIteratorTransformer (138) +- ShuffleQueryStage (136) +- ColumnarExchange (135) - +- VeloxAppendBatches (134) + +- VeloxResizeBatches (134) +- ^ ProjectExecTransformer (132) +- ^ RegularHashAggregateExecTransformer (131) +- ^ InputIteratorTransformer (130) +- ShuffleQueryStage (128) +- ColumnarExchange (127) - +- VeloxAppendBatches (126) + +- VeloxResizeBatches (126) +- ^ ProjectExecTransformer (124) +- ^ FlushableHashAggregateExecTransformer (123) +- ^ ProjectExecTransformer (122) @@ -20,92 +20,92 @@ AdaptiveSparkPlan (207) :- ^ InputIteratorTransformer (111) : +- ShuffleQueryStage (109) : +- ColumnarExchange (108) - : +- VeloxAppendBatches (107) + : +- VeloxResizeBatches (107) : +- ^ ProjectExecTransformer (105) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (104) : :- ^ InputIteratorTransformer (94) : : +- ShuffleQueryStage (92) : : +- ColumnarExchange (91) - : : +- VeloxAppendBatches (90) + : : +- VeloxResizeBatches (90) : : +- ^ ProjectExecTransformer (88) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (87) : : :- ^ InputIteratorTransformer (77) : : : +- ShuffleQueryStage (75) : : : +- ColumnarExchange (74) - : : : +- VeloxAppendBatches (73) + : : : +- VeloxResizeBatches (73) : : : +- ^ ProjectExecTransformer (71) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (70) : : : :- ^ InputIteratorTransformer (60) : : : : +- ShuffleQueryStage (58) : : : : +- ColumnarExchange (57) - : : : : +- VeloxAppendBatches (56) + : : : : +- VeloxResizeBatches (56) : : : : +- ^ ProjectExecTransformer (54) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (53) : : : : :- ^ InputIteratorTransformer (43) : : : : : +- ShuffleQueryStage (41) : : : : : +- ColumnarExchange (40) - : : : : : +- VeloxAppendBatches (39) + : : : : : +- VeloxResizeBatches (39) : : : : : +- ^ ProjectExecTransformer (37) : : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : : : :- ^ InputIteratorTransformer (26) : : : : : : +- ShuffleQueryStage (24) : : : : : : +- ColumnarExchange (23) - : : : : : : +- VeloxAppendBatches (22) + : : : : : : +- VeloxResizeBatches (22) : : : : : : +- ^ ProjectExecTransformer (20) : : : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : : : :- ^ InputIteratorTransformer (9) : : : : : : : +- ShuffleQueryStage (7) : : : : : : : +- ColumnarExchange (6) - : : : : : : : +- VeloxAppendBatches (5) + : : : : : : : +- VeloxResizeBatches (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ NoopFilter (2) : : : : : : : +- ^ Scan parquet (1) : : : : : : +- ^ InputIteratorTransformer (18) : : : : : : +- ShuffleQueryStage (16) : : : : : : +- ColumnarExchange (15) - : : : : : : +- VeloxAppendBatches (14) + : : : : : : +- VeloxResizeBatches (14) : : : : : : +- ^ ProjectExecTransformer (12) : : : : : : +- ^ NoopFilter (11) : : : : : : +- ^ Scan parquet (10) : : : : : +- ^ InputIteratorTransformer (35) : : : : : +- ShuffleQueryStage (33) : : : : : +- ColumnarExchange (32) - : : : : : +- VeloxAppendBatches (31) + : : : : : +- VeloxResizeBatches (31) : : : : : +- ^ ProjectExecTransformer (29) : : : : : +- ^ NoopFilter (28) : : : : : +- ^ Scan parquet (27) : : : : +- ^ InputIteratorTransformer (52) : : : : +- ShuffleQueryStage (50) : : : : +- ColumnarExchange (49) - : : : : +- VeloxAppendBatches (48) + : : : : +- VeloxResizeBatches (48) : : : : +- ^ ProjectExecTransformer (46) : : : : +- ^ NoopFilter (45) : : : : +- ^ Scan parquet (44) : : : +- ^ InputIteratorTransformer (69) : : : +- ShuffleQueryStage (67) : : : +- ColumnarExchange (66) - : : : +- VeloxAppendBatches (65) + : : : +- VeloxResizeBatches (65) : : : +- ^ ProjectExecTransformer (63) : : : +- ^ NoopFilter (62) : : : +- ^ Scan parquet (61) : : +- ^ InputIteratorTransformer (86) : : +- ShuffleQueryStage (84) : : +- ColumnarExchange (83) - : : +- VeloxAppendBatches (82) + : : +- VeloxResizeBatches (82) : : +- ^ ProjectExecTransformer (80) : : +- ^ NoopFilter (79) : : +- ^ Scan parquet (78) : +- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101) : +- ColumnarExchange (100) - : +- VeloxAppendBatches (99) + : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ NoopFilter (96) : +- ^ Scan parquet (95) +- ^ InputIteratorTransformer (120) +- ShuffleQueryStage (118) +- ColumnarExchange (117) - +- VeloxAppendBatches (116) + +- VeloxResizeBatches (116) +- ^ ProjectExecTransformer (114) +- ^ NoopFilter (113) +- ^ Scan parquet (112) @@ -196,9 +196,9 @@ Input [2]: [p_partkey#X, p_type#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -233,9 +233,9 @@ Input [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discoun Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -264,9 +264,9 @@ Input [6]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -301,9 +301,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -332,9 +332,9 @@ Input [6]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppke Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -369,9 +369,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] @@ -400,9 +400,9 @@ Input [7]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orde Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] @@ -437,9 +437,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -468,9 +468,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_order Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] @@ -505,9 +505,9 @@ Input [2]: [n_nationkey#X, n_regionkey#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] @@ -536,9 +536,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nat Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] Arguments: false -(90) VeloxAppendBatches +(90) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] -Arguments: X +Arguments: X, X (91) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] @@ -573,9 +573,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -604,9 +604,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_reg Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] Arguments: false -(107) VeloxAppendBatches +(107) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] -Arguments: X +Arguments: X, X (108) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] @@ -641,9 +641,9 @@ Input [2]: [r_regionkey#X, r_name#X] Input [2]: [hash_partition_key#X, r_regionkey#X] Arguments: false -(116) VeloxAppendBatches +(116) VeloxResizeBatches Input [2]: [hash_partition_key#X, r_regionkey#X] -Arguments: X +Arguments: X, X (117) ColumnarExchange Input [2]: [hash_partition_key#X, r_regionkey#X] @@ -683,9 +683,9 @@ Input [5]: [o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(126) VeloxAppendBatches +(126) VeloxResizeBatches Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (127) ColumnarExchange Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] @@ -716,9 +716,9 @@ Input [3]: [o_year#X, sum(CASE WHEN (nation#X = BRAZIL) THEN volume#X ELSE 0.000 Input [2]: [o_year#X, mkt_share#X] Arguments: false -(134) VeloxAppendBatches +(134) VeloxResizeBatches Input [2]: [o_year#X, mkt_share#X] -Arguments: X +Arguments: X, X (135) ColumnarExchange Input [2]: [o_year#X, mkt_share#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/9.txt index 2abb6ec215c6c..06d8d6e0ef000 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/9.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (155) +- ^ InputIteratorTransformer (103) +- ShuffleQueryStage (101) +- ColumnarExchange (100) - +- VeloxAppendBatches (99) + +- VeloxResizeBatches (99) +- ^ RegularHashAggregateExecTransformer (97) +- ^ InputIteratorTransformer (96) +- ShuffleQueryStage (94) +- ColumnarExchange (93) - +- VeloxAppendBatches (92) + +- VeloxResizeBatches (92) +- ^ ProjectExecTransformer (90) +- ^ FlushableHashAggregateExecTransformer (89) +- ^ ProjectExecTransformer (88) @@ -19,66 +19,66 @@ AdaptiveSparkPlan (155) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ NoopFilter (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ NoopFilter (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ NoopFilter (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ NoopFilter (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ NoopFilter (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84) +- ColumnarExchange (83) - +- VeloxAppendBatches (82) + +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ NoopFilter (79) +- ^ Scan parquet (78) @@ -152,9 +152,9 @@ Input [2]: [p_partkey#X, p_name#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -189,9 +189,9 @@ Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -220,9 +220,9 @@ Input [7]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -257,9 +257,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -288,9 +288,9 @@ Input [8]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -325,9 +325,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] @@ -356,9 +356,9 @@ Input [10]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpri Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] @@ -393,9 +393,9 @@ Input [2]: [o_orderkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] @@ -424,9 +424,9 @@ Input [8]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_natio Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] @@ -461,9 +461,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -503,9 +503,9 @@ Input [4]: [nation#X, o_year#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] Arguments: false -(92) VeloxAppendBatches +(92) VeloxResizeBatches Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (93) ColumnarExchange Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] @@ -532,9 +532,9 @@ Results [3]: [nation#X, o_year#X, sum(amount#X)#X AS sum_profit#X] Input [3]: [nation#X, o_year#X, sum_profit#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [3]: [nation#X, o_year#X, sum_profit#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [3]: [nation#X, o_year#X, sum_profit#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/1.txt index afea15af53d24..159a1598c9bf5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/1.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (30) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ RegularHashAggregateExecTransformer (12) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9), Statistics(X) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ ProjectExecTransformer (5) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) @@ -58,9 +58,9 @@ Input [17]: [l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] @@ -87,9 +87,9 @@ Results [10]: [l_returnflag#X, l_linestatus#X, sum(l_quantity#X)#X AS sum_qty#X, Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/10.txt index 3be5f1996fa8c..35cf0c574fde7 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/10.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (100) +- ^ InputIteratorTransformer (62) +- ShuffleQueryStage (60), Statistics(X) +- ColumnarExchange (59) - +- VeloxAppendBatches (58) + +- VeloxResizeBatches (58) +- ^ ProjectExecTransformer (56) +- ^ FlushableHashAggregateExecTransformer (55) +- ^ ProjectExecTransformer (54) @@ -16,40 +16,40 @@ AdaptiveSparkPlan (100) :- ^ InputIteratorTransformer (43) : +- ShuffleQueryStage (41), Statistics(X) : +- ColumnarExchange (40) - : +- VeloxAppendBatches (39) + : +- VeloxResizeBatches (39) : +- ^ ProjectExecTransformer (37) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) : :- ^ InputIteratorTransformer (26) : : +- ShuffleQueryStage (24), Statistics(X) : : +- ColumnarExchange (23) - : : +- VeloxAppendBatches (22) + : : +- VeloxResizeBatches (22) : : +- ^ ProjectExecTransformer (20) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) : : :- ^ InputIteratorTransformer (9) : : : +- ShuffleQueryStage (7), Statistics(X) : : : +- ColumnarExchange (6) - : : : +- VeloxAppendBatches (5) + : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ NoopFilter (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16), Statistics(X) : : +- ColumnarExchange (15) - : : +- VeloxAppendBatches (14) + : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ NoopFilter (11) : : +- ^ Scan parquet (10) : +- ^ InputIteratorTransformer (35) : +- ShuffleQueryStage (33), Statistics(X) : +- ColumnarExchange (32) - : +- VeloxAppendBatches (31) + : +- VeloxResizeBatches (31) : +- ^ ProjectExecTransformer (29) : +- ^ NoopFilter (28) : +- ^ Scan parquet (27) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50), Statistics(X) +- ColumnarExchange (49) - +- VeloxAppendBatches (48) + +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ NoopFilter (45) +- ^ Scan parquet (44) @@ -107,9 +107,9 @@ Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] @@ -144,9 +144,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -175,9 +175,9 @@ Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] @@ -212,9 +212,9 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] @@ -243,9 +243,9 @@ Input [11]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acc Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] @@ -280,9 +280,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -322,9 +322,9 @@ Input [9]: [c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] Arguments: false -(58) VeloxAppendBatches +(58) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (59) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/11.txt index 2347eb9b677e2..e24c94d48247e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/11.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (82) +- ^ InputIteratorTransformer (53) +- ShuffleQueryStage (51), Statistics(X) +- ColumnarExchange (50) - +- VeloxAppendBatches (49) + +- VeloxResizeBatches (49) +- ^ FilterExecTransformer (47) +- ^ RegularHashAggregateExecTransformer (46) +- ^ InputIteratorTransformer (45) +- ShuffleQueryStage (43), Statistics(X) +- ColumnarExchange (42) - +- VeloxAppendBatches (41) + +- VeloxResizeBatches (41) +- ^ ProjectExecTransformer (39) +- ^ FlushableHashAggregateExecTransformer (38) +- ^ ProjectExecTransformer (37) @@ -20,27 +20,27 @@ AdaptiveSparkPlan (82) :- ^ InputIteratorTransformer (26) : +- ShuffleQueryStage (24), Statistics(X) : +- ColumnarExchange (23) - : +- VeloxAppendBatches (22) + : +- VeloxResizeBatches (22) : +- ^ ProjectExecTransformer (20) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ NoopFilter (11) : +- ^ Scan parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ NoopFilter (28) +- ^ Scan parquet (27) @@ -91,9 +91,9 @@ Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] @@ -128,9 +128,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -159,9 +159,9 @@ Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppke Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] @@ -196,9 +196,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -238,9 +238,9 @@ Input [3]: [ps_partkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] Arguments: false -(41) VeloxAppendBatches +(41) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (42) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] @@ -271,9 +271,9 @@ Arguments: (isnotnull(value#X) AND (cast(value#X as decimal(38,6)) > Subquery su Input [2]: [ps_partkey#X, value#X] Arguments: false -(49) VeloxAppendBatches +(49) VeloxResizeBatches Input [2]: [ps_partkey#X, value#X] -Arguments: X +Arguments: X, X (50) ColumnarExchange Input [2]: [ps_partkey#X, value#X] @@ -435,13 +435,13 @@ AdaptiveSparkPlan (136) :- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101), Statistics(X) : +- ColumnarExchange (100) - : +- VeloxAppendBatches (99) + : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96) : :- ^ InputIteratorTransformer (91) : : +- ShuffleQueryStage (89), Statistics(X) : : +- ColumnarExchange (88) - : : +- VeloxAppendBatches (87) + : : +- VeloxResizeBatches (87) : : +- ^ ProjectExecTransformer (85) : : +- ^ NoopFilter (84) : : +- ^ Scan parquet (83) @@ -494,9 +494,9 @@ Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Input [4]: [hash_partition_key#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: false -(87) VeloxAppendBatches +(87) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (88) ColumnarExchange Input [4]: [hash_partition_key#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] @@ -538,9 +538,9 @@ Input [5]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppkey#X, s_nationk Input [4]: [hash_partition_key#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [4]: [hash_partition_key#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/12.txt index b0f084e2d0486..870f9ccebd86c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/12.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (55) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ NoopFilter (11) +- ^ Scan parquet (10) @@ -68,9 +68,9 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] @@ -105,9 +105,9 @@ Input [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipm Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] @@ -147,9 +147,9 @@ Input [3]: [l_shipmode#X, sum#X, sum#X] Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] @@ -176,9 +176,9 @@ Results [3]: [l_shipmode#X, sum(CASE WHEN ((o_orderpriority#X = 1-URGENT) OR (o_ Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/13.txt index 07c32ff95fb1e..ff5bfe2540967 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/13.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (58) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35), Statistics(X) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ RegularHashAggregateExecTransformer (31) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ ProjectExecTransformer (22) @@ -22,13 +22,13 @@ AdaptiveSparkPlan (58) :- ^ InputIteratorTransformer (8) : +- ShuffleQueryStage (6), Statistics(X) : +- ColumnarExchange (5) - : +- VeloxAppendBatches (4) + : +- VeloxResizeBatches (4) : +- ^ ProjectExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15), Statistics(X) +- ColumnarExchange (14) - +- VeloxAppendBatches (13) + +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) +- ^ NoopFilter (10) +- ^ Scan parquet (9) @@ -66,9 +66,9 @@ Input [1]: [c_custkey#X] Input [2]: [hash_partition_key#X, c_custkey#X] Arguments: false -(4) VeloxAppendBatches +(4) VeloxResizeBatches Input [2]: [hash_partition_key#X, c_custkey#X] -Arguments: X +Arguments: X, X (5) ColumnarExchange Input [2]: [hash_partition_key#X, c_custkey#X] @@ -103,9 +103,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(13) VeloxAppendBatches +(13) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (14) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -163,9 +163,9 @@ Input [2]: [c_count#X, count#X] Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] @@ -192,9 +192,9 @@ Results [2]: [c_count#X, count(1)#X AS custdist#X] Input [2]: [c_count#X, custdist#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [2]: [c_count#X, custdist#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/14.txt index 11bbb2a71e795..247b15e33ba3d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/14.txt @@ -10,14 +10,14 @@ AdaptiveSparkPlan (39) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ NoopFilter (11) +- ^ Scan parquet (10) @@ -56,9 +56,9 @@ Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] @@ -93,9 +93,9 @@ Input [2]: [p_partkey#X, p_type#X] Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/15.txt index be97f58cf4386..ecb61718853ca 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/15.txt @@ -5,13 +5,13 @@ AdaptiveSparkPlan (47) +- AQEShuffleRead (29) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (23) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (47) +- ^ InputIteratorTransformer (20) +- ShuffleQueryStage (18), Statistics(X) +- ColumnarExchange (17) - +- VeloxAppendBatches (16) + +- VeloxResizeBatches (16) +- ^ ProjectExecTransformer (14) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) @@ -64,9 +64,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] @@ -112,9 +112,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(16) VeloxAppendBatches +(16) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (17) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] @@ -154,9 +154,9 @@ Input [6]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, supplier_no#X, total_ Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] @@ -267,7 +267,7 @@ AdaptiveSparkPlan (73) +- ^ InputIteratorTransformer (58) +- ShuffleQueryStage (56), Statistics(X) +- ColumnarExchange (55) - +- VeloxAppendBatches (54) + +- VeloxResizeBatches (54) +- ^ ProjectExecTransformer (52) +- ^ FlushableHashAggregateExecTransformer (51) +- ^ ProjectExecTransformer (50) @@ -314,9 +314,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(54) VeloxAppendBatches +(54) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (55) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/16.txt index 86d2f321f653e..50a27d401534d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/16.txt @@ -6,19 +6,19 @@ AdaptiveSparkPlan (71) +- ^ InputIteratorTransformer (44) +- ShuffleQueryStage (42), Statistics(X) +- ColumnarExchange (41) - +- VeloxAppendBatches (40) + +- VeloxResizeBatches (40) +- ^ RegularHashAggregateExecTransformer (38) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35), Statistics(X) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ ProjectExecTransformer (31) +- ^ FlushableHashAggregateExecTransformer (30) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -26,14 +26,14 @@ AdaptiveSparkPlan (71) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ NoopFilter (11) +- ^ Scan parquet (10) @@ -82,9 +82,9 @@ Input [2]: [ps_partkey#X, ps_suppkey#X] Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] @@ -119,9 +119,9 @@ Input [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] @@ -161,9 +161,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] @@ -201,9 +201,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, count#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] @@ -230,9 +230,9 @@ Results [4]: [p_brand#X, p_type#X, p_size#X, count(ps_suppkey#X)#X AS supplier_c Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/17.txt index 6a2e47576cadf..e3596c36ce389 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/17.txt @@ -12,14 +12,14 @@ AdaptiveSparkPlan (63) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ NoopFilter (11) : +- ^ Scan parquet (10) @@ -29,7 +29,7 @@ AdaptiveSparkPlan (63) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ NoopFilter (22) @@ -78,9 +78,9 @@ Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] @@ -115,9 +115,9 @@ Input [3]: [p_partkey#X, p_brand#X, p_container#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -168,9 +168,9 @@ Input [3]: [l_partkey#X, sum#X, count#X] Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/18.txt index 7fe13a003017b..e84c68a092d6b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/18.txt @@ -10,26 +10,26 @@ AdaptiveSparkPlan (110) :- ^ InputIteratorTransformer (46) : +- ShuffleQueryStage (44), Statistics(X) : +- ColumnarExchange (43) - : +- VeloxAppendBatches (42) + : +- VeloxResizeBatches (42) : +- ^ ProjectExecTransformer (40) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (38) : +- ShuffleQueryStage (36), Statistics(X) : +- ColumnarExchange (35) - : +- VeloxAppendBatches (34) + : +- VeloxResizeBatches (34) : +- ^ ProjectExecTransformer (32) : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) : :- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16), Statistics(X) : : +- ColumnarExchange (15) - : : +- VeloxAppendBatches (14) + : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ NoopFilter (11) : : +- ^ Scan parquet (10) @@ -39,7 +39,7 @@ AdaptiveSparkPlan (110) : +- ^ InputIteratorTransformer (27) : +- ShuffleQueryStage (25), Statistics(X) : +- ColumnarExchange (24) - : +- VeloxAppendBatches (23) + : +- VeloxResizeBatches (23) : +- ^ ProjectExecTransformer (21) : +- ^ FlushableHashAggregateExecTransformer (20) : +- ^ Scan parquet (19) @@ -47,7 +47,7 @@ AdaptiveSparkPlan (110) :- ^ InputIteratorTransformer (55) : +- ShuffleQueryStage (53), Statistics(X) : +- ColumnarExchange (52) - : +- VeloxAppendBatches (51) + : +- VeloxResizeBatches (51) : +- ^ ProjectExecTransformer (49) : +- ^ NoopFilter (48) : +- ^ Scan parquet (47) @@ -118,9 +118,9 @@ Input [2]: [c_custkey#X, c_name#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] @@ -155,9 +155,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] @@ -194,9 +194,9 @@ Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] @@ -240,9 +240,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(34) VeloxAppendBatches +(34) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (35) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] @@ -271,9 +271,9 @@ Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_ Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(42) VeloxAppendBatches +(42) VeloxResizeBatches Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (43) ColumnarExchange Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] @@ -308,9 +308,9 @@ Input [2]: [l_orderkey#X, l_quantity#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] Arguments: false -(51) VeloxAppendBatches +(51) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] -Arguments: X +Arguments: X, X (52) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/19.txt index 34abb726b85af..ec373f6a4c881 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/19.txt @@ -9,14 +9,14 @@ AdaptiveSparkPlan (38) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ NoopFilter (11) +- ^ Scan parquet (10) @@ -55,9 +55,9 @@ Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipin Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -92,9 +92,9 @@ Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/20.txt index 2cf50b2a3a986..bb3a76ec6bb6d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/20.txt @@ -5,52 +5,52 @@ AdaptiveSparkPlan (143) +- AQEShuffleRead (92) +- ShuffleQueryStage (91), Statistics(X) +- ColumnarExchange (90) - +- VeloxAppendBatches (89) + +- VeloxResizeBatches (89) +- ^ ProjectExecTransformer (87) +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (86) :- ^ InputIteratorTransformer (76) : +- ShuffleQueryStage (74), Statistics(X) : +- ColumnarExchange (73) - : +- VeloxAppendBatches (72) + : +- VeloxResizeBatches (72) : +- ^ ProjectExecTransformer (70) : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (69) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (68) : +- ShuffleQueryStage (66), Statistics(X) : +- ColumnarExchange (65) - : +- VeloxAppendBatches (64) + : +- VeloxResizeBatches (64) : +- ^ ProjectExecTransformer (62) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (61) : :- ^ InputIteratorTransformer (35) : : +- ShuffleQueryStage (33), Statistics(X) : : +- ColumnarExchange (32) - : : +- VeloxAppendBatches (31) + : : +- VeloxResizeBatches (31) : : +- ^ ProjectExecTransformer (29) : : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (28) : : :- ^ InputIteratorTransformer (18) : : : +- ShuffleQueryStage (16), Statistics(X) : : : +- ColumnarExchange (15) - : : : +- VeloxAppendBatches (14) + : : : +- VeloxResizeBatches (14) : : : +- ^ ProjectExecTransformer (12) : : : +- ^ NoopFilter (11) : : : +- ^ Scan parquet (10) : : +- ^ InputIteratorTransformer (27) : : +- ShuffleQueryStage (25), Statistics(X) : : +- ColumnarExchange (24) - : : +- VeloxAppendBatches (23) + : : +- VeloxResizeBatches (23) : : +- ^ ProjectExecTransformer (21) : : +- ^ NoopFilter (20) : : +- ^ Scan parquet (19) : +- ^ InputIteratorTransformer (60) : +- ShuffleQueryStage (58), Statistics(X) : +- ColumnarExchange (57) - : +- VeloxAppendBatches (56) + : +- VeloxResizeBatches (56) : +- ^ ProjectExecTransformer (54) : +- ^ FilterExecTransformer (53) : +- ^ ProjectExecTransformer (52) @@ -60,7 +60,7 @@ AdaptiveSparkPlan (143) : :- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42), Statistics(X) : : +- ColumnarExchange (41) - : : +- VeloxAppendBatches (40) + : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) : : +- ^ NoopFilter (37) : : +- ^ Scan parquet (36) @@ -70,7 +70,7 @@ AdaptiveSparkPlan (143) +- ^ InputIteratorTransformer (85) +- ShuffleQueryStage (83), Statistics(X) +- ColumnarExchange (82) - +- VeloxAppendBatches (81) + +- VeloxResizeBatches (81) +- ^ ProjectExecTransformer (79) +- ^ NoopFilter (78) +- ^ Scan parquet (77) @@ -145,9 +145,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] @@ -182,9 +182,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] @@ -219,9 +219,9 @@ Input [2]: [p_partkey#X, p_name#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -250,9 +250,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] @@ -287,9 +287,9 @@ Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] @@ -353,9 +353,9 @@ Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] @@ -384,9 +384,9 @@ Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: false -(64) VeloxAppendBatches +(64) VeloxResizeBatches Input [2]: [hash_partition_key#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (65) ColumnarExchange Input [2]: [hash_partition_key#X, ps_suppkey#X] @@ -415,9 +415,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(72) VeloxAppendBatches +(72) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (73) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] @@ -452,9 +452,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(81) VeloxAppendBatches +(81) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (82) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -483,9 +483,9 @@ Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] Input [2]: [s_name#X, s_address#X] Arguments: false -(89) VeloxAppendBatches +(89) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] -Arguments: X +Arguments: X, X (90) ColumnarExchange Input [2]: [s_name#X, s_address#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/21.txt index f92684b17b15c..428acf4c91224 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/21.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (137) +- ^ InputIteratorTransformer (88) +- ShuffleQueryStage (86), Statistics(X) +- ColumnarExchange (85) - +- VeloxAppendBatches (84) + +- VeloxResizeBatches (84) +- ^ ProjectExecTransformer (82) +- ^ FlushableHashAggregateExecTransformer (81) +- ^ ProjectExecTransformer (80) @@ -14,60 +14,60 @@ AdaptiveSparkPlan (137) :- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (62) : :- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (45) : : :- ^ InputIteratorTransformer (9) : : : +- ShuffleQueryStage (7), Statistics(X) : : : +- ColumnarExchange (6) - : : : +- VeloxAppendBatches (5) + : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ NoopFilter (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42), Statistics(X) : : +- ColumnarExchange (41) - : : +- VeloxAppendBatches (40) + : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) : : +- ^ ShuffledHashJoinExecTransformer LeftAnti BuildRight (37) : : :- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (27) : : : :- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ NoopFilter (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (26) : : : +- ShuffleQueryStage (24), Statistics(X) : : : +- ColumnarExchange (23) - : : : +- VeloxAppendBatches (22) + : : : +- VeloxResizeBatches (22) : : : +- ^ ProjectExecTransformer (20) : : : +- ^ Scan parquet (19) : : +- ^ InputIteratorTransformer (36) : : +- ShuffleQueryStage (34), Statistics(X) : : +- ColumnarExchange (33) - : : +- VeloxAppendBatches (32) + : : +- VeloxResizeBatches (32) : : +- ^ ProjectExecTransformer (30) : : +- ^ NoopFilter (29) : : +- ^ Scan parquet (28) : +- ^ InputIteratorTransformer (61) : +- ShuffleQueryStage (59), Statistics(X) : +- ColumnarExchange (58) - : +- VeloxAppendBatches (57) + : +- VeloxResizeBatches (57) : +- ^ ProjectExecTransformer (55) : +- ^ NoopFilter (54) : +- ^ Scan parquet (53) +- ^ InputIteratorTransformer (78) +- ShuffleQueryStage (76), Statistics(X) +- ColumnarExchange (75) - +- VeloxAppendBatches (74) + +- VeloxResizeBatches (74) +- ^ ProjectExecTransformer (72) +- ^ NoopFilter (71) +- ^ Scan parquet (70) @@ -138,9 +138,9 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] @@ -175,9 +175,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -207,9 +207,9 @@ Input [2]: [l_orderkey#X, l_suppkey#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -249,9 +249,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(32) VeloxAppendBatches +(32) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (33) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -280,9 +280,9 @@ Input [2]: [l_orderkey#X, l_suppkey#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -311,9 +311,9 @@ Input [5]: [s_suppkey#X, s_name#X, s_nationkey#X, l_orderkey#X, l_suppkey#X] Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] @@ -348,9 +348,9 @@ Input [2]: [o_orderkey#X, o_orderstatus#X] Input [2]: [hash_partition_key#X, o_orderkey#X] Arguments: false -(57) VeloxAppendBatches +(57) VeloxResizeBatches Input [2]: [hash_partition_key#X, o_orderkey#X] -Arguments: X +Arguments: X, X (58) ColumnarExchange Input [2]: [hash_partition_key#X, o_orderkey#X] @@ -379,9 +379,9 @@ Input [4]: [s_name#X, s_nationkey#X, l_orderkey#X, o_orderkey#X] Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] @@ -416,9 +416,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(74) VeloxAppendBatches +(74) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (75) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -458,9 +458,9 @@ Input [2]: [s_name#X, count#X] Input [3]: [hash_partition_key#X, s_name#X, count#X] Arguments: false -(84) VeloxAppendBatches +(84) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, count#X] -Arguments: X +Arguments: X, X (85) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/22.txt index 1c2790a4a9999..86272c2a803ea 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/22.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (52) +- ^ InputIteratorTransformer (34) +- ShuffleQueryStage (32), Statistics(X) +- ColumnarExchange (31) - +- VeloxAppendBatches (30) + +- VeloxResizeBatches (30) +- ^ RegularHashAggregateExecTransformer (28) +- ^ InputIteratorTransformer (27) +- ShuffleQueryStage (25), Statistics(X) +- ColumnarExchange (24) - +- VeloxAppendBatches (23) + +- VeloxResizeBatches (23) +- ^ ProjectExecTransformer (21) +- ^ FlushableHashAggregateExecTransformer (20) +- ^ ProjectExecTransformer (19) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (52) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15), Statistics(X) +- ColumnarExchange (14) - +- VeloxAppendBatches (13) + +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) +- ^ Scan parquet (10) +- == Initial Plan == @@ -65,9 +65,9 @@ Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] @@ -97,9 +97,9 @@ Input [1]: [o_custkey#X] Input [2]: [hash_partition_key#X, o_custkey#X] Arguments: false -(13) VeloxAppendBatches +(13) VeloxResizeBatches Input [2]: [hash_partition_key#X, o_custkey#X] -Arguments: X +Arguments: X, X (14) ColumnarExchange Input [2]: [hash_partition_key#X, o_custkey#X] @@ -139,9 +139,9 @@ Input [4]: [cntrycode#X, count#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] @@ -168,9 +168,9 @@ Results [3]: [cntrycode#X, count(1)#X AS numcust#X, sum(c_acctbal#X)#X AS totacc Input [3]: [cntrycode#X, numcust#X, totacctbal#X] Arguments: false -(30) VeloxAppendBatches +(30) VeloxResizeBatches Input [3]: [cntrycode#X, numcust#X, totacctbal#X] -Arguments: X +Arguments: X, X (31) ColumnarExchange Input [3]: [cntrycode#X, numcust#X, totacctbal#X] @@ -279,7 +279,7 @@ AdaptiveSparkPlan (72) +- ^ InputIteratorTransformer (62) +- ShuffleQueryStage (60), Statistics(X) +- ColumnarExchange (59) - +- VeloxAppendBatches (58) + +- VeloxResizeBatches (58) +- ^ FlushableHashAggregateExecTransformer (56) +- ^ ProjectExecTransformer (55) +- ^ NoopFilter (54) @@ -319,9 +319,9 @@ Results [2]: [sum#X, count#X] Input [2]: [sum#X, count#X] Arguments: false -(58) VeloxAppendBatches +(58) VeloxResizeBatches Input [2]: [sum#X, count#X] -Arguments: X +Arguments: X, X (59) ColumnarExchange Input [2]: [sum#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/3.txt index 1f99052941446..40f7aeb35749b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/3.txt @@ -11,27 +11,27 @@ AdaptiveSparkPlan (67) :- ^ InputIteratorTransformer (26) : +- ShuffleQueryStage (24), Statistics(X) : +- ColumnarExchange (23) - : +- VeloxAppendBatches (22) + : +- VeloxResizeBatches (22) : +- ^ ProjectExecTransformer (20) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ NoopFilter (11) : +- ^ Scan parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ NoopFilter (28) +- ^ Scan parquet (27) @@ -80,9 +80,9 @@ Input [2]: [c_custkey#X, c_mktsegment#X] Input [2]: [hash_partition_key#X, c_custkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, c_custkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, c_custkey#X] @@ -117,9 +117,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] @@ -148,9 +148,9 @@ Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriorit Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] @@ -185,9 +185,9 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/4.txt index 130bc2983040c..eb6a3e27f9f39 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/4.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (56) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (56) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ NoopFilter (11) +- ^ Scan parquet (10) @@ -69,9 +69,9 @@ Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] @@ -106,9 +106,9 @@ Input [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Input [2]: [hash_partition_key#X, l_orderkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [2]: [hash_partition_key#X, l_orderkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [2]: [hash_partition_key#X, l_orderkey#X] @@ -148,9 +148,9 @@ Input [2]: [o_orderpriority#X, count#X] Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] @@ -177,9 +177,9 @@ Results [2]: [o_orderpriority#X, count(1)#X AS order_count#X] Input [2]: [o_orderpriority#X, order_count#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [2]: [o_orderpriority#X, order_count#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [2]: [o_orderpriority#X, order_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/5.txt index 10ce074fd7601..20d0aa276e59b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/5.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (156) +- ^ InputIteratorTransformer (103) +- ShuffleQueryStage (101), Statistics(X) +- ColumnarExchange (100) - +- VeloxAppendBatches (99) + +- VeloxResizeBatches (99) +- ^ RegularHashAggregateExecTransformer (97) +- ^ InputIteratorTransformer (96) +- ShuffleQueryStage (94), Statistics(X) +- ColumnarExchange (93) - +- VeloxAppendBatches (92) + +- VeloxResizeBatches (92) +- ^ ProjectExecTransformer (90) +- ^ FlushableHashAggregateExecTransformer (89) +- ^ ProjectExecTransformer (88) @@ -19,66 +19,66 @@ AdaptiveSparkPlan (156) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75), Statistics(X) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58), Statistics(X) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41), Statistics(X) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ NoopFilter (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ NoopFilter (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ NoopFilter (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ NoopFilter (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ NoopFilter (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) - +- VeloxAppendBatches (82) + +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ NoopFilter (79) +- ^ Scan parquet (78) @@ -153,9 +153,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -190,9 +190,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -221,9 +221,9 @@ Input [4]: [c_custkey#X, c_nationkey#X, o_orderkey#X, o_custkey#X] Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] @@ -258,9 +258,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -289,9 +289,9 @@ Input [6]: [c_nationkey#X, o_orderkey#X, l_orderkey#X, l_suppkey#X, l_extendedpr Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -326,9 +326,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -357,9 +357,9 @@ Input [6]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppk Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -394,9 +394,9 @@ Input [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] @@ -425,9 +425,9 @@ Input [6]: [l_extendedprice#X, l_discount#X, s_nationkey#X, n_nationkey#X, n_nam Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] @@ -462,9 +462,9 @@ Input [2]: [r_regionkey#X, r_name#X] Input [2]: [hash_partition_key#X, r_regionkey#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [2]: [hash_partition_key#X, r_regionkey#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [2]: [hash_partition_key#X, r_regionkey#X] @@ -504,9 +504,9 @@ Input [3]: [n_name#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] Arguments: false -(92) VeloxAppendBatches +(92) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (93) ColumnarExchange Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] @@ -533,9 +533,9 @@ Results [2]: [n_name#X, sum(CheckOverflow((promote_precision(cast(l_extendedpric Input [2]: [n_name#X, revenue#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [2]: [n_name#X, revenue#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [2]: [n_name#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/6.txt index fa9c936a0ca11..1aaed506d7e08 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/6.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (20) +- ^ InputIteratorTransformer (10) +- ShuffleQueryStage (8), Statistics(X) +- ColumnarExchange (7) - +- VeloxAppendBatches (6) + +- VeloxResizeBatches (6) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ NoopFilter (2) @@ -46,9 +46,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(6) VeloxAppendBatches +(6) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (7) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/7.txt index 651cfa840be88..2261654d4a500 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/7.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (149) +- ^ InputIteratorTransformer (98) +- ShuffleQueryStage (96), Statistics(X) +- ColumnarExchange (95) - +- VeloxAppendBatches (94) + +- VeloxResizeBatches (94) +- ^ RegularHashAggregateExecTransformer (92) +- ^ InputIteratorTransformer (91) +- ShuffleQueryStage (89), Statistics(X) +- ColumnarExchange (88) - +- VeloxAppendBatches (87) + +- VeloxResizeBatches (87) +- ^ ProjectExecTransformer (85) +- ^ FlushableHashAggregateExecTransformer (84) +- ^ ProjectExecTransformer (83) @@ -19,59 +19,59 @@ AdaptiveSparkPlan (149) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75), Statistics(X) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58), Statistics(X) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41), Statistics(X) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ NoopFilter (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ NoopFilter (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ NoopFilter (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ NoopFilter (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ NoopFilter (62) : +- ^ Scan parquet (61) @@ -147,9 +147,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -184,9 +184,9 @@ Input [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipda Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] @@ -215,9 +215,9 @@ Input [7]: [s_suppkey#X, s_nationkey#X, l_orderkey#X, l_suppkey#X, l_extendedpri Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] @@ -252,9 +252,9 @@ Input [2]: [o_orderkey#X, o_custkey#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -283,9 +283,9 @@ Input [7]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_ship Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] @@ -320,9 +320,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -351,9 +351,9 @@ Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_cust Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] @@ -388,9 +388,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -419,9 +419,9 @@ Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nati Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] @@ -474,9 +474,9 @@ Input [5]: [supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Arguments: false -(87) VeloxAppendBatches +(87) VeloxResizeBatches Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (88) ColumnarExchange Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] @@ -503,9 +503,9 @@ Results [4]: [supp_nation#X, cust_nation#X, l_year#X, sum(volume#X)#X AS revenue Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] Arguments: false -(94) VeloxAppendBatches +(94) VeloxResizeBatches Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] -Arguments: X +Arguments: X, X (95) ColumnarExchange Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/8.txt index e359f4c944e72..3e836d995cb96 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/8.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (207) +- ^ InputIteratorTransformer (138) +- ShuffleQueryStage (136), Statistics(X) +- ColumnarExchange (135) - +- VeloxAppendBatches (134) + +- VeloxResizeBatches (134) +- ^ ProjectExecTransformer (132) +- ^ RegularHashAggregateExecTransformer (131) +- ^ InputIteratorTransformer (130) +- ShuffleQueryStage (128), Statistics(X) +- ColumnarExchange (127) - +- VeloxAppendBatches (126) + +- VeloxResizeBatches (126) +- ^ ProjectExecTransformer (124) +- ^ FlushableHashAggregateExecTransformer (123) +- ^ ProjectExecTransformer (122) @@ -20,92 +20,92 @@ AdaptiveSparkPlan (207) :- ^ InputIteratorTransformer (111) : +- ShuffleQueryStage (109), Statistics(X) : +- ColumnarExchange (108) - : +- VeloxAppendBatches (107) + : +- VeloxResizeBatches (107) : +- ^ ProjectExecTransformer (105) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (104) : :- ^ InputIteratorTransformer (94) : : +- ShuffleQueryStage (92), Statistics(X) : : +- ColumnarExchange (91) - : : +- VeloxAppendBatches (90) + : : +- VeloxResizeBatches (90) : : +- ^ ProjectExecTransformer (88) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (87) : : :- ^ InputIteratorTransformer (77) : : : +- ShuffleQueryStage (75), Statistics(X) : : : +- ColumnarExchange (74) - : : : +- VeloxAppendBatches (73) + : : : +- VeloxResizeBatches (73) : : : +- ^ ProjectExecTransformer (71) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (70) : : : :- ^ InputIteratorTransformer (60) : : : : +- ShuffleQueryStage (58), Statistics(X) : : : : +- ColumnarExchange (57) - : : : : +- VeloxAppendBatches (56) + : : : : +- VeloxResizeBatches (56) : : : : +- ^ ProjectExecTransformer (54) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (53) : : : : :- ^ InputIteratorTransformer (43) : : : : : +- ShuffleQueryStage (41), Statistics(X) : : : : : +- ColumnarExchange (40) - : : : : : +- VeloxAppendBatches (39) + : : : : : +- VeloxResizeBatches (39) : : : : : +- ^ ProjectExecTransformer (37) : : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : : : :- ^ InputIteratorTransformer (26) : : : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : : : +- ColumnarExchange (23) - : : : : : : +- VeloxAppendBatches (22) + : : : : : : +- VeloxResizeBatches (22) : : : : : : +- ^ ProjectExecTransformer (20) : : : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : : : :- ^ InputIteratorTransformer (9) : : : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : : : +- ColumnarExchange (6) - : : : : : : : +- VeloxAppendBatches (5) + : : : : : : : +- VeloxResizeBatches (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ NoopFilter (2) : : : : : : : +- ^ Scan parquet (1) : : : : : : +- ^ InputIteratorTransformer (18) : : : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : : : +- ColumnarExchange (15) - : : : : : : +- VeloxAppendBatches (14) + : : : : : : +- VeloxResizeBatches (14) : : : : : : +- ^ ProjectExecTransformer (12) : : : : : : +- ^ NoopFilter (11) : : : : : : +- ^ Scan parquet (10) : : : : : +- ^ InputIteratorTransformer (35) : : : : : +- ShuffleQueryStage (33), Statistics(X) : : : : : +- ColumnarExchange (32) - : : : : : +- VeloxAppendBatches (31) + : : : : : +- VeloxResizeBatches (31) : : : : : +- ^ ProjectExecTransformer (29) : : : : : +- ^ NoopFilter (28) : : : : : +- ^ Scan parquet (27) : : : : +- ^ InputIteratorTransformer (52) : : : : +- ShuffleQueryStage (50), Statistics(X) : : : : +- ColumnarExchange (49) - : : : : +- VeloxAppendBatches (48) + : : : : +- VeloxResizeBatches (48) : : : : +- ^ ProjectExecTransformer (46) : : : : +- ^ NoopFilter (45) : : : : +- ^ Scan parquet (44) : : : +- ^ InputIteratorTransformer (69) : : : +- ShuffleQueryStage (67), Statistics(X) : : : +- ColumnarExchange (66) - : : : +- VeloxAppendBatches (65) + : : : +- VeloxResizeBatches (65) : : : +- ^ ProjectExecTransformer (63) : : : +- ^ NoopFilter (62) : : : +- ^ Scan parquet (61) : : +- ^ InputIteratorTransformer (86) : : +- ShuffleQueryStage (84), Statistics(X) : : +- ColumnarExchange (83) - : : +- VeloxAppendBatches (82) + : : +- VeloxResizeBatches (82) : : +- ^ ProjectExecTransformer (80) : : +- ^ NoopFilter (79) : : +- ^ Scan parquet (78) : +- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101), Statistics(X) : +- ColumnarExchange (100) - : +- VeloxAppendBatches (99) + : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ NoopFilter (96) : +- ^ Scan parquet (95) +- ^ InputIteratorTransformer (120) +- ShuffleQueryStage (118), Statistics(X) +- ColumnarExchange (117) - +- VeloxAppendBatches (116) + +- VeloxResizeBatches (116) +- ^ ProjectExecTransformer (114) +- ^ NoopFilter (113) +- ^ Scan parquet (112) @@ -196,9 +196,9 @@ Input [2]: [p_partkey#X, p_type#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -233,9 +233,9 @@ Input [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discoun Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -264,9 +264,9 @@ Input [6]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -301,9 +301,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -332,9 +332,9 @@ Input [6]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppke Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -369,9 +369,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] @@ -400,9 +400,9 @@ Input [7]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orde Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] @@ -437,9 +437,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -468,9 +468,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_order Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] @@ -505,9 +505,9 @@ Input [2]: [n_nationkey#X, n_regionkey#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] @@ -536,9 +536,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nat Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] Arguments: false -(90) VeloxAppendBatches +(90) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] -Arguments: X +Arguments: X, X (91) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] @@ -573,9 +573,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -604,9 +604,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_reg Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] Arguments: false -(107) VeloxAppendBatches +(107) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] -Arguments: X +Arguments: X, X (108) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] @@ -641,9 +641,9 @@ Input [2]: [r_regionkey#X, r_name#X] Input [2]: [hash_partition_key#X, r_regionkey#X] Arguments: false -(116) VeloxAppendBatches +(116) VeloxResizeBatches Input [2]: [hash_partition_key#X, r_regionkey#X] -Arguments: X +Arguments: X, X (117) ColumnarExchange Input [2]: [hash_partition_key#X, r_regionkey#X] @@ -683,9 +683,9 @@ Input [5]: [o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(126) VeloxAppendBatches +(126) VeloxResizeBatches Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (127) ColumnarExchange Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] @@ -716,9 +716,9 @@ Input [3]: [o_year#X, sum(CASE WHEN (nation#X = BRAZIL) THEN volume#X ELSE 0.000 Input [2]: [o_year#X, mkt_share#X] Arguments: false -(134) VeloxAppendBatches +(134) VeloxResizeBatches Input [2]: [o_year#X, mkt_share#X] -Arguments: X +Arguments: X, X (135) ColumnarExchange Input [2]: [o_year#X, mkt_share#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/9.txt index 21c91ca141800..8ed229096ba6a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/9.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (155) +- ^ InputIteratorTransformer (103) +- ShuffleQueryStage (101), Statistics(X) +- ColumnarExchange (100) - +- VeloxAppendBatches (99) + +- VeloxResizeBatches (99) +- ^ RegularHashAggregateExecTransformer (97) +- ^ InputIteratorTransformer (96) +- ShuffleQueryStage (94), Statistics(X) +- ColumnarExchange (93) - +- VeloxAppendBatches (92) + +- VeloxResizeBatches (92) +- ^ ProjectExecTransformer (90) +- ^ FlushableHashAggregateExecTransformer (89) +- ^ ProjectExecTransformer (88) @@ -19,66 +19,66 @@ AdaptiveSparkPlan (155) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75), Statistics(X) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58), Statistics(X) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41), Statistics(X) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ NoopFilter (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ NoopFilter (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ NoopFilter (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ NoopFilter (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ NoopFilter (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) - +- VeloxAppendBatches (82) + +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ NoopFilter (79) +- ^ Scan parquet (78) @@ -152,9 +152,9 @@ Input [2]: [p_partkey#X, p_name#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -189,9 +189,9 @@ Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -220,9 +220,9 @@ Input [7]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -257,9 +257,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -288,9 +288,9 @@ Input [8]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -325,9 +325,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] @@ -356,9 +356,9 @@ Input [10]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpri Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] @@ -393,9 +393,9 @@ Input [2]: [o_orderkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] @@ -424,9 +424,9 @@ Input [8]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_natio Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] @@ -461,9 +461,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -503,9 +503,9 @@ Input [4]: [nation#X, o_year#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] Arguments: false -(92) VeloxAppendBatches +(92) VeloxResizeBatches Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (93) ColumnarExchange Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] @@ -532,9 +532,9 @@ Results [3]: [nation#X, o_year#X, sum(amount#X)#X AS sum_profit#X] Input [3]: [nation#X, o_year#X, sum_profit#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [3]: [nation#X, o_year#X, sum_profit#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [3]: [nation#X, o_year#X, sum_profit#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/1.txt index 545f2e7e086df..2db104cfeb12e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/1.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (30) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ RegularHashAggregateExecTransformer (12) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9), Statistics(X) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ ProjectExecTransformer (5) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) @@ -58,9 +58,9 @@ Input [17]: [l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] @@ -87,9 +87,9 @@ Results [10]: [l_returnflag#X, l_linestatus#X, sum(l_quantity#X)#X AS sum_qty#X, Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/10.txt index 5e48ceb742d71..02aaa69a3ea8f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/10.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (100) +- ^ InputIteratorTransformer (62) +- ShuffleQueryStage (60), Statistics(X) +- ColumnarExchange (59) - +- VeloxAppendBatches (58) + +- VeloxResizeBatches (58) +- ^ ProjectExecTransformer (56) +- ^ FlushableHashAggregateExecTransformer (55) +- ^ ProjectExecTransformer (54) @@ -16,40 +16,40 @@ AdaptiveSparkPlan (100) :- ^ InputIteratorTransformer (43) : +- ShuffleQueryStage (41), Statistics(X) : +- ColumnarExchange (40) - : +- VeloxAppendBatches (39) + : +- VeloxResizeBatches (39) : +- ^ ProjectExecTransformer (37) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) : :- ^ InputIteratorTransformer (26) : : +- ShuffleQueryStage (24), Statistics(X) : : +- ColumnarExchange (23) - : : +- VeloxAppendBatches (22) + : : +- VeloxResizeBatches (22) : : +- ^ ProjectExecTransformer (20) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) : : :- ^ InputIteratorTransformer (9) : : : +- ShuffleQueryStage (7), Statistics(X) : : : +- ColumnarExchange (6) - : : : +- VeloxAppendBatches (5) + : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ NoopFilter (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16), Statistics(X) : : +- ColumnarExchange (15) - : : +- VeloxAppendBatches (14) + : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ NoopFilter (11) : : +- ^ Scan parquet (10) : +- ^ InputIteratorTransformer (35) : +- ShuffleQueryStage (33), Statistics(X) : +- ColumnarExchange (32) - : +- VeloxAppendBatches (31) + : +- VeloxResizeBatches (31) : +- ^ ProjectExecTransformer (29) : +- ^ NoopFilter (28) : +- ^ Scan parquet (27) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50), Statistics(X) +- ColumnarExchange (49) - +- VeloxAppendBatches (48) + +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ NoopFilter (45) +- ^ Scan parquet (44) @@ -107,9 +107,9 @@ Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] @@ -144,9 +144,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -176,9 +176,9 @@ Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] @@ -213,9 +213,9 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] @@ -245,9 +245,9 @@ Input [11]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acc Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] @@ -282,9 +282,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -325,9 +325,9 @@ Input [9]: [c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] Arguments: false -(58) VeloxAppendBatches +(58) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (59) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/11.txt index 6c9eef6b1f1fc..39187c1a1c262 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/11.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (82) +- ^ InputIteratorTransformer (53) +- ShuffleQueryStage (51), Statistics(X) +- ColumnarExchange (50) - +- VeloxAppendBatches (49) + +- VeloxResizeBatches (49) +- ^ FilterExecTransformer (47) +- ^ RegularHashAggregateExecTransformer (46) +- ^ InputIteratorTransformer (45) +- ShuffleQueryStage (43), Statistics(X) +- ColumnarExchange (42) - +- VeloxAppendBatches (41) + +- VeloxResizeBatches (41) +- ^ ProjectExecTransformer (39) +- ^ FlushableHashAggregateExecTransformer (38) +- ^ ProjectExecTransformer (37) @@ -20,27 +20,27 @@ AdaptiveSparkPlan (82) :- ^ InputIteratorTransformer (26) : +- ShuffleQueryStage (24), Statistics(X) : +- ColumnarExchange (23) - : +- VeloxAppendBatches (22) + : +- VeloxResizeBatches (22) : +- ^ ProjectExecTransformer (20) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ NoopFilter (11) : +- ^ Scan parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ NoopFilter (28) +- ^ Scan parquet (27) @@ -91,9 +91,9 @@ Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] @@ -128,9 +128,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -160,9 +160,9 @@ Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppke Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] @@ -197,9 +197,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -240,9 +240,9 @@ Input [3]: [ps_partkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] Arguments: false -(41) VeloxAppendBatches +(41) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (42) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] @@ -273,9 +273,9 @@ Arguments: (isnotnull(value#X) AND (cast(value#X as decimal(38,6)) > Subquery su Input [2]: [ps_partkey#X, value#X] Arguments: false -(49) VeloxAppendBatches +(49) VeloxResizeBatches Input [2]: [ps_partkey#X, value#X] -Arguments: X +Arguments: X, X (50) ColumnarExchange Input [2]: [ps_partkey#X, value#X] @@ -439,13 +439,13 @@ AdaptiveSparkPlan (136) :- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101), Statistics(X) : +- ColumnarExchange (100) - : +- VeloxAppendBatches (99) + : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96) : :- ^ InputIteratorTransformer (91) : : +- ShuffleQueryStage (89), Statistics(X) : : +- ColumnarExchange (88) - : : +- VeloxAppendBatches (87) + : : +- VeloxResizeBatches (87) : : +- ^ ProjectExecTransformer (85) : : +- ^ NoopFilter (84) : : +- ^ Scan parquet (83) @@ -498,9 +498,9 @@ Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Input [4]: [hash_partition_key#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: false -(87) VeloxAppendBatches +(87) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (88) ColumnarExchange Input [4]: [hash_partition_key#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] @@ -543,9 +543,9 @@ Input [5]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppkey#X, s_nationk Input [4]: [hash_partition_key#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [4]: [hash_partition_key#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/12.txt index 8f963c49aba83..2ce16d0a2db54 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/12.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (55) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ NoopFilter (11) +- ^ Scan parquet (10) @@ -68,9 +68,9 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] @@ -105,9 +105,9 @@ Input [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipm Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] @@ -148,9 +148,9 @@ Input [3]: [l_shipmode#X, sum#X, sum#X] Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] @@ -177,9 +177,9 @@ Results [3]: [l_shipmode#X, sum(CASE WHEN ((o_orderpriority#X = 1-URGENT) OR (o_ Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/13.txt index 9584f92628cd8..8989431cc83c4 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/13.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (58) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35), Statistics(X) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ RegularHashAggregateExecTransformer (31) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ ProjectExecTransformer (22) @@ -22,13 +22,13 @@ AdaptiveSparkPlan (58) :- ^ InputIteratorTransformer (8) : +- ShuffleQueryStage (6), Statistics(X) : +- ColumnarExchange (5) - : +- VeloxAppendBatches (4) + : +- VeloxResizeBatches (4) : +- ^ ProjectExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15), Statistics(X) +- ColumnarExchange (14) - +- VeloxAppendBatches (13) + +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) +- ^ NoopFilter (10) +- ^ Scan parquet (9) @@ -66,9 +66,9 @@ Input [1]: [c_custkey#X] Input [2]: [hash_partition_key#X, c_custkey#X] Arguments: false -(4) VeloxAppendBatches +(4) VeloxResizeBatches Input [2]: [hash_partition_key#X, c_custkey#X] -Arguments: X +Arguments: X, X (5) ColumnarExchange Input [2]: [hash_partition_key#X, c_custkey#X] @@ -103,9 +103,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(13) VeloxAppendBatches +(13) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (14) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -164,9 +164,9 @@ Input [2]: [c_count#X, count#X] Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] @@ -193,9 +193,9 @@ Results [2]: [c_count#X, count(1)#X AS custdist#X] Input [2]: [c_count#X, custdist#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [2]: [c_count#X, custdist#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/14.txt index 175a0e5a97cce..f9e3f564c7828 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/14.txt @@ -10,14 +10,14 @@ AdaptiveSparkPlan (39) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ NoopFilter (11) +- ^ Scan parquet (10) @@ -56,9 +56,9 @@ Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] @@ -93,9 +93,9 @@ Input [2]: [p_partkey#X, p_type#X] Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/15.txt index 130d9036b4a25..ca94a960e811f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/15.txt @@ -5,13 +5,13 @@ AdaptiveSparkPlan (47) +- AQEShuffleRead (29) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (23) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (47) +- ^ InputIteratorTransformer (20) +- ShuffleQueryStage (18), Statistics(X) +- ColumnarExchange (17) - +- VeloxAppendBatches (16) + +- VeloxResizeBatches (16) +- ^ ProjectExecTransformer (14) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) @@ -64,9 +64,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] @@ -112,9 +112,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(16) VeloxAppendBatches +(16) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (17) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] @@ -155,9 +155,9 @@ Input [6]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, supplier_no#X, total_ Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] @@ -269,7 +269,7 @@ AdaptiveSparkPlan (73) +- ^ InputIteratorTransformer (58) +- ShuffleQueryStage (56), Statistics(X) +- ColumnarExchange (55) - +- VeloxAppendBatches (54) + +- VeloxResizeBatches (54) +- ^ ProjectExecTransformer (52) +- ^ FlushableHashAggregateExecTransformer (51) +- ^ ProjectExecTransformer (50) @@ -316,9 +316,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(54) VeloxAppendBatches +(54) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (55) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/16.txt index 32a24beb94b95..5813bbd1af9b7 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/16.txt @@ -6,19 +6,19 @@ AdaptiveSparkPlan (71) +- ^ InputIteratorTransformer (44) +- ShuffleQueryStage (42), Statistics(X) +- ColumnarExchange (41) - +- VeloxAppendBatches (40) + +- VeloxResizeBatches (40) +- ^ RegularHashAggregateExecTransformer (38) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35), Statistics(X) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ ProjectExecTransformer (31) +- ^ FlushableHashAggregateExecTransformer (30) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -26,14 +26,14 @@ AdaptiveSparkPlan (71) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ NoopFilter (11) +- ^ Scan parquet (10) @@ -82,9 +82,9 @@ Input [2]: [ps_partkey#X, ps_suppkey#X] Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] @@ -119,9 +119,9 @@ Input [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] @@ -162,9 +162,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] @@ -202,9 +202,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, count#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] @@ -231,9 +231,9 @@ Results [4]: [p_brand#X, p_type#X, p_size#X, count(ps_suppkey#X)#X AS supplier_c Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/17.txt index 59baa2d7a08ca..8b4e121891f29 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/17.txt @@ -12,14 +12,14 @@ AdaptiveSparkPlan (63) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ NoopFilter (11) : +- ^ Scan parquet (10) @@ -29,7 +29,7 @@ AdaptiveSparkPlan (63) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ NoopFilter (22) @@ -78,9 +78,9 @@ Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] @@ -115,9 +115,9 @@ Input [3]: [p_partkey#X, p_brand#X, p_container#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -169,9 +169,9 @@ Input [3]: [l_partkey#X, sum#X, count#X] Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/18.txt index c78e265e54d95..3f7c919c62129 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/18.txt @@ -10,26 +10,26 @@ AdaptiveSparkPlan (110) :- ^ InputIteratorTransformer (46) : +- ShuffleQueryStage (44), Statistics(X) : +- ColumnarExchange (43) - : +- VeloxAppendBatches (42) + : +- VeloxResizeBatches (42) : +- ^ ProjectExecTransformer (40) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (38) : +- ShuffleQueryStage (36), Statistics(X) : +- ColumnarExchange (35) - : +- VeloxAppendBatches (34) + : +- VeloxResizeBatches (34) : +- ^ ProjectExecTransformer (32) : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) : :- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16), Statistics(X) : : +- ColumnarExchange (15) - : : +- VeloxAppendBatches (14) + : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ NoopFilter (11) : : +- ^ Scan parquet (10) @@ -39,7 +39,7 @@ AdaptiveSparkPlan (110) : +- ^ InputIteratorTransformer (27) : +- ShuffleQueryStage (25), Statistics(X) : +- ColumnarExchange (24) - : +- VeloxAppendBatches (23) + : +- VeloxResizeBatches (23) : +- ^ ProjectExecTransformer (21) : +- ^ FlushableHashAggregateExecTransformer (20) : +- ^ Scan parquet (19) @@ -47,7 +47,7 @@ AdaptiveSparkPlan (110) :- ^ InputIteratorTransformer (55) : +- ShuffleQueryStage (53), Statistics(X) : +- ColumnarExchange (52) - : +- VeloxAppendBatches (51) + : +- VeloxResizeBatches (51) : +- ^ ProjectExecTransformer (49) : +- ^ NoopFilter (48) : +- ^ Scan parquet (47) @@ -118,9 +118,9 @@ Input [2]: [c_custkey#X, c_name#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] @@ -155,9 +155,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] @@ -194,9 +194,9 @@ Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] @@ -241,9 +241,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(34) VeloxAppendBatches +(34) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (35) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] @@ -273,9 +273,9 @@ Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_ Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(42) VeloxAppendBatches +(42) VeloxResizeBatches Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (43) ColumnarExchange Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] @@ -310,9 +310,9 @@ Input [2]: [l_orderkey#X, l_quantity#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] Arguments: false -(51) VeloxAppendBatches +(51) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] -Arguments: X +Arguments: X, X (52) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/19.txt index a9c629524fb71..f29554305a6df 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/19.txt @@ -9,14 +9,14 @@ AdaptiveSparkPlan (38) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ NoopFilter (11) +- ^ Scan parquet (10) @@ -55,9 +55,9 @@ Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipin Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -92,9 +92,9 @@ Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/20.txt index 8e929ff7b2961..227d18bd0f213 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/20.txt @@ -5,52 +5,52 @@ AdaptiveSparkPlan (143) +- AQEShuffleRead (92) +- ShuffleQueryStage (91), Statistics(X) +- ColumnarExchange (90) - +- VeloxAppendBatches (89) + +- VeloxResizeBatches (89) +- ^ ProjectExecTransformer (87) +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (86) :- ^ InputIteratorTransformer (76) : +- ShuffleQueryStage (74), Statistics(X) : +- ColumnarExchange (73) - : +- VeloxAppendBatches (72) + : +- VeloxResizeBatches (72) : +- ^ ProjectExecTransformer (70) : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (69) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (68) : +- ShuffleQueryStage (66), Statistics(X) : +- ColumnarExchange (65) - : +- VeloxAppendBatches (64) + : +- VeloxResizeBatches (64) : +- ^ ProjectExecTransformer (62) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (61) : :- ^ InputIteratorTransformer (35) : : +- ShuffleQueryStage (33), Statistics(X) : : +- ColumnarExchange (32) - : : +- VeloxAppendBatches (31) + : : +- VeloxResizeBatches (31) : : +- ^ ProjectExecTransformer (29) : : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (28) : : :- ^ InputIteratorTransformer (18) : : : +- ShuffleQueryStage (16), Statistics(X) : : : +- ColumnarExchange (15) - : : : +- VeloxAppendBatches (14) + : : : +- VeloxResizeBatches (14) : : : +- ^ ProjectExecTransformer (12) : : : +- ^ NoopFilter (11) : : : +- ^ Scan parquet (10) : : +- ^ InputIteratorTransformer (27) : : +- ShuffleQueryStage (25), Statistics(X) : : +- ColumnarExchange (24) - : : +- VeloxAppendBatches (23) + : : +- VeloxResizeBatches (23) : : +- ^ ProjectExecTransformer (21) : : +- ^ NoopFilter (20) : : +- ^ Scan parquet (19) : +- ^ InputIteratorTransformer (60) : +- ShuffleQueryStage (58), Statistics(X) : +- ColumnarExchange (57) - : +- VeloxAppendBatches (56) + : +- VeloxResizeBatches (56) : +- ^ ProjectExecTransformer (54) : +- ^ FilterExecTransformer (53) : +- ^ ProjectExecTransformer (52) @@ -60,7 +60,7 @@ AdaptiveSparkPlan (143) : :- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42), Statistics(X) : : +- ColumnarExchange (41) - : : +- VeloxAppendBatches (40) + : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) : : +- ^ NoopFilter (37) : : +- ^ Scan parquet (36) @@ -70,7 +70,7 @@ AdaptiveSparkPlan (143) +- ^ InputIteratorTransformer (85) +- ShuffleQueryStage (83), Statistics(X) +- ColumnarExchange (82) - +- VeloxAppendBatches (81) + +- VeloxResizeBatches (81) +- ^ ProjectExecTransformer (79) +- ^ NoopFilter (78) +- ^ Scan parquet (77) @@ -145,9 +145,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] @@ -182,9 +182,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] @@ -219,9 +219,9 @@ Input [2]: [p_partkey#X, p_name#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -251,9 +251,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] @@ -288,9 +288,9 @@ Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] @@ -355,9 +355,9 @@ Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] @@ -387,9 +387,9 @@ Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: false -(64) VeloxAppendBatches +(64) VeloxResizeBatches Input [2]: [hash_partition_key#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (65) ColumnarExchange Input [2]: [hash_partition_key#X, ps_suppkey#X] @@ -419,9 +419,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(72) VeloxAppendBatches +(72) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (73) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] @@ -456,9 +456,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(81) VeloxAppendBatches +(81) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (82) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -488,9 +488,9 @@ Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] Input [2]: [s_name#X, s_address#X] Arguments: false -(89) VeloxAppendBatches +(89) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] -Arguments: X +Arguments: X, X (90) ColumnarExchange Input [2]: [s_name#X, s_address#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/21.txt index 279f4f0966927..1ffbdd45fef90 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/21.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (137) +- ^ InputIteratorTransformer (88) +- ShuffleQueryStage (86), Statistics(X) +- ColumnarExchange (85) - +- VeloxAppendBatches (84) + +- VeloxResizeBatches (84) +- ^ ProjectExecTransformer (82) +- ^ FlushableHashAggregateExecTransformer (81) +- ^ ProjectExecTransformer (80) @@ -14,60 +14,60 @@ AdaptiveSparkPlan (137) :- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (62) : :- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (45) : : :- ^ InputIteratorTransformer (9) : : : +- ShuffleQueryStage (7), Statistics(X) : : : +- ColumnarExchange (6) - : : : +- VeloxAppendBatches (5) + : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ NoopFilter (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42), Statistics(X) : : +- ColumnarExchange (41) - : : +- VeloxAppendBatches (40) + : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) : : +- ^ ShuffledHashJoinExecTransformer LeftAnti BuildRight (37) : : :- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (27) : : : :- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ NoopFilter (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (26) : : : +- ShuffleQueryStage (24), Statistics(X) : : : +- ColumnarExchange (23) - : : : +- VeloxAppendBatches (22) + : : : +- VeloxResizeBatches (22) : : : +- ^ ProjectExecTransformer (20) : : : +- ^ Scan parquet (19) : : +- ^ InputIteratorTransformer (36) : : +- ShuffleQueryStage (34), Statistics(X) : : +- ColumnarExchange (33) - : : +- VeloxAppendBatches (32) + : : +- VeloxResizeBatches (32) : : +- ^ ProjectExecTransformer (30) : : +- ^ NoopFilter (29) : : +- ^ Scan parquet (28) : +- ^ InputIteratorTransformer (61) : +- ShuffleQueryStage (59), Statistics(X) : +- ColumnarExchange (58) - : +- VeloxAppendBatches (57) + : +- VeloxResizeBatches (57) : +- ^ ProjectExecTransformer (55) : +- ^ NoopFilter (54) : +- ^ Scan parquet (53) +- ^ InputIteratorTransformer (78) +- ShuffleQueryStage (76), Statistics(X) +- ColumnarExchange (75) - +- VeloxAppendBatches (74) + +- VeloxResizeBatches (74) +- ^ ProjectExecTransformer (72) +- ^ NoopFilter (71) +- ^ Scan parquet (70) @@ -138,9 +138,9 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] @@ -175,9 +175,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -207,9 +207,9 @@ Input [2]: [l_orderkey#X, l_suppkey#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -250,9 +250,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(32) VeloxAppendBatches +(32) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (33) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -282,9 +282,9 @@ Input [2]: [l_orderkey#X, l_suppkey#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -314,9 +314,9 @@ Input [5]: [s_suppkey#X, s_name#X, s_nationkey#X, l_orderkey#X, l_suppkey#X] Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] @@ -351,9 +351,9 @@ Input [2]: [o_orderkey#X, o_orderstatus#X] Input [2]: [hash_partition_key#X, o_orderkey#X] Arguments: false -(57) VeloxAppendBatches +(57) VeloxResizeBatches Input [2]: [hash_partition_key#X, o_orderkey#X] -Arguments: X +Arguments: X, X (58) ColumnarExchange Input [2]: [hash_partition_key#X, o_orderkey#X] @@ -383,9 +383,9 @@ Input [4]: [s_name#X, s_nationkey#X, l_orderkey#X, o_orderkey#X] Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] @@ -420,9 +420,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(74) VeloxAppendBatches +(74) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (75) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -463,9 +463,9 @@ Input [2]: [s_name#X, count#X] Input [3]: [hash_partition_key#X, s_name#X, count#X] Arguments: false -(84) VeloxAppendBatches +(84) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, count#X] -Arguments: X +Arguments: X, X (85) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/22.txt index 2b93055014bd0..fd4141aec0af9 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/22.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (52) +- ^ InputIteratorTransformer (34) +- ShuffleQueryStage (32), Statistics(X) +- ColumnarExchange (31) - +- VeloxAppendBatches (30) + +- VeloxResizeBatches (30) +- ^ RegularHashAggregateExecTransformer (28) +- ^ InputIteratorTransformer (27) +- ShuffleQueryStage (25), Statistics(X) +- ColumnarExchange (24) - +- VeloxAppendBatches (23) + +- VeloxResizeBatches (23) +- ^ ProjectExecTransformer (21) +- ^ FlushableHashAggregateExecTransformer (20) +- ^ ProjectExecTransformer (19) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (52) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15), Statistics(X) +- ColumnarExchange (14) - +- VeloxAppendBatches (13) + +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) +- ^ Scan parquet (10) +- == Initial Plan == @@ -65,9 +65,9 @@ Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] @@ -97,9 +97,9 @@ Input [1]: [o_custkey#X] Input [2]: [hash_partition_key#X, o_custkey#X] Arguments: false -(13) VeloxAppendBatches +(13) VeloxResizeBatches Input [2]: [hash_partition_key#X, o_custkey#X] -Arguments: X +Arguments: X, X (14) ColumnarExchange Input [2]: [hash_partition_key#X, o_custkey#X] @@ -140,9 +140,9 @@ Input [4]: [cntrycode#X, count#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] @@ -169,9 +169,9 @@ Results [3]: [cntrycode#X, count(1)#X AS numcust#X, sum(c_acctbal#X)#X AS totacc Input [3]: [cntrycode#X, numcust#X, totacctbal#X] Arguments: false -(30) VeloxAppendBatches +(30) VeloxResizeBatches Input [3]: [cntrycode#X, numcust#X, totacctbal#X] -Arguments: X +Arguments: X, X (31) ColumnarExchange Input [3]: [cntrycode#X, numcust#X, totacctbal#X] @@ -281,7 +281,7 @@ AdaptiveSparkPlan (72) +- ^ InputIteratorTransformer (62) +- ShuffleQueryStage (60), Statistics(X) +- ColumnarExchange (59) - +- VeloxAppendBatches (58) + +- VeloxResizeBatches (58) +- ^ FlushableHashAggregateExecTransformer (56) +- ^ ProjectExecTransformer (55) +- ^ NoopFilter (54) @@ -321,9 +321,9 @@ Results [2]: [sum#X, count#X] Input [2]: [sum#X, count#X] Arguments: false -(58) VeloxAppendBatches +(58) VeloxResizeBatches Input [2]: [sum#X, count#X] -Arguments: X +Arguments: X, X (59) ColumnarExchange Input [2]: [sum#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/3.txt index aa679861da7c3..c7aed3993470e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/3.txt @@ -11,27 +11,27 @@ AdaptiveSparkPlan (67) :- ^ InputIteratorTransformer (26) : +- ShuffleQueryStage (24), Statistics(X) : +- ColumnarExchange (23) - : +- VeloxAppendBatches (22) + : +- VeloxResizeBatches (22) : +- ^ ProjectExecTransformer (20) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ NoopFilter (11) : +- ^ Scan parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ NoopFilter (28) +- ^ Scan parquet (27) @@ -80,9 +80,9 @@ Input [2]: [c_custkey#X, c_mktsegment#X] Input [2]: [hash_partition_key#X, c_custkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, c_custkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, c_custkey#X] @@ -117,9 +117,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] @@ -149,9 +149,9 @@ Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriorit Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] @@ -186,9 +186,9 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/4.txt index 02c494288f95c..72999dce97b3c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/4.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (56) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (56) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ NoopFilter (11) +- ^ Scan parquet (10) @@ -69,9 +69,9 @@ Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] @@ -106,9 +106,9 @@ Input [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Input [2]: [hash_partition_key#X, l_orderkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [2]: [hash_partition_key#X, l_orderkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [2]: [hash_partition_key#X, l_orderkey#X] @@ -149,9 +149,9 @@ Input [2]: [o_orderpriority#X, count#X] Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] @@ -178,9 +178,9 @@ Results [2]: [o_orderpriority#X, count(1)#X AS order_count#X] Input [2]: [o_orderpriority#X, order_count#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [2]: [o_orderpriority#X, order_count#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [2]: [o_orderpriority#X, order_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/5.txt index 67150984ab61e..388d18a0314d7 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/5.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (156) +- ^ InputIteratorTransformer (103) +- ShuffleQueryStage (101), Statistics(X) +- ColumnarExchange (100) - +- VeloxAppendBatches (99) + +- VeloxResizeBatches (99) +- ^ RegularHashAggregateExecTransformer (97) +- ^ InputIteratorTransformer (96) +- ShuffleQueryStage (94), Statistics(X) +- ColumnarExchange (93) - +- VeloxAppendBatches (92) + +- VeloxResizeBatches (92) +- ^ ProjectExecTransformer (90) +- ^ FlushableHashAggregateExecTransformer (89) +- ^ ProjectExecTransformer (88) @@ -19,66 +19,66 @@ AdaptiveSparkPlan (156) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75), Statistics(X) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58), Statistics(X) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41), Statistics(X) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ NoopFilter (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ NoopFilter (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ NoopFilter (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ NoopFilter (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ NoopFilter (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) - +- VeloxAppendBatches (82) + +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ NoopFilter (79) +- ^ Scan parquet (78) @@ -153,9 +153,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -190,9 +190,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -222,9 +222,9 @@ Input [4]: [c_custkey#X, c_nationkey#X, o_orderkey#X, o_custkey#X] Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] @@ -259,9 +259,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -291,9 +291,9 @@ Input [6]: [c_nationkey#X, o_orderkey#X, l_orderkey#X, l_suppkey#X, l_extendedpr Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -328,9 +328,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -360,9 +360,9 @@ Input [6]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppk Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -397,9 +397,9 @@ Input [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] @@ -429,9 +429,9 @@ Input [6]: [l_extendedprice#X, l_discount#X, s_nationkey#X, n_nationkey#X, n_nam Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] @@ -466,9 +466,9 @@ Input [2]: [r_regionkey#X, r_name#X] Input [2]: [hash_partition_key#X, r_regionkey#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [2]: [hash_partition_key#X, r_regionkey#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [2]: [hash_partition_key#X, r_regionkey#X] @@ -509,9 +509,9 @@ Input [3]: [n_name#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] Arguments: false -(92) VeloxAppendBatches +(92) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (93) ColumnarExchange Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] @@ -538,9 +538,9 @@ Results [2]: [n_name#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X AS reven Input [2]: [n_name#X, revenue#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [2]: [n_name#X, revenue#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [2]: [n_name#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/6.txt index 64624c791f726..89cf24d874744 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/6.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (20) +- ^ InputIteratorTransformer (10) +- ShuffleQueryStage (8), Statistics(X) +- ColumnarExchange (7) - +- VeloxAppendBatches (6) + +- VeloxResizeBatches (6) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ NoopFilter (2) @@ -46,9 +46,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(6) VeloxAppendBatches +(6) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (7) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/7.txt index 65dfab993c3c3..6284f06e5d2c5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/7.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (149) +- ^ InputIteratorTransformer (98) +- ShuffleQueryStage (96), Statistics(X) +- ColumnarExchange (95) - +- VeloxAppendBatches (94) + +- VeloxResizeBatches (94) +- ^ RegularHashAggregateExecTransformer (92) +- ^ InputIteratorTransformer (91) +- ShuffleQueryStage (89), Statistics(X) +- ColumnarExchange (88) - +- VeloxAppendBatches (87) + +- VeloxResizeBatches (87) +- ^ ProjectExecTransformer (85) +- ^ FlushableHashAggregateExecTransformer (84) +- ^ ProjectExecTransformer (83) @@ -19,59 +19,59 @@ AdaptiveSparkPlan (149) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75), Statistics(X) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58), Statistics(X) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41), Statistics(X) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ NoopFilter (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ NoopFilter (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ NoopFilter (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ NoopFilter (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ NoopFilter (62) : +- ^ Scan parquet (61) @@ -147,9 +147,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -184,9 +184,9 @@ Input [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipda Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] @@ -216,9 +216,9 @@ Input [7]: [s_suppkey#X, s_nationkey#X, l_orderkey#X, l_suppkey#X, l_extendedpri Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] @@ -253,9 +253,9 @@ Input [2]: [o_orderkey#X, o_custkey#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -285,9 +285,9 @@ Input [7]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_ship Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] @@ -322,9 +322,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -354,9 +354,9 @@ Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_cust Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] @@ -391,9 +391,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -423,9 +423,9 @@ Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nati Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] @@ -479,9 +479,9 @@ Input [5]: [supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Arguments: false -(87) VeloxAppendBatches +(87) VeloxResizeBatches Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (88) ColumnarExchange Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] @@ -508,9 +508,9 @@ Results [4]: [supp_nation#X, cust_nation#X, l_year#X, sum(volume#X)#X AS revenue Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] Arguments: false -(94) VeloxAppendBatches +(94) VeloxResizeBatches Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] -Arguments: X +Arguments: X, X (95) ColumnarExchange Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/8.txt index 61f6287c2429d..2dc7abc112bd4 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/8.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (207) +- ^ InputIteratorTransformer (138) +- ShuffleQueryStage (136), Statistics(X) +- ColumnarExchange (135) - +- VeloxAppendBatches (134) + +- VeloxResizeBatches (134) +- ^ ProjectExecTransformer (132) +- ^ RegularHashAggregateExecTransformer (131) +- ^ InputIteratorTransformer (130) +- ShuffleQueryStage (128), Statistics(X) +- ColumnarExchange (127) - +- VeloxAppendBatches (126) + +- VeloxResizeBatches (126) +- ^ ProjectExecTransformer (124) +- ^ FlushableHashAggregateExecTransformer (123) +- ^ ProjectExecTransformer (122) @@ -20,92 +20,92 @@ AdaptiveSparkPlan (207) :- ^ InputIteratorTransformer (111) : +- ShuffleQueryStage (109), Statistics(X) : +- ColumnarExchange (108) - : +- VeloxAppendBatches (107) + : +- VeloxResizeBatches (107) : +- ^ ProjectExecTransformer (105) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (104) : :- ^ InputIteratorTransformer (94) : : +- ShuffleQueryStage (92), Statistics(X) : : +- ColumnarExchange (91) - : : +- VeloxAppendBatches (90) + : : +- VeloxResizeBatches (90) : : +- ^ ProjectExecTransformer (88) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (87) : : :- ^ InputIteratorTransformer (77) : : : +- ShuffleQueryStage (75), Statistics(X) : : : +- ColumnarExchange (74) - : : : +- VeloxAppendBatches (73) + : : : +- VeloxResizeBatches (73) : : : +- ^ ProjectExecTransformer (71) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (70) : : : :- ^ InputIteratorTransformer (60) : : : : +- ShuffleQueryStage (58), Statistics(X) : : : : +- ColumnarExchange (57) - : : : : +- VeloxAppendBatches (56) + : : : : +- VeloxResizeBatches (56) : : : : +- ^ ProjectExecTransformer (54) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (53) : : : : :- ^ InputIteratorTransformer (43) : : : : : +- ShuffleQueryStage (41), Statistics(X) : : : : : +- ColumnarExchange (40) - : : : : : +- VeloxAppendBatches (39) + : : : : : +- VeloxResizeBatches (39) : : : : : +- ^ ProjectExecTransformer (37) : : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : : : :- ^ InputIteratorTransformer (26) : : : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : : : +- ColumnarExchange (23) - : : : : : : +- VeloxAppendBatches (22) + : : : : : : +- VeloxResizeBatches (22) : : : : : : +- ^ ProjectExecTransformer (20) : : : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : : : :- ^ InputIteratorTransformer (9) : : : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : : : +- ColumnarExchange (6) - : : : : : : : +- VeloxAppendBatches (5) + : : : : : : : +- VeloxResizeBatches (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ NoopFilter (2) : : : : : : : +- ^ Scan parquet (1) : : : : : : +- ^ InputIteratorTransformer (18) : : : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : : : +- ColumnarExchange (15) - : : : : : : +- VeloxAppendBatches (14) + : : : : : : +- VeloxResizeBatches (14) : : : : : : +- ^ ProjectExecTransformer (12) : : : : : : +- ^ NoopFilter (11) : : : : : : +- ^ Scan parquet (10) : : : : : +- ^ InputIteratorTransformer (35) : : : : : +- ShuffleQueryStage (33), Statistics(X) : : : : : +- ColumnarExchange (32) - : : : : : +- VeloxAppendBatches (31) + : : : : : +- VeloxResizeBatches (31) : : : : : +- ^ ProjectExecTransformer (29) : : : : : +- ^ NoopFilter (28) : : : : : +- ^ Scan parquet (27) : : : : +- ^ InputIteratorTransformer (52) : : : : +- ShuffleQueryStage (50), Statistics(X) : : : : +- ColumnarExchange (49) - : : : : +- VeloxAppendBatches (48) + : : : : +- VeloxResizeBatches (48) : : : : +- ^ ProjectExecTransformer (46) : : : : +- ^ NoopFilter (45) : : : : +- ^ Scan parquet (44) : : : +- ^ InputIteratorTransformer (69) : : : +- ShuffleQueryStage (67), Statistics(X) : : : +- ColumnarExchange (66) - : : : +- VeloxAppendBatches (65) + : : : +- VeloxResizeBatches (65) : : : +- ^ ProjectExecTransformer (63) : : : +- ^ NoopFilter (62) : : : +- ^ Scan parquet (61) : : +- ^ InputIteratorTransformer (86) : : +- ShuffleQueryStage (84), Statistics(X) : : +- ColumnarExchange (83) - : : +- VeloxAppendBatches (82) + : : +- VeloxResizeBatches (82) : : +- ^ ProjectExecTransformer (80) : : +- ^ NoopFilter (79) : : +- ^ Scan parquet (78) : +- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101), Statistics(X) : +- ColumnarExchange (100) - : +- VeloxAppendBatches (99) + : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ NoopFilter (96) : +- ^ Scan parquet (95) +- ^ InputIteratorTransformer (120) +- ShuffleQueryStage (118), Statistics(X) +- ColumnarExchange (117) - +- VeloxAppendBatches (116) + +- VeloxResizeBatches (116) +- ^ ProjectExecTransformer (114) +- ^ NoopFilter (113) +- ^ Scan parquet (112) @@ -196,9 +196,9 @@ Input [2]: [p_partkey#X, p_type#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -233,9 +233,9 @@ Input [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discoun Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -265,9 +265,9 @@ Input [6]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -302,9 +302,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -334,9 +334,9 @@ Input [6]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppke Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -371,9 +371,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] @@ -403,9 +403,9 @@ Input [7]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orde Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] @@ -440,9 +440,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -472,9 +472,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_order Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] @@ -509,9 +509,9 @@ Input [2]: [n_nationkey#X, n_regionkey#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] @@ -541,9 +541,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nat Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] Arguments: false -(90) VeloxAppendBatches +(90) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] -Arguments: X +Arguments: X, X (91) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] @@ -578,9 +578,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -610,9 +610,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_reg Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] Arguments: false -(107) VeloxAppendBatches +(107) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] -Arguments: X +Arguments: X, X (108) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] @@ -647,9 +647,9 @@ Input [2]: [r_regionkey#X, r_name#X] Input [2]: [hash_partition_key#X, r_regionkey#X] Arguments: false -(116) VeloxAppendBatches +(116) VeloxResizeBatches Input [2]: [hash_partition_key#X, r_regionkey#X] -Arguments: X +Arguments: X, X (117) ColumnarExchange Input [2]: [hash_partition_key#X, r_regionkey#X] @@ -690,9 +690,9 @@ Input [5]: [o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(126) VeloxAppendBatches +(126) VeloxResizeBatches Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (127) ColumnarExchange Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] @@ -723,9 +723,9 @@ Input [3]: [o_year#X, sum(CASE WHEN (nation#X = BRAZIL) THEN volume#X ELSE 0.000 Input [2]: [o_year#X, mkt_share#X] Arguments: false -(134) VeloxAppendBatches +(134) VeloxResizeBatches Input [2]: [o_year#X, mkt_share#X] -Arguments: X +Arguments: X, X (135) ColumnarExchange Input [2]: [o_year#X, mkt_share#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/9.txt index 4b983de23fdec..ce095ca705bcf 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/9.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (155) +- ^ InputIteratorTransformer (103) +- ShuffleQueryStage (101), Statistics(X) +- ColumnarExchange (100) - +- VeloxAppendBatches (99) + +- VeloxResizeBatches (99) +- ^ RegularHashAggregateExecTransformer (97) +- ^ InputIteratorTransformer (96) +- ShuffleQueryStage (94), Statistics(X) +- ColumnarExchange (93) - +- VeloxAppendBatches (92) + +- VeloxResizeBatches (92) +- ^ ProjectExecTransformer (90) +- ^ FlushableHashAggregateExecTransformer (89) +- ^ ProjectExecTransformer (88) @@ -19,66 +19,66 @@ AdaptiveSparkPlan (155) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75), Statistics(X) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58), Statistics(X) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41), Statistics(X) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ NoopFilter (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ NoopFilter (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ NoopFilter (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ NoopFilter (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ NoopFilter (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) - +- VeloxAppendBatches (82) + +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ NoopFilter (79) +- ^ Scan parquet (78) @@ -152,9 +152,9 @@ Input [2]: [p_partkey#X, p_name#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -189,9 +189,9 @@ Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -221,9 +221,9 @@ Input [7]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -258,9 +258,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -290,9 +290,9 @@ Input [8]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -327,9 +327,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] @@ -359,9 +359,9 @@ Input [10]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpri Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] @@ -396,9 +396,9 @@ Input [2]: [o_orderkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] @@ -428,9 +428,9 @@ Input [8]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_natio Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] @@ -465,9 +465,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -508,9 +508,9 @@ Input [4]: [nation#X, o_year#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] Arguments: false -(92) VeloxAppendBatches +(92) VeloxResizeBatches Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (93) ColumnarExchange Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] @@ -537,9 +537,9 @@ Results [3]: [nation#X, o_year#X, sum(amount#X)#X AS sum_profit#X] Input [3]: [nation#X, o_year#X, sum_profit#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [3]: [nation#X, o_year#X, sum_profit#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [3]: [nation#X, o_year#X, sum_profit#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/1.txt index 63b7d317f3cf2..5ceb73b301db8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/1.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (30) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ RegularHashAggregateExecTransformer (12) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ ProjectExecTransformer (5) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) @@ -58,9 +58,9 @@ Input [17]: [l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] @@ -87,9 +87,9 @@ Results [10]: [l_returnflag#X, l_linestatus#X, sum(l_quantity#X)#X AS sum_qty#X, Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/10.txt index 993884df3f3ac..b6c3dc67a1a5d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/10.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (100) +- ^ InputIteratorTransformer (62) +- ShuffleQueryStage (60) +- ColumnarExchange (59) - +- VeloxAppendBatches (58) + +- VeloxResizeBatches (58) +- ^ ProjectExecTransformer (56) +- ^ FlushableHashAggregateExecTransformer (55) +- ^ ProjectExecTransformer (54) @@ -16,40 +16,40 @@ AdaptiveSparkPlan (100) :- ^ InputIteratorTransformer (43) : +- ShuffleQueryStage (41) : +- ColumnarExchange (40) - : +- VeloxAppendBatches (39) + : +- VeloxResizeBatches (39) : +- ^ ProjectExecTransformer (37) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) : :- ^ InputIteratorTransformer (26) : : +- ShuffleQueryStage (24) : : +- ColumnarExchange (23) - : : +- VeloxAppendBatches (22) + : : +- VeloxResizeBatches (22) : : +- ^ ProjectExecTransformer (20) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) : : :- ^ InputIteratorTransformer (9) : : : +- ShuffleQueryStage (7) : : : +- ColumnarExchange (6) - : : : +- VeloxAppendBatches (5) + : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16) : : +- ColumnarExchange (15) - : : +- VeloxAppendBatches (14) + : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ FilterExecTransformer (11) : : +- ^ Scan parquet (10) : +- ^ InputIteratorTransformer (35) : +- ShuffleQueryStage (33) : +- ColumnarExchange (32) - : +- VeloxAppendBatches (31) + : +- VeloxResizeBatches (31) : +- ^ ProjectExecTransformer (29) : +- ^ FilterExecTransformer (28) : +- ^ Scan parquet (27) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50) +- ColumnarExchange (49) - +- VeloxAppendBatches (48) + +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ FilterExecTransformer (45) +- ^ Scan parquet (44) @@ -107,9 +107,9 @@ Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] @@ -144,9 +144,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -175,9 +175,9 @@ Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] @@ -212,9 +212,9 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] @@ -243,9 +243,9 @@ Input [11]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acc Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] @@ -280,9 +280,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -322,9 +322,9 @@ Input [9]: [c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] Arguments: false -(58) VeloxAppendBatches +(58) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (59) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/11.txt index 8142375d9ead2..48d14d1812d18 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/11.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (82) +- ^ InputIteratorTransformer (53) +- ShuffleQueryStage (51) +- ColumnarExchange (50) - +- VeloxAppendBatches (49) + +- VeloxResizeBatches (49) +- ^ FilterExecTransformer (47) +- ^ RegularHashAggregateExecTransformer (46) +- ^ InputIteratorTransformer (45) +- ShuffleQueryStage (43) +- ColumnarExchange (42) - +- VeloxAppendBatches (41) + +- VeloxResizeBatches (41) +- ^ ProjectExecTransformer (39) +- ^ FlushableHashAggregateExecTransformer (38) +- ^ ProjectExecTransformer (37) @@ -20,27 +20,27 @@ AdaptiveSparkPlan (82) :- ^ InputIteratorTransformer (26) : +- ShuffleQueryStage (24) : +- ColumnarExchange (23) - : +- VeloxAppendBatches (22) + : +- VeloxResizeBatches (22) : +- ^ ProjectExecTransformer (20) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) : +- ^ Scan parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ FilterExecTransformer (28) +- ^ Scan parquet (27) @@ -91,9 +91,9 @@ Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] @@ -128,9 +128,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -159,9 +159,9 @@ Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppke Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] @@ -196,9 +196,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -238,9 +238,9 @@ Input [3]: [ps_partkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] Arguments: false -(41) VeloxAppendBatches +(41) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (42) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] @@ -271,9 +271,9 @@ Arguments: (isnotnull(value#X) AND (cast(value#X as decimal(38,6)) > Subquery su Input [2]: [ps_partkey#X, value#X] Arguments: false -(49) VeloxAppendBatches +(49) VeloxResizeBatches Input [2]: [ps_partkey#X, value#X] -Arguments: X +Arguments: X, X (50) ColumnarExchange Input [2]: [ps_partkey#X, value#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/12.txt index 802a79759235a..0a836a6d8ed77 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/12.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (55) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) @@ -68,9 +68,9 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] @@ -105,9 +105,9 @@ Input [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipm Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] @@ -147,9 +147,9 @@ Input [3]: [l_shipmode#X, sum#X, sum#X] Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] @@ -176,9 +176,9 @@ Results [3]: [l_shipmode#X, sum(CASE WHEN ((o_orderpriority#X = 1-URGENT) OR (o_ Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/13.txt index b9bf0f1fad603..522a695f73b16 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/13.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (58) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ RegularHashAggregateExecTransformer (31) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ ProjectExecTransformer (22) @@ -22,13 +22,13 @@ AdaptiveSparkPlan (58) :- ^ InputIteratorTransformer (8) : +- ShuffleQueryStage (6) : +- ColumnarExchange (5) - : +- VeloxAppendBatches (4) + : +- VeloxResizeBatches (4) : +- ^ ProjectExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15) +- ColumnarExchange (14) - +- VeloxAppendBatches (13) + +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) +- ^ FilterExecTransformer (10) +- ^ Scan parquet (9) @@ -66,9 +66,9 @@ Input [1]: [c_custkey#X] Input [2]: [hash_partition_key#X, c_custkey#X] Arguments: false -(4) VeloxAppendBatches +(4) VeloxResizeBatches Input [2]: [hash_partition_key#X, c_custkey#X] -Arguments: X +Arguments: X, X (5) ColumnarExchange Input [2]: [hash_partition_key#X, c_custkey#X] @@ -103,9 +103,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(13) VeloxAppendBatches +(13) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (14) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -163,9 +163,9 @@ Input [2]: [c_count#X, count#X] Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] @@ -192,9 +192,9 @@ Results [2]: [c_count#X, count(1)#X AS custdist#X] Input [2]: [c_count#X, custdist#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [2]: [c_count#X, custdist#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/14.txt index 425c55f5a4ce2..d050850332af2 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/14.txt @@ -10,14 +10,14 @@ AdaptiveSparkPlan (39) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) @@ -56,9 +56,9 @@ Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] @@ -93,9 +93,9 @@ Input [2]: [p_partkey#X, p_type#X] Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/15.txt index f003eed994d08..1d085e52a5d4d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/15.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (50) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (23) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) @@ -21,7 +21,7 @@ AdaptiveSparkPlan (50) +- ^ InputIteratorTransformer (20) +- ShuffleQueryStage (18) +- ColumnarExchange (17) - +- VeloxAppendBatches (16) + +- VeloxResizeBatches (16) +- ^ ProjectExecTransformer (14) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) @@ -65,9 +65,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] @@ -113,9 +113,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(16) VeloxAppendBatches +(16) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (17) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] @@ -155,9 +155,9 @@ Input [6]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, supplier_no#X, total_ Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/16.txt index c9374b01ff021..4b6a18e968609 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/16.txt @@ -6,19 +6,19 @@ AdaptiveSparkPlan (71) +- ^ InputIteratorTransformer (44) +- ShuffleQueryStage (42) +- ColumnarExchange (41) - +- VeloxAppendBatches (40) + +- VeloxResizeBatches (40) +- ^ RegularHashAggregateExecTransformer (38) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ ProjectExecTransformer (31) +- ^ FlushableHashAggregateExecTransformer (30) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -26,14 +26,14 @@ AdaptiveSparkPlan (71) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) @@ -82,9 +82,9 @@ Input [2]: [ps_partkey#X, ps_suppkey#X] Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] @@ -119,9 +119,9 @@ Input [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] @@ -161,9 +161,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] @@ -201,9 +201,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, count#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] @@ -230,9 +230,9 @@ Results [4]: [p_brand#X, p_type#X, p_size#X, count(ps_suppkey#X)#X AS supplier_c Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/17.txt index 69f50fa16ab02..6ebaa345a5a99 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/17.txt @@ -12,14 +12,14 @@ AdaptiveSparkPlan (63) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) : +- ^ Scan parquet (10) @@ -29,7 +29,7 @@ AdaptiveSparkPlan (63) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ FilterExecTransformer (22) @@ -78,9 +78,9 @@ Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] @@ -115,9 +115,9 @@ Input [3]: [p_partkey#X, p_brand#X, p_container#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -168,9 +168,9 @@ Input [3]: [l_partkey#X, sum#X, count#X] Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/18.txt index 96fa1cd826060..de38e4d98b651 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/18.txt @@ -10,26 +10,26 @@ AdaptiveSparkPlan (110) :- ^ InputIteratorTransformer (46) : +- ShuffleQueryStage (44) : +- ColumnarExchange (43) - : +- VeloxAppendBatches (42) + : +- VeloxResizeBatches (42) : +- ^ ProjectExecTransformer (40) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (38) : +- ShuffleQueryStage (36) : +- ColumnarExchange (35) - : +- VeloxAppendBatches (34) + : +- VeloxResizeBatches (34) : +- ^ ProjectExecTransformer (32) : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) : :- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16) : : +- ColumnarExchange (15) - : : +- VeloxAppendBatches (14) + : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ FilterExecTransformer (11) : : +- ^ Scan parquet (10) @@ -39,7 +39,7 @@ AdaptiveSparkPlan (110) : +- ^ InputIteratorTransformer (27) : +- ShuffleQueryStage (25) : +- ColumnarExchange (24) - : +- VeloxAppendBatches (23) + : +- VeloxResizeBatches (23) : +- ^ ProjectExecTransformer (21) : +- ^ FlushableHashAggregateExecTransformer (20) : +- ^ Scan parquet (19) @@ -47,7 +47,7 @@ AdaptiveSparkPlan (110) :- ^ InputIteratorTransformer (55) : +- ShuffleQueryStage (53) : +- ColumnarExchange (52) - : +- VeloxAppendBatches (51) + : +- VeloxResizeBatches (51) : +- ^ ProjectExecTransformer (49) : +- ^ FilterExecTransformer (48) : +- ^ Scan parquet (47) @@ -118,9 +118,9 @@ Input [2]: [c_custkey#X, c_name#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] @@ -155,9 +155,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] @@ -194,9 +194,9 @@ Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] @@ -240,9 +240,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(34) VeloxAppendBatches +(34) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (35) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] @@ -271,9 +271,9 @@ Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_ Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(42) VeloxAppendBatches +(42) VeloxResizeBatches Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (43) ColumnarExchange Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] @@ -308,9 +308,9 @@ Input [2]: [l_orderkey#X, l_quantity#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] Arguments: false -(51) VeloxAppendBatches +(51) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] -Arguments: X +Arguments: X, X (52) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/19.txt index ae0feb5dfd565..84520a9699026 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/19.txt @@ -9,14 +9,14 @@ AdaptiveSparkPlan (38) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) @@ -55,9 +55,9 @@ Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipin Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -92,9 +92,9 @@ Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/20.txt index bab7855516360..d581db858a0c1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/20.txt @@ -6,52 +6,52 @@ AdaptiveSparkPlan (146) +- ^ InputIteratorTransformer (93) +- ShuffleQueryStage (91) +- ColumnarExchange (90) - +- VeloxAppendBatches (89) + +- VeloxResizeBatches (89) +- ^ ProjectExecTransformer (87) +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (86) :- ^ InputIteratorTransformer (76) : +- ShuffleQueryStage (74) : +- ColumnarExchange (73) - : +- VeloxAppendBatches (72) + : +- VeloxResizeBatches (72) : +- ^ ProjectExecTransformer (70) : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (69) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (68) : +- ShuffleQueryStage (66) : +- ColumnarExchange (65) - : +- VeloxAppendBatches (64) + : +- VeloxResizeBatches (64) : +- ^ ProjectExecTransformer (62) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (61) : :- ^ InputIteratorTransformer (35) : : +- ShuffleQueryStage (33) : : +- ColumnarExchange (32) - : : +- VeloxAppendBatches (31) + : : +- VeloxResizeBatches (31) : : +- ^ ProjectExecTransformer (29) : : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (28) : : :- ^ InputIteratorTransformer (18) : : : +- ShuffleQueryStage (16) : : : +- ColumnarExchange (15) - : : : +- VeloxAppendBatches (14) + : : : +- VeloxResizeBatches (14) : : : +- ^ ProjectExecTransformer (12) : : : +- ^ FilterExecTransformer (11) : : : +- ^ Scan parquet (10) : : +- ^ InputIteratorTransformer (27) : : +- ShuffleQueryStage (25) : : +- ColumnarExchange (24) - : : +- VeloxAppendBatches (23) + : : +- VeloxResizeBatches (23) : : +- ^ ProjectExecTransformer (21) : : +- ^ FilterExecTransformer (20) : : +- ^ Scan parquet (19) : +- ^ InputIteratorTransformer (60) : +- ShuffleQueryStage (58) : +- ColumnarExchange (57) - : +- VeloxAppendBatches (56) + : +- VeloxResizeBatches (56) : +- ^ ProjectExecTransformer (54) : +- ^ FilterExecTransformer (53) : +- ^ ProjectExecTransformer (52) @@ -61,7 +61,7 @@ AdaptiveSparkPlan (146) : :- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42) : : +- ColumnarExchange (41) - : : +- VeloxAppendBatches (40) + : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) : : +- ^ FilterExecTransformer (37) : : +- ^ Scan parquet (36) @@ -71,7 +71,7 @@ AdaptiveSparkPlan (146) +- ^ InputIteratorTransformer (85) +- ShuffleQueryStage (83) +- ColumnarExchange (82) - +- VeloxAppendBatches (81) + +- VeloxResizeBatches (81) +- ^ ProjectExecTransformer (79) +- ^ FilterExecTransformer (78) +- ^ Scan parquet (77) @@ -146,9 +146,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] @@ -183,9 +183,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] @@ -220,9 +220,9 @@ Input [2]: [p_partkey#X, p_name#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -251,9 +251,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] @@ -288,9 +288,9 @@ Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] @@ -354,9 +354,9 @@ Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] @@ -385,9 +385,9 @@ Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: false -(64) VeloxAppendBatches +(64) VeloxResizeBatches Input [2]: [hash_partition_key#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (65) ColumnarExchange Input [2]: [hash_partition_key#X, ps_suppkey#X] @@ -416,9 +416,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(72) VeloxAppendBatches +(72) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (73) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] @@ -453,9 +453,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(81) VeloxAppendBatches +(81) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (82) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -484,9 +484,9 @@ Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] Input [2]: [s_name#X, s_address#X] Arguments: false -(89) VeloxAppendBatches +(89) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] -Arguments: X +Arguments: X, X (90) ColumnarExchange Input [2]: [s_name#X, s_address#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/21.txt index ef4e87bb1de4d..c9375e9823a34 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/21.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (138) +- ^ InputIteratorTransformer (88) +- ShuffleQueryStage (86) +- ColumnarExchange (85) - +- VeloxAppendBatches (84) + +- VeloxResizeBatches (84) +- ^ ProjectExecTransformer (82) +- ^ FlushableHashAggregateExecTransformer (81) +- ^ ProjectExecTransformer (80) @@ -15,60 +15,60 @@ AdaptiveSparkPlan (138) :- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (62) : :- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (45) : : :- ^ InputIteratorTransformer (9) : : : +- ShuffleQueryStage (7) : : : +- ColumnarExchange (6) - : : : +- VeloxAppendBatches (5) + : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42) : : +- ColumnarExchange (41) - : : +- VeloxAppendBatches (40) + : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) : : +- ^ ShuffledHashJoinExecTransformer LeftAnti BuildRight (37) : : :- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (27) : : : :- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (26) : : : +- ShuffleQueryStage (24) : : : +- ColumnarExchange (23) - : : : +- VeloxAppendBatches (22) + : : : +- VeloxResizeBatches (22) : : : +- ^ ProjectExecTransformer (20) : : : +- ^ Scan parquet (19) : : +- ^ InputIteratorTransformer (36) : : +- ShuffleQueryStage (34) : : +- ColumnarExchange (33) - : : +- VeloxAppendBatches (32) + : : +- VeloxResizeBatches (32) : : +- ^ ProjectExecTransformer (30) : : +- ^ FilterExecTransformer (29) : : +- ^ Scan parquet (28) : +- ^ InputIteratorTransformer (61) : +- ShuffleQueryStage (59) : +- ColumnarExchange (58) - : +- VeloxAppendBatches (57) + : +- VeloxResizeBatches (57) : +- ^ ProjectExecTransformer (55) : +- ^ FilterExecTransformer (54) : +- ^ Scan parquet (53) +- ^ InputIteratorTransformer (78) +- ShuffleQueryStage (76) +- ColumnarExchange (75) - +- VeloxAppendBatches (74) + +- VeloxResizeBatches (74) +- ^ ProjectExecTransformer (72) +- ^ FilterExecTransformer (71) +- ^ Scan parquet (70) @@ -139,9 +139,9 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] @@ -176,9 +176,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -208,9 +208,9 @@ Input [2]: [l_orderkey#X, l_suppkey#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -250,9 +250,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(32) VeloxAppendBatches +(32) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (33) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -281,9 +281,9 @@ Input [2]: [l_orderkey#X, l_suppkey#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -312,9 +312,9 @@ Input [5]: [s_suppkey#X, s_name#X, s_nationkey#X, l_orderkey#X, l_suppkey#X] Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] @@ -349,9 +349,9 @@ Input [2]: [o_orderkey#X, o_orderstatus#X] Input [2]: [hash_partition_key#X, o_orderkey#X] Arguments: false -(57) VeloxAppendBatches +(57) VeloxResizeBatches Input [2]: [hash_partition_key#X, o_orderkey#X] -Arguments: X +Arguments: X, X (58) ColumnarExchange Input [2]: [hash_partition_key#X, o_orderkey#X] @@ -380,9 +380,9 @@ Input [4]: [s_name#X, s_nationkey#X, l_orderkey#X, o_orderkey#X] Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] @@ -417,9 +417,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(74) VeloxAppendBatches +(74) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (75) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -459,9 +459,9 @@ Input [2]: [s_name#X, count#X] Input [3]: [hash_partition_key#X, s_name#X, count#X] Arguments: false -(84) VeloxAppendBatches +(84) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, count#X] -Arguments: X +Arguments: X, X (85) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/22.txt index fcf712a9d5fdf..9e899a1b5e15b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/22.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (52) +- ^ InputIteratorTransformer (34) +- ShuffleQueryStage (32) +- ColumnarExchange (31) - +- VeloxAppendBatches (30) + +- VeloxResizeBatches (30) +- ^ RegularHashAggregateExecTransformer (28) +- ^ InputIteratorTransformer (27) +- ShuffleQueryStage (25) +- ColumnarExchange (24) - +- VeloxAppendBatches (23) + +- VeloxResizeBatches (23) +- ^ ProjectExecTransformer (21) +- ^ FlushableHashAggregateExecTransformer (20) +- ^ ProjectExecTransformer (19) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (52) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15) +- ColumnarExchange (14) - +- VeloxAppendBatches (13) + +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) +- ^ Scan parquet (10) +- == Initial Plan == @@ -65,9 +65,9 @@ Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] @@ -97,9 +97,9 @@ Input [1]: [o_custkey#X] Input [2]: [hash_partition_key#X, o_custkey#X] Arguments: false -(13) VeloxAppendBatches +(13) VeloxResizeBatches Input [2]: [hash_partition_key#X, o_custkey#X] -Arguments: X +Arguments: X, X (14) ColumnarExchange Input [2]: [hash_partition_key#X, o_custkey#X] @@ -139,9 +139,9 @@ Input [4]: [cntrycode#X, count#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] @@ -168,9 +168,9 @@ Results [3]: [cntrycode#X, count(1)#X AS numcust#X, sum(c_acctbal#X)#X AS totacc Input [3]: [cntrycode#X, numcust#X, totacctbal#X] Arguments: false -(30) VeloxAppendBatches +(30) VeloxResizeBatches Input [3]: [cntrycode#X, numcust#X, totacctbal#X] -Arguments: X +Arguments: X, X (31) ColumnarExchange Input [3]: [cntrycode#X, numcust#X, totacctbal#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/3.txt index 607d6444f4324..a3fb4062c83d0 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/3.txt @@ -11,27 +11,27 @@ AdaptiveSparkPlan (67) :- ^ InputIteratorTransformer (26) : +- ShuffleQueryStage (24) : +- ColumnarExchange (23) - : +- VeloxAppendBatches (22) + : +- VeloxResizeBatches (22) : +- ^ ProjectExecTransformer (20) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) : +- ^ Scan parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ FilterExecTransformer (28) +- ^ Scan parquet (27) @@ -80,9 +80,9 @@ Input [2]: [c_custkey#X, c_mktsegment#X] Input [2]: [hash_partition_key#X, c_custkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, c_custkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, c_custkey#X] @@ -117,9 +117,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] @@ -148,9 +148,9 @@ Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriorit Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] @@ -185,9 +185,9 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/4.txt index cc6b8f351600a..eae6f66bbb18b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/4.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (56) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (56) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) @@ -69,9 +69,9 @@ Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] @@ -106,9 +106,9 @@ Input [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Input [2]: [hash_partition_key#X, l_orderkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [2]: [hash_partition_key#X, l_orderkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [2]: [hash_partition_key#X, l_orderkey#X] @@ -148,9 +148,9 @@ Input [2]: [o_orderpriority#X, count#X] Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] @@ -177,9 +177,9 @@ Results [2]: [o_orderpriority#X, count(1)#X AS order_count#X] Input [2]: [o_orderpriority#X, order_count#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [2]: [o_orderpriority#X, order_count#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [2]: [o_orderpriority#X, order_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/5.txt index a1f95887aae33..662616ac077a8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/5.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (156) +- ^ InputIteratorTransformer (103) +- ShuffleQueryStage (101) +- ColumnarExchange (100) - +- VeloxAppendBatches (99) + +- VeloxResizeBatches (99) +- ^ RegularHashAggregateExecTransformer (97) +- ^ InputIteratorTransformer (96) +- ShuffleQueryStage (94) +- ColumnarExchange (93) - +- VeloxAppendBatches (92) + +- VeloxResizeBatches (92) +- ^ ProjectExecTransformer (90) +- ^ FlushableHashAggregateExecTransformer (89) +- ^ ProjectExecTransformer (88) @@ -19,66 +19,66 @@ AdaptiveSparkPlan (156) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84) +- ColumnarExchange (83) - +- VeloxAppendBatches (82) + +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) +- ^ Scan parquet (78) @@ -153,9 +153,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -190,9 +190,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -221,9 +221,9 @@ Input [4]: [c_custkey#X, c_nationkey#X, o_orderkey#X, o_custkey#X] Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] @@ -258,9 +258,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -289,9 +289,9 @@ Input [6]: [c_nationkey#X, o_orderkey#X, l_orderkey#X, l_suppkey#X, l_extendedpr Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -326,9 +326,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -357,9 +357,9 @@ Input [6]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppk Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -394,9 +394,9 @@ Input [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] @@ -425,9 +425,9 @@ Input [6]: [l_extendedprice#X, l_discount#X, s_nationkey#X, n_nationkey#X, n_nam Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] @@ -462,9 +462,9 @@ Input [2]: [r_regionkey#X, r_name#X] Input [2]: [hash_partition_key#X, r_regionkey#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [2]: [hash_partition_key#X, r_regionkey#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [2]: [hash_partition_key#X, r_regionkey#X] @@ -504,9 +504,9 @@ Input [3]: [n_name#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] Arguments: false -(92) VeloxAppendBatches +(92) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (93) ColumnarExchange Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] @@ -533,9 +533,9 @@ Results [2]: [n_name#X, sum(CheckOverflow((promote_precision(cast(l_extendedpric Input [2]: [n_name#X, revenue#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [2]: [n_name#X, revenue#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [2]: [n_name#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/6.txt index 786a89fe715ae..ee8c494f56a30 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/6.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (20) +- ^ InputIteratorTransformer (10) +- ShuffleQueryStage (8) +- ColumnarExchange (7) - +- VeloxAppendBatches (6) + +- VeloxResizeBatches (6) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) @@ -46,9 +46,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(6) VeloxAppendBatches +(6) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (7) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/7.txt index 64d51413a0849..ef168ad7bb8ff 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/7.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (149) +- ^ InputIteratorTransformer (98) +- ShuffleQueryStage (96) +- ColumnarExchange (95) - +- VeloxAppendBatches (94) + +- VeloxResizeBatches (94) +- ^ RegularHashAggregateExecTransformer (92) +- ^ InputIteratorTransformer (91) +- ShuffleQueryStage (89) +- ColumnarExchange (88) - +- VeloxAppendBatches (87) + +- VeloxResizeBatches (87) +- ^ ProjectExecTransformer (85) +- ^ FlushableHashAggregateExecTransformer (84) +- ^ ProjectExecTransformer (83) @@ -19,59 +19,59 @@ AdaptiveSparkPlan (149) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) : +- ^ Scan parquet (61) @@ -147,9 +147,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -184,9 +184,9 @@ Input [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipda Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] @@ -215,9 +215,9 @@ Input [7]: [s_suppkey#X, s_nationkey#X, l_orderkey#X, l_suppkey#X, l_extendedpri Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] @@ -252,9 +252,9 @@ Input [2]: [o_orderkey#X, o_custkey#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -283,9 +283,9 @@ Input [7]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_ship Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] @@ -320,9 +320,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -351,9 +351,9 @@ Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_cust Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] @@ -388,9 +388,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -419,9 +419,9 @@ Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nati Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] @@ -474,9 +474,9 @@ Input [5]: [supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Arguments: false -(87) VeloxAppendBatches +(87) VeloxResizeBatches Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (88) ColumnarExchange Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] @@ -503,9 +503,9 @@ Results [4]: [supp_nation#X, cust_nation#X, l_year#X, sum(volume#X)#X AS revenue Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] Arguments: false -(94) VeloxAppendBatches +(94) VeloxResizeBatches Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] -Arguments: X +Arguments: X, X (95) ColumnarExchange Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/8.txt index 8934d1a2e7a39..55e4f19764d1d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/8.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (207) +- ^ InputIteratorTransformer (138) +- ShuffleQueryStage (136) +- ColumnarExchange (135) - +- VeloxAppendBatches (134) + +- VeloxResizeBatches (134) +- ^ ProjectExecTransformer (132) +- ^ RegularHashAggregateExecTransformer (131) +- ^ InputIteratorTransformer (130) +- ShuffleQueryStage (128) +- ColumnarExchange (127) - +- VeloxAppendBatches (126) + +- VeloxResizeBatches (126) +- ^ ProjectExecTransformer (124) +- ^ FlushableHashAggregateExecTransformer (123) +- ^ ProjectExecTransformer (122) @@ -20,92 +20,92 @@ AdaptiveSparkPlan (207) :- ^ InputIteratorTransformer (111) : +- ShuffleQueryStage (109) : +- ColumnarExchange (108) - : +- VeloxAppendBatches (107) + : +- VeloxResizeBatches (107) : +- ^ ProjectExecTransformer (105) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (104) : :- ^ InputIteratorTransformer (94) : : +- ShuffleQueryStage (92) : : +- ColumnarExchange (91) - : : +- VeloxAppendBatches (90) + : : +- VeloxResizeBatches (90) : : +- ^ ProjectExecTransformer (88) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (87) : : :- ^ InputIteratorTransformer (77) : : : +- ShuffleQueryStage (75) : : : +- ColumnarExchange (74) - : : : +- VeloxAppendBatches (73) + : : : +- VeloxResizeBatches (73) : : : +- ^ ProjectExecTransformer (71) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (70) : : : :- ^ InputIteratorTransformer (60) : : : : +- ShuffleQueryStage (58) : : : : +- ColumnarExchange (57) - : : : : +- VeloxAppendBatches (56) + : : : : +- VeloxResizeBatches (56) : : : : +- ^ ProjectExecTransformer (54) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (53) : : : : :- ^ InputIteratorTransformer (43) : : : : : +- ShuffleQueryStage (41) : : : : : +- ColumnarExchange (40) - : : : : : +- VeloxAppendBatches (39) + : : : : : +- VeloxResizeBatches (39) : : : : : +- ^ ProjectExecTransformer (37) : : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : : : :- ^ InputIteratorTransformer (26) : : : : : : +- ShuffleQueryStage (24) : : : : : : +- ColumnarExchange (23) - : : : : : : +- VeloxAppendBatches (22) + : : : : : : +- VeloxResizeBatches (22) : : : : : : +- ^ ProjectExecTransformer (20) : : : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : : : :- ^ InputIteratorTransformer (9) : : : : : : : +- ShuffleQueryStage (7) : : : : : : : +- ColumnarExchange (6) - : : : : : : : +- VeloxAppendBatches (5) + : : : : : : : +- VeloxResizeBatches (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ FilterExecTransformer (2) : : : : : : : +- ^ Scan parquet (1) : : : : : : +- ^ InputIteratorTransformer (18) : : : : : : +- ShuffleQueryStage (16) : : : : : : +- ColumnarExchange (15) - : : : : : : +- VeloxAppendBatches (14) + : : : : : : +- VeloxResizeBatches (14) : : : : : : +- ^ ProjectExecTransformer (12) : : : : : : +- ^ FilterExecTransformer (11) : : : : : : +- ^ Scan parquet (10) : : : : : +- ^ InputIteratorTransformer (35) : : : : : +- ShuffleQueryStage (33) : : : : : +- ColumnarExchange (32) - : : : : : +- VeloxAppendBatches (31) + : : : : : +- VeloxResizeBatches (31) : : : : : +- ^ ProjectExecTransformer (29) : : : : : +- ^ FilterExecTransformer (28) : : : : : +- ^ Scan parquet (27) : : : : +- ^ InputIteratorTransformer (52) : : : : +- ShuffleQueryStage (50) : : : : +- ColumnarExchange (49) - : : : : +- VeloxAppendBatches (48) + : : : : +- VeloxResizeBatches (48) : : : : +- ^ ProjectExecTransformer (46) : : : : +- ^ FilterExecTransformer (45) : : : : +- ^ Scan parquet (44) : : : +- ^ InputIteratorTransformer (69) : : : +- ShuffleQueryStage (67) : : : +- ColumnarExchange (66) - : : : +- VeloxAppendBatches (65) + : : : +- VeloxResizeBatches (65) : : : +- ^ ProjectExecTransformer (63) : : : +- ^ FilterExecTransformer (62) : : : +- ^ Scan parquet (61) : : +- ^ InputIteratorTransformer (86) : : +- ShuffleQueryStage (84) : : +- ColumnarExchange (83) - : : +- VeloxAppendBatches (82) + : : +- VeloxResizeBatches (82) : : +- ^ ProjectExecTransformer (80) : : +- ^ FilterExecTransformer (79) : : +- ^ Scan parquet (78) : +- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101) : +- ColumnarExchange (100) - : +- VeloxAppendBatches (99) + : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ FilterExecTransformer (96) : +- ^ Scan parquet (95) +- ^ InputIteratorTransformer (120) +- ShuffleQueryStage (118) +- ColumnarExchange (117) - +- VeloxAppendBatches (116) + +- VeloxResizeBatches (116) +- ^ ProjectExecTransformer (114) +- ^ FilterExecTransformer (113) +- ^ Scan parquet (112) @@ -196,9 +196,9 @@ Input [2]: [p_partkey#X, p_type#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -233,9 +233,9 @@ Input [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discoun Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -264,9 +264,9 @@ Input [6]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -301,9 +301,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -332,9 +332,9 @@ Input [6]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppke Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -369,9 +369,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] @@ -400,9 +400,9 @@ Input [7]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orde Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] @@ -437,9 +437,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -468,9 +468,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_order Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] @@ -505,9 +505,9 @@ Input [2]: [n_nationkey#X, n_regionkey#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] @@ -536,9 +536,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nat Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] Arguments: false -(90) VeloxAppendBatches +(90) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] -Arguments: X +Arguments: X, X (91) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] @@ -573,9 +573,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -604,9 +604,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_reg Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] Arguments: false -(107) VeloxAppendBatches +(107) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] -Arguments: X +Arguments: X, X (108) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] @@ -641,9 +641,9 @@ Input [2]: [r_regionkey#X, r_name#X] Input [2]: [hash_partition_key#X, r_regionkey#X] Arguments: false -(116) VeloxAppendBatches +(116) VeloxResizeBatches Input [2]: [hash_partition_key#X, r_regionkey#X] -Arguments: X +Arguments: X, X (117) ColumnarExchange Input [2]: [hash_partition_key#X, r_regionkey#X] @@ -683,9 +683,9 @@ Input [5]: [o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(126) VeloxAppendBatches +(126) VeloxResizeBatches Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (127) ColumnarExchange Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] @@ -716,9 +716,9 @@ Input [3]: [o_year#X, sum(CASE WHEN (nation#X = BRAZIL) THEN volume#X ELSE 0.000 Input [2]: [o_year#X, mkt_share#X] Arguments: false -(134) VeloxAppendBatches +(134) VeloxResizeBatches Input [2]: [o_year#X, mkt_share#X] -Arguments: X +Arguments: X, X (135) ColumnarExchange Input [2]: [o_year#X, mkt_share#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/9.txt index cb207c0800c3a..f07b5ce81c144 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/9.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (155) +- ^ InputIteratorTransformer (103) +- ShuffleQueryStage (101) +- ColumnarExchange (100) - +- VeloxAppendBatches (99) + +- VeloxResizeBatches (99) +- ^ RegularHashAggregateExecTransformer (97) +- ^ InputIteratorTransformer (96) +- ShuffleQueryStage (94) +- ColumnarExchange (93) - +- VeloxAppendBatches (92) + +- VeloxResizeBatches (92) +- ^ ProjectExecTransformer (90) +- ^ FlushableHashAggregateExecTransformer (89) +- ^ ProjectExecTransformer (88) @@ -19,66 +19,66 @@ AdaptiveSparkPlan (155) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84) +- ColumnarExchange (83) - +- VeloxAppendBatches (82) + +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) +- ^ Scan parquet (78) @@ -152,9 +152,9 @@ Input [2]: [p_partkey#X, p_name#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -189,9 +189,9 @@ Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -220,9 +220,9 @@ Input [7]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -257,9 +257,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -288,9 +288,9 @@ Input [8]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -325,9 +325,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] @@ -356,9 +356,9 @@ Input [10]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpri Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] @@ -393,9 +393,9 @@ Input [2]: [o_orderkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] @@ -424,9 +424,9 @@ Input [8]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_natio Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] @@ -461,9 +461,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -503,9 +503,9 @@ Input [4]: [nation#X, o_year#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] Arguments: false -(92) VeloxAppendBatches +(92) VeloxResizeBatches Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (93) ColumnarExchange Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] @@ -532,9 +532,9 @@ Results [3]: [nation#X, o_year#X, sum(amount#X)#X AS sum_profit#X] Input [3]: [nation#X, o_year#X, sum_profit#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [3]: [nation#X, o_year#X, sum_profit#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [3]: [nation#X, o_year#X, sum_profit#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/1.txt index 22dd5100c4fb3..12c19c45e38d1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/1.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (30) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ RegularHashAggregateExecTransformer (12) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9), Statistics(X) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ ProjectExecTransformer (5) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) @@ -58,9 +58,9 @@ Input [17]: [l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] @@ -87,9 +87,9 @@ Results [10]: [l_returnflag#X, l_linestatus#X, sum(l_quantity#X)#X AS sum_qty#X, Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/10.txt index c295515b8a6cb..d26861c8bcb37 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/10.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (100) +- ^ InputIteratorTransformer (62) +- ShuffleQueryStage (60), Statistics(X) +- ColumnarExchange (59) - +- VeloxAppendBatches (58) + +- VeloxResizeBatches (58) +- ^ ProjectExecTransformer (56) +- ^ FlushableHashAggregateExecTransformer (55) +- ^ ProjectExecTransformer (54) @@ -16,40 +16,40 @@ AdaptiveSparkPlan (100) :- ^ InputIteratorTransformer (43) : +- ShuffleQueryStage (41), Statistics(X) : +- ColumnarExchange (40) - : +- VeloxAppendBatches (39) + : +- VeloxResizeBatches (39) : +- ^ ProjectExecTransformer (37) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) : :- ^ InputIteratorTransformer (26) : : +- ShuffleQueryStage (24), Statistics(X) : : +- ColumnarExchange (23) - : : +- VeloxAppendBatches (22) + : : +- VeloxResizeBatches (22) : : +- ^ ProjectExecTransformer (20) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) : : :- ^ InputIteratorTransformer (9) : : : +- ShuffleQueryStage (7), Statistics(X) : : : +- ColumnarExchange (6) - : : : +- VeloxAppendBatches (5) + : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16), Statistics(X) : : +- ColumnarExchange (15) - : : +- VeloxAppendBatches (14) + : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ FilterExecTransformer (11) : : +- ^ Scan parquet (10) : +- ^ InputIteratorTransformer (35) : +- ShuffleQueryStage (33), Statistics(X) : +- ColumnarExchange (32) - : +- VeloxAppendBatches (31) + : +- VeloxResizeBatches (31) : +- ^ ProjectExecTransformer (29) : +- ^ FilterExecTransformer (28) : +- ^ Scan parquet (27) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50), Statistics(X) +- ColumnarExchange (49) - +- VeloxAppendBatches (48) + +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ FilterExecTransformer (45) +- ^ Scan parquet (44) @@ -107,9 +107,9 @@ Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] @@ -144,9 +144,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -175,9 +175,9 @@ Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] @@ -212,9 +212,9 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] @@ -243,9 +243,9 @@ Input [11]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acc Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] @@ -280,9 +280,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -322,9 +322,9 @@ Input [9]: [c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] Arguments: false -(58) VeloxAppendBatches +(58) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (59) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/11.txt index 20bb486f38418..5dee0fa9091b5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/11.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (82) +- ^ InputIteratorTransformer (53) +- ShuffleQueryStage (51), Statistics(X) +- ColumnarExchange (50) - +- VeloxAppendBatches (49) + +- VeloxResizeBatches (49) +- ^ FilterExecTransformer (47) +- ^ RegularHashAggregateExecTransformer (46) +- ^ InputIteratorTransformer (45) +- ShuffleQueryStage (43), Statistics(X) +- ColumnarExchange (42) - +- VeloxAppendBatches (41) + +- VeloxResizeBatches (41) +- ^ ProjectExecTransformer (39) +- ^ FlushableHashAggregateExecTransformer (38) +- ^ ProjectExecTransformer (37) @@ -20,27 +20,27 @@ AdaptiveSparkPlan (82) :- ^ InputIteratorTransformer (26) : +- ShuffleQueryStage (24), Statistics(X) : +- ColumnarExchange (23) - : +- VeloxAppendBatches (22) + : +- VeloxResizeBatches (22) : +- ^ ProjectExecTransformer (20) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) : +- ^ Scan parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ FilterExecTransformer (28) +- ^ Scan parquet (27) @@ -91,9 +91,9 @@ Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] @@ -128,9 +128,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -159,9 +159,9 @@ Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppke Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] @@ -196,9 +196,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -238,9 +238,9 @@ Input [3]: [ps_partkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] Arguments: false -(41) VeloxAppendBatches +(41) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (42) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] @@ -271,9 +271,9 @@ Arguments: (isnotnull(value#X) AND (cast(value#X as decimal(38,6)) > Subquery su Input [2]: [ps_partkey#X, value#X] Arguments: false -(49) VeloxAppendBatches +(49) VeloxResizeBatches Input [2]: [ps_partkey#X, value#X] -Arguments: X +Arguments: X, X (50) ColumnarExchange Input [2]: [ps_partkey#X, value#X] @@ -435,13 +435,13 @@ AdaptiveSparkPlan (136) :- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101), Statistics(X) : +- ColumnarExchange (100) - : +- VeloxAppendBatches (99) + : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96) : :- ^ InputIteratorTransformer (91) : : +- ShuffleQueryStage (89), Statistics(X) : : +- ColumnarExchange (88) - : : +- VeloxAppendBatches (87) + : : +- VeloxResizeBatches (87) : : +- ^ ProjectExecTransformer (85) : : +- ^ FilterExecTransformer (84) : : +- ^ Scan parquet (83) @@ -494,9 +494,9 @@ Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Input [4]: [hash_partition_key#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: false -(87) VeloxAppendBatches +(87) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (88) ColumnarExchange Input [4]: [hash_partition_key#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] @@ -538,9 +538,9 @@ Input [5]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppkey#X, s_nationk Input [4]: [hash_partition_key#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [4]: [hash_partition_key#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/12.txt index 1b36d274aab44..4545acbc282aa 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/12.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (55) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) @@ -68,9 +68,9 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] @@ -105,9 +105,9 @@ Input [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipm Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] @@ -147,9 +147,9 @@ Input [3]: [l_shipmode#X, sum#X, sum#X] Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] @@ -176,9 +176,9 @@ Results [3]: [l_shipmode#X, sum(CASE WHEN ((o_orderpriority#X = 1-URGENT) OR (o_ Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/13.txt index 83ec9aeda98a2..758ece2d8fce7 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/13.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (58) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35), Statistics(X) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ RegularHashAggregateExecTransformer (31) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ ProjectExecTransformer (22) @@ -22,13 +22,13 @@ AdaptiveSparkPlan (58) :- ^ InputIteratorTransformer (8) : +- ShuffleQueryStage (6), Statistics(X) : +- ColumnarExchange (5) - : +- VeloxAppendBatches (4) + : +- VeloxResizeBatches (4) : +- ^ ProjectExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15), Statistics(X) +- ColumnarExchange (14) - +- VeloxAppendBatches (13) + +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) +- ^ FilterExecTransformer (10) +- ^ Scan parquet (9) @@ -66,9 +66,9 @@ Input [1]: [c_custkey#X] Input [2]: [hash_partition_key#X, c_custkey#X] Arguments: false -(4) VeloxAppendBatches +(4) VeloxResizeBatches Input [2]: [hash_partition_key#X, c_custkey#X] -Arguments: X +Arguments: X, X (5) ColumnarExchange Input [2]: [hash_partition_key#X, c_custkey#X] @@ -103,9 +103,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(13) VeloxAppendBatches +(13) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (14) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -163,9 +163,9 @@ Input [2]: [c_count#X, count#X] Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] @@ -192,9 +192,9 @@ Results [2]: [c_count#X, count(1)#X AS custdist#X] Input [2]: [c_count#X, custdist#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [2]: [c_count#X, custdist#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/14.txt index a1108606b5bb9..ce943bd043e02 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/14.txt @@ -10,14 +10,14 @@ AdaptiveSparkPlan (39) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) @@ -56,9 +56,9 @@ Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] @@ -93,9 +93,9 @@ Input [2]: [p_partkey#X, p_type#X] Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/15.txt index 88730deb3c32b..62b0047bcc06c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/15.txt @@ -5,13 +5,13 @@ AdaptiveSparkPlan (47) +- AQEShuffleRead (29) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (23) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (47) +- ^ InputIteratorTransformer (20) +- ShuffleQueryStage (18), Statistics(X) +- ColumnarExchange (17) - +- VeloxAppendBatches (16) + +- VeloxResizeBatches (16) +- ^ ProjectExecTransformer (14) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) @@ -64,9 +64,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] @@ -112,9 +112,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(16) VeloxAppendBatches +(16) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (17) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] @@ -154,9 +154,9 @@ Input [6]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, supplier_no#X, total_ Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] @@ -267,7 +267,7 @@ AdaptiveSparkPlan (73) +- ^ InputIteratorTransformer (58) +- ShuffleQueryStage (56), Statistics(X) +- ColumnarExchange (55) - +- VeloxAppendBatches (54) + +- VeloxResizeBatches (54) +- ^ ProjectExecTransformer (52) +- ^ FlushableHashAggregateExecTransformer (51) +- ^ ProjectExecTransformer (50) @@ -314,9 +314,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(54) VeloxAppendBatches +(54) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (55) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/16.txt index 535b6940301d8..573dfb3514f58 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/16.txt @@ -6,19 +6,19 @@ AdaptiveSparkPlan (71) +- ^ InputIteratorTransformer (44) +- ShuffleQueryStage (42), Statistics(X) +- ColumnarExchange (41) - +- VeloxAppendBatches (40) + +- VeloxResizeBatches (40) +- ^ RegularHashAggregateExecTransformer (38) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35), Statistics(X) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ ProjectExecTransformer (31) +- ^ FlushableHashAggregateExecTransformer (30) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -26,14 +26,14 @@ AdaptiveSparkPlan (71) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) @@ -82,9 +82,9 @@ Input [2]: [ps_partkey#X, ps_suppkey#X] Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] @@ -119,9 +119,9 @@ Input [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] @@ -161,9 +161,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] @@ -201,9 +201,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, count#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] @@ -230,9 +230,9 @@ Results [4]: [p_brand#X, p_type#X, p_size#X, count(ps_suppkey#X)#X AS supplier_c Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/17.txt index d360b6c948e34..7e1d41ca42510 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/17.txt @@ -12,14 +12,14 @@ AdaptiveSparkPlan (63) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) : +- ^ Scan parquet (10) @@ -29,7 +29,7 @@ AdaptiveSparkPlan (63) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ FilterExecTransformer (22) @@ -78,9 +78,9 @@ Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] @@ -115,9 +115,9 @@ Input [3]: [p_partkey#X, p_brand#X, p_container#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -168,9 +168,9 @@ Input [3]: [l_partkey#X, sum#X, count#X] Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/18.txt index a664adfd3175c..42b2e6f762911 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/18.txt @@ -10,26 +10,26 @@ AdaptiveSparkPlan (110) :- ^ InputIteratorTransformer (46) : +- ShuffleQueryStage (44), Statistics(X) : +- ColumnarExchange (43) - : +- VeloxAppendBatches (42) + : +- VeloxResizeBatches (42) : +- ^ ProjectExecTransformer (40) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (38) : +- ShuffleQueryStage (36), Statistics(X) : +- ColumnarExchange (35) - : +- VeloxAppendBatches (34) + : +- VeloxResizeBatches (34) : +- ^ ProjectExecTransformer (32) : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) : :- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16), Statistics(X) : : +- ColumnarExchange (15) - : : +- VeloxAppendBatches (14) + : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ FilterExecTransformer (11) : : +- ^ Scan parquet (10) @@ -39,7 +39,7 @@ AdaptiveSparkPlan (110) : +- ^ InputIteratorTransformer (27) : +- ShuffleQueryStage (25), Statistics(X) : +- ColumnarExchange (24) - : +- VeloxAppendBatches (23) + : +- VeloxResizeBatches (23) : +- ^ ProjectExecTransformer (21) : +- ^ FlushableHashAggregateExecTransformer (20) : +- ^ Scan parquet (19) @@ -47,7 +47,7 @@ AdaptiveSparkPlan (110) :- ^ InputIteratorTransformer (55) : +- ShuffleQueryStage (53), Statistics(X) : +- ColumnarExchange (52) - : +- VeloxAppendBatches (51) + : +- VeloxResizeBatches (51) : +- ^ ProjectExecTransformer (49) : +- ^ FilterExecTransformer (48) : +- ^ Scan parquet (47) @@ -118,9 +118,9 @@ Input [2]: [c_custkey#X, c_name#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] @@ -155,9 +155,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] @@ -194,9 +194,9 @@ Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] @@ -240,9 +240,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(34) VeloxAppendBatches +(34) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (35) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] @@ -271,9 +271,9 @@ Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_ Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(42) VeloxAppendBatches +(42) VeloxResizeBatches Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (43) ColumnarExchange Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] @@ -308,9 +308,9 @@ Input [2]: [l_orderkey#X, l_quantity#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] Arguments: false -(51) VeloxAppendBatches +(51) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] -Arguments: X +Arguments: X, X (52) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/19.txt index 58e80362020ff..d05ba10a15a65 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/19.txt @@ -9,14 +9,14 @@ AdaptiveSparkPlan (38) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) @@ -55,9 +55,9 @@ Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipin Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -92,9 +92,9 @@ Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/20.txt index c22b822e6f7df..d74e9ecde9991 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/20.txt @@ -5,52 +5,52 @@ AdaptiveSparkPlan (143) +- AQEShuffleRead (92) +- ShuffleQueryStage (91), Statistics(X) +- ColumnarExchange (90) - +- VeloxAppendBatches (89) + +- VeloxResizeBatches (89) +- ^ ProjectExecTransformer (87) +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (86) :- ^ InputIteratorTransformer (76) : +- ShuffleQueryStage (74), Statistics(X) : +- ColumnarExchange (73) - : +- VeloxAppendBatches (72) + : +- VeloxResizeBatches (72) : +- ^ ProjectExecTransformer (70) : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (69) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (68) : +- ShuffleQueryStage (66), Statistics(X) : +- ColumnarExchange (65) - : +- VeloxAppendBatches (64) + : +- VeloxResizeBatches (64) : +- ^ ProjectExecTransformer (62) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (61) : :- ^ InputIteratorTransformer (35) : : +- ShuffleQueryStage (33), Statistics(X) : : +- ColumnarExchange (32) - : : +- VeloxAppendBatches (31) + : : +- VeloxResizeBatches (31) : : +- ^ ProjectExecTransformer (29) : : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (28) : : :- ^ InputIteratorTransformer (18) : : : +- ShuffleQueryStage (16), Statistics(X) : : : +- ColumnarExchange (15) - : : : +- VeloxAppendBatches (14) + : : : +- VeloxResizeBatches (14) : : : +- ^ ProjectExecTransformer (12) : : : +- ^ FilterExecTransformer (11) : : : +- ^ Scan parquet (10) : : +- ^ InputIteratorTransformer (27) : : +- ShuffleQueryStage (25), Statistics(X) : : +- ColumnarExchange (24) - : : +- VeloxAppendBatches (23) + : : +- VeloxResizeBatches (23) : : +- ^ ProjectExecTransformer (21) : : +- ^ FilterExecTransformer (20) : : +- ^ Scan parquet (19) : +- ^ InputIteratorTransformer (60) : +- ShuffleQueryStage (58), Statistics(X) : +- ColumnarExchange (57) - : +- VeloxAppendBatches (56) + : +- VeloxResizeBatches (56) : +- ^ ProjectExecTransformer (54) : +- ^ FilterExecTransformer (53) : +- ^ ProjectExecTransformer (52) @@ -60,7 +60,7 @@ AdaptiveSparkPlan (143) : :- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42), Statistics(X) : : +- ColumnarExchange (41) - : : +- VeloxAppendBatches (40) + : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) : : +- ^ FilterExecTransformer (37) : : +- ^ Scan parquet (36) @@ -70,7 +70,7 @@ AdaptiveSparkPlan (143) +- ^ InputIteratorTransformer (85) +- ShuffleQueryStage (83), Statistics(X) +- ColumnarExchange (82) - +- VeloxAppendBatches (81) + +- VeloxResizeBatches (81) +- ^ ProjectExecTransformer (79) +- ^ FilterExecTransformer (78) +- ^ Scan parquet (77) @@ -145,9 +145,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] @@ -182,9 +182,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] @@ -219,9 +219,9 @@ Input [2]: [p_partkey#X, p_name#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -250,9 +250,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] @@ -287,9 +287,9 @@ Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] @@ -353,9 +353,9 @@ Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] @@ -384,9 +384,9 @@ Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: false -(64) VeloxAppendBatches +(64) VeloxResizeBatches Input [2]: [hash_partition_key#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (65) ColumnarExchange Input [2]: [hash_partition_key#X, ps_suppkey#X] @@ -415,9 +415,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(72) VeloxAppendBatches +(72) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (73) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] @@ -452,9 +452,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(81) VeloxAppendBatches +(81) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (82) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -483,9 +483,9 @@ Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] Input [2]: [s_name#X, s_address#X] Arguments: false -(89) VeloxAppendBatches +(89) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] -Arguments: X +Arguments: X, X (90) ColumnarExchange Input [2]: [s_name#X, s_address#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/21.txt index 8413e2f8f2325..36ecd282aa4ff 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/21.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (137) +- ^ InputIteratorTransformer (88) +- ShuffleQueryStage (86), Statistics(X) +- ColumnarExchange (85) - +- VeloxAppendBatches (84) + +- VeloxResizeBatches (84) +- ^ ProjectExecTransformer (82) +- ^ FlushableHashAggregateExecTransformer (81) +- ^ ProjectExecTransformer (80) @@ -14,60 +14,60 @@ AdaptiveSparkPlan (137) :- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (62) : :- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (45) : : :- ^ InputIteratorTransformer (9) : : : +- ShuffleQueryStage (7), Statistics(X) : : : +- ColumnarExchange (6) - : : : +- VeloxAppendBatches (5) + : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42), Statistics(X) : : +- ColumnarExchange (41) - : : +- VeloxAppendBatches (40) + : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) : : +- ^ ShuffledHashJoinExecTransformer LeftAnti BuildRight (37) : : :- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (27) : : : :- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (26) : : : +- ShuffleQueryStage (24), Statistics(X) : : : +- ColumnarExchange (23) - : : : +- VeloxAppendBatches (22) + : : : +- VeloxResizeBatches (22) : : : +- ^ ProjectExecTransformer (20) : : : +- ^ Scan parquet (19) : : +- ^ InputIteratorTransformer (36) : : +- ShuffleQueryStage (34), Statistics(X) : : +- ColumnarExchange (33) - : : +- VeloxAppendBatches (32) + : : +- VeloxResizeBatches (32) : : +- ^ ProjectExecTransformer (30) : : +- ^ FilterExecTransformer (29) : : +- ^ Scan parquet (28) : +- ^ InputIteratorTransformer (61) : +- ShuffleQueryStage (59), Statistics(X) : +- ColumnarExchange (58) - : +- VeloxAppendBatches (57) + : +- VeloxResizeBatches (57) : +- ^ ProjectExecTransformer (55) : +- ^ FilterExecTransformer (54) : +- ^ Scan parquet (53) +- ^ InputIteratorTransformer (78) +- ShuffleQueryStage (76), Statistics(X) +- ColumnarExchange (75) - +- VeloxAppendBatches (74) + +- VeloxResizeBatches (74) +- ^ ProjectExecTransformer (72) +- ^ FilterExecTransformer (71) +- ^ Scan parquet (70) @@ -138,9 +138,9 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] @@ -175,9 +175,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -207,9 +207,9 @@ Input [2]: [l_orderkey#X, l_suppkey#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -249,9 +249,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(32) VeloxAppendBatches +(32) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (33) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -280,9 +280,9 @@ Input [2]: [l_orderkey#X, l_suppkey#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -311,9 +311,9 @@ Input [5]: [s_suppkey#X, s_name#X, s_nationkey#X, l_orderkey#X, l_suppkey#X] Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] @@ -348,9 +348,9 @@ Input [2]: [o_orderkey#X, o_orderstatus#X] Input [2]: [hash_partition_key#X, o_orderkey#X] Arguments: false -(57) VeloxAppendBatches +(57) VeloxResizeBatches Input [2]: [hash_partition_key#X, o_orderkey#X] -Arguments: X +Arguments: X, X (58) ColumnarExchange Input [2]: [hash_partition_key#X, o_orderkey#X] @@ -379,9 +379,9 @@ Input [4]: [s_name#X, s_nationkey#X, l_orderkey#X, o_orderkey#X] Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] @@ -416,9 +416,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(74) VeloxAppendBatches +(74) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (75) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -458,9 +458,9 @@ Input [2]: [s_name#X, count#X] Input [3]: [hash_partition_key#X, s_name#X, count#X] Arguments: false -(84) VeloxAppendBatches +(84) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, count#X] -Arguments: X +Arguments: X, X (85) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/22.txt index 214b34066a8f8..50a049ea3ce9b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/22.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (52) +- ^ InputIteratorTransformer (34) +- ShuffleQueryStage (32), Statistics(X) +- ColumnarExchange (31) - +- VeloxAppendBatches (30) + +- VeloxResizeBatches (30) +- ^ RegularHashAggregateExecTransformer (28) +- ^ InputIteratorTransformer (27) +- ShuffleQueryStage (25), Statistics(X) +- ColumnarExchange (24) - +- VeloxAppendBatches (23) + +- VeloxResizeBatches (23) +- ^ ProjectExecTransformer (21) +- ^ FlushableHashAggregateExecTransformer (20) +- ^ ProjectExecTransformer (19) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (52) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15), Statistics(X) +- ColumnarExchange (14) - +- VeloxAppendBatches (13) + +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) +- ^ Scan parquet (10) +- == Initial Plan == @@ -65,9 +65,9 @@ Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] @@ -97,9 +97,9 @@ Input [1]: [o_custkey#X] Input [2]: [hash_partition_key#X, o_custkey#X] Arguments: false -(13) VeloxAppendBatches +(13) VeloxResizeBatches Input [2]: [hash_partition_key#X, o_custkey#X] -Arguments: X +Arguments: X, X (14) ColumnarExchange Input [2]: [hash_partition_key#X, o_custkey#X] @@ -139,9 +139,9 @@ Input [4]: [cntrycode#X, count#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] @@ -168,9 +168,9 @@ Results [3]: [cntrycode#X, count(1)#X AS numcust#X, sum(c_acctbal#X)#X AS totacc Input [3]: [cntrycode#X, numcust#X, totacctbal#X] Arguments: false -(30) VeloxAppendBatches +(30) VeloxResizeBatches Input [3]: [cntrycode#X, numcust#X, totacctbal#X] -Arguments: X +Arguments: X, X (31) ColumnarExchange Input [3]: [cntrycode#X, numcust#X, totacctbal#X] @@ -279,7 +279,7 @@ AdaptiveSparkPlan (72) +- ^ InputIteratorTransformer (62) +- ShuffleQueryStage (60), Statistics(X) +- ColumnarExchange (59) - +- VeloxAppendBatches (58) + +- VeloxResizeBatches (58) +- ^ FlushableHashAggregateExecTransformer (56) +- ^ ProjectExecTransformer (55) +- ^ FilterExecTransformer (54) @@ -319,9 +319,9 @@ Results [2]: [sum#X, count#X] Input [2]: [sum#X, count#X] Arguments: false -(58) VeloxAppendBatches +(58) VeloxResizeBatches Input [2]: [sum#X, count#X] -Arguments: X +Arguments: X, X (59) ColumnarExchange Input [2]: [sum#X, count#X] @@ -396,7 +396,7 @@ AdaptiveSparkPlan (72) +- ^ InputIteratorTransformer (62) +- ShuffleQueryStage (60), Statistics(X) +- ColumnarExchange (59) - +- VeloxAppendBatches (58) + +- VeloxResizeBatches (58) +- ^ FlushableHashAggregateExecTransformer (56) +- ^ ProjectExecTransformer (55) +- ^ FilterExecTransformer (54) diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/3.txt index df17819cafe96..4d701845a38af 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/3.txt @@ -11,27 +11,27 @@ AdaptiveSparkPlan (67) :- ^ InputIteratorTransformer (26) : +- ShuffleQueryStage (24), Statistics(X) : +- ColumnarExchange (23) - : +- VeloxAppendBatches (22) + : +- VeloxResizeBatches (22) : +- ^ ProjectExecTransformer (20) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) : +- ^ Scan parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ FilterExecTransformer (28) +- ^ Scan parquet (27) @@ -80,9 +80,9 @@ Input [2]: [c_custkey#X, c_mktsegment#X] Input [2]: [hash_partition_key#X, c_custkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, c_custkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, c_custkey#X] @@ -117,9 +117,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] @@ -148,9 +148,9 @@ Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriorit Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] @@ -185,9 +185,9 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/4.txt index 85d303df874f2..163bb15a43fa7 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/4.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (56) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (56) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) @@ -69,9 +69,9 @@ Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] @@ -106,9 +106,9 @@ Input [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Input [2]: [hash_partition_key#X, l_orderkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [2]: [hash_partition_key#X, l_orderkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [2]: [hash_partition_key#X, l_orderkey#X] @@ -148,9 +148,9 @@ Input [2]: [o_orderpriority#X, count#X] Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] @@ -177,9 +177,9 @@ Results [2]: [o_orderpriority#X, count(1)#X AS order_count#X] Input [2]: [o_orderpriority#X, order_count#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [2]: [o_orderpriority#X, order_count#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [2]: [o_orderpriority#X, order_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/5.txt index 8978f9563c68f..6aadfc753a9af 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/5.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (156) +- ^ InputIteratorTransformer (103) +- ShuffleQueryStage (101), Statistics(X) +- ColumnarExchange (100) - +- VeloxAppendBatches (99) + +- VeloxResizeBatches (99) +- ^ RegularHashAggregateExecTransformer (97) +- ^ InputIteratorTransformer (96) +- ShuffleQueryStage (94), Statistics(X) +- ColumnarExchange (93) - +- VeloxAppendBatches (92) + +- VeloxResizeBatches (92) +- ^ ProjectExecTransformer (90) +- ^ FlushableHashAggregateExecTransformer (89) +- ^ ProjectExecTransformer (88) @@ -19,66 +19,66 @@ AdaptiveSparkPlan (156) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75), Statistics(X) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58), Statistics(X) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41), Statistics(X) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) - +- VeloxAppendBatches (82) + +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) +- ^ Scan parquet (78) @@ -153,9 +153,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -190,9 +190,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -221,9 +221,9 @@ Input [4]: [c_custkey#X, c_nationkey#X, o_orderkey#X, o_custkey#X] Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] @@ -258,9 +258,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -289,9 +289,9 @@ Input [6]: [c_nationkey#X, o_orderkey#X, l_orderkey#X, l_suppkey#X, l_extendedpr Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -326,9 +326,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -357,9 +357,9 @@ Input [6]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppk Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -394,9 +394,9 @@ Input [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] @@ -425,9 +425,9 @@ Input [6]: [l_extendedprice#X, l_discount#X, s_nationkey#X, n_nationkey#X, n_nam Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] @@ -462,9 +462,9 @@ Input [2]: [r_regionkey#X, r_name#X] Input [2]: [hash_partition_key#X, r_regionkey#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [2]: [hash_partition_key#X, r_regionkey#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [2]: [hash_partition_key#X, r_regionkey#X] @@ -504,9 +504,9 @@ Input [3]: [n_name#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] Arguments: false -(92) VeloxAppendBatches +(92) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (93) ColumnarExchange Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] @@ -533,9 +533,9 @@ Results [2]: [n_name#X, sum(CheckOverflow((promote_precision(cast(l_extendedpric Input [2]: [n_name#X, revenue#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [2]: [n_name#X, revenue#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [2]: [n_name#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/6.txt index 3432579a0de06..864cdf3832e13 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/6.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (20) +- ^ InputIteratorTransformer (10) +- ShuffleQueryStage (8), Statistics(X) +- ColumnarExchange (7) - +- VeloxAppendBatches (6) + +- VeloxResizeBatches (6) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) @@ -46,9 +46,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(6) VeloxAppendBatches +(6) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (7) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/7.txt index 244f650f3a72c..85a7dbf545a47 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/7.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (149) +- ^ InputIteratorTransformer (98) +- ShuffleQueryStage (96), Statistics(X) +- ColumnarExchange (95) - +- VeloxAppendBatches (94) + +- VeloxResizeBatches (94) +- ^ RegularHashAggregateExecTransformer (92) +- ^ InputIteratorTransformer (91) +- ShuffleQueryStage (89), Statistics(X) +- ColumnarExchange (88) - +- VeloxAppendBatches (87) + +- VeloxResizeBatches (87) +- ^ ProjectExecTransformer (85) +- ^ FlushableHashAggregateExecTransformer (84) +- ^ ProjectExecTransformer (83) @@ -19,59 +19,59 @@ AdaptiveSparkPlan (149) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75), Statistics(X) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58), Statistics(X) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41), Statistics(X) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) : +- ^ Scan parquet (61) @@ -147,9 +147,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -184,9 +184,9 @@ Input [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipda Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] @@ -215,9 +215,9 @@ Input [7]: [s_suppkey#X, s_nationkey#X, l_orderkey#X, l_suppkey#X, l_extendedpri Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] @@ -252,9 +252,9 @@ Input [2]: [o_orderkey#X, o_custkey#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -283,9 +283,9 @@ Input [7]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_ship Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] @@ -320,9 +320,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -351,9 +351,9 @@ Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_cust Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] @@ -388,9 +388,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -419,9 +419,9 @@ Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nati Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] @@ -474,9 +474,9 @@ Input [5]: [supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Arguments: false -(87) VeloxAppendBatches +(87) VeloxResizeBatches Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (88) ColumnarExchange Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] @@ -503,9 +503,9 @@ Results [4]: [supp_nation#X, cust_nation#X, l_year#X, sum(volume#X)#X AS revenue Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] Arguments: false -(94) VeloxAppendBatches +(94) VeloxResizeBatches Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] -Arguments: X +Arguments: X, X (95) ColumnarExchange Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/8.txt index 282790ba65070..88597b61d0eeb 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/8.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (207) +- ^ InputIteratorTransformer (138) +- ShuffleQueryStage (136), Statistics(X) +- ColumnarExchange (135) - +- VeloxAppendBatches (134) + +- VeloxResizeBatches (134) +- ^ ProjectExecTransformer (132) +- ^ RegularHashAggregateExecTransformer (131) +- ^ InputIteratorTransformer (130) +- ShuffleQueryStage (128), Statistics(X) +- ColumnarExchange (127) - +- VeloxAppendBatches (126) + +- VeloxResizeBatches (126) +- ^ ProjectExecTransformer (124) +- ^ FlushableHashAggregateExecTransformer (123) +- ^ ProjectExecTransformer (122) @@ -20,92 +20,92 @@ AdaptiveSparkPlan (207) :- ^ InputIteratorTransformer (111) : +- ShuffleQueryStage (109), Statistics(X) : +- ColumnarExchange (108) - : +- VeloxAppendBatches (107) + : +- VeloxResizeBatches (107) : +- ^ ProjectExecTransformer (105) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (104) : :- ^ InputIteratorTransformer (94) : : +- ShuffleQueryStage (92), Statistics(X) : : +- ColumnarExchange (91) - : : +- VeloxAppendBatches (90) + : : +- VeloxResizeBatches (90) : : +- ^ ProjectExecTransformer (88) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (87) : : :- ^ InputIteratorTransformer (77) : : : +- ShuffleQueryStage (75), Statistics(X) : : : +- ColumnarExchange (74) - : : : +- VeloxAppendBatches (73) + : : : +- VeloxResizeBatches (73) : : : +- ^ ProjectExecTransformer (71) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (70) : : : :- ^ InputIteratorTransformer (60) : : : : +- ShuffleQueryStage (58), Statistics(X) : : : : +- ColumnarExchange (57) - : : : : +- VeloxAppendBatches (56) + : : : : +- VeloxResizeBatches (56) : : : : +- ^ ProjectExecTransformer (54) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (53) : : : : :- ^ InputIteratorTransformer (43) : : : : : +- ShuffleQueryStage (41), Statistics(X) : : : : : +- ColumnarExchange (40) - : : : : : +- VeloxAppendBatches (39) + : : : : : +- VeloxResizeBatches (39) : : : : : +- ^ ProjectExecTransformer (37) : : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : : : :- ^ InputIteratorTransformer (26) : : : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : : : +- ColumnarExchange (23) - : : : : : : +- VeloxAppendBatches (22) + : : : : : : +- VeloxResizeBatches (22) : : : : : : +- ^ ProjectExecTransformer (20) : : : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : : : :- ^ InputIteratorTransformer (9) : : : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : : : +- ColumnarExchange (6) - : : : : : : : +- VeloxAppendBatches (5) + : : : : : : : +- VeloxResizeBatches (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ FilterExecTransformer (2) : : : : : : : +- ^ Scan parquet (1) : : : : : : +- ^ InputIteratorTransformer (18) : : : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : : : +- ColumnarExchange (15) - : : : : : : +- VeloxAppendBatches (14) + : : : : : : +- VeloxResizeBatches (14) : : : : : : +- ^ ProjectExecTransformer (12) : : : : : : +- ^ FilterExecTransformer (11) : : : : : : +- ^ Scan parquet (10) : : : : : +- ^ InputIteratorTransformer (35) : : : : : +- ShuffleQueryStage (33), Statistics(X) : : : : : +- ColumnarExchange (32) - : : : : : +- VeloxAppendBatches (31) + : : : : : +- VeloxResizeBatches (31) : : : : : +- ^ ProjectExecTransformer (29) : : : : : +- ^ FilterExecTransformer (28) : : : : : +- ^ Scan parquet (27) : : : : +- ^ InputIteratorTransformer (52) : : : : +- ShuffleQueryStage (50), Statistics(X) : : : : +- ColumnarExchange (49) - : : : : +- VeloxAppendBatches (48) + : : : : +- VeloxResizeBatches (48) : : : : +- ^ ProjectExecTransformer (46) : : : : +- ^ FilterExecTransformer (45) : : : : +- ^ Scan parquet (44) : : : +- ^ InputIteratorTransformer (69) : : : +- ShuffleQueryStage (67), Statistics(X) : : : +- ColumnarExchange (66) - : : : +- VeloxAppendBatches (65) + : : : +- VeloxResizeBatches (65) : : : +- ^ ProjectExecTransformer (63) : : : +- ^ FilterExecTransformer (62) : : : +- ^ Scan parquet (61) : : +- ^ InputIteratorTransformer (86) : : +- ShuffleQueryStage (84), Statistics(X) : : +- ColumnarExchange (83) - : : +- VeloxAppendBatches (82) + : : +- VeloxResizeBatches (82) : : +- ^ ProjectExecTransformer (80) : : +- ^ FilterExecTransformer (79) : : +- ^ Scan parquet (78) : +- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101), Statistics(X) : +- ColumnarExchange (100) - : +- VeloxAppendBatches (99) + : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ FilterExecTransformer (96) : +- ^ Scan parquet (95) +- ^ InputIteratorTransformer (120) +- ShuffleQueryStage (118), Statistics(X) +- ColumnarExchange (117) - +- VeloxAppendBatches (116) + +- VeloxResizeBatches (116) +- ^ ProjectExecTransformer (114) +- ^ FilterExecTransformer (113) +- ^ Scan parquet (112) @@ -196,9 +196,9 @@ Input [2]: [p_partkey#X, p_type#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -233,9 +233,9 @@ Input [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discoun Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -264,9 +264,9 @@ Input [6]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -301,9 +301,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -332,9 +332,9 @@ Input [6]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppke Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -369,9 +369,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] @@ -400,9 +400,9 @@ Input [7]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orde Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] @@ -437,9 +437,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -468,9 +468,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_order Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] @@ -505,9 +505,9 @@ Input [2]: [n_nationkey#X, n_regionkey#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] @@ -536,9 +536,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nat Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] Arguments: false -(90) VeloxAppendBatches +(90) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] -Arguments: X +Arguments: X, X (91) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] @@ -573,9 +573,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -604,9 +604,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_reg Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] Arguments: false -(107) VeloxAppendBatches +(107) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] -Arguments: X +Arguments: X, X (108) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] @@ -641,9 +641,9 @@ Input [2]: [r_regionkey#X, r_name#X] Input [2]: [hash_partition_key#X, r_regionkey#X] Arguments: false -(116) VeloxAppendBatches +(116) VeloxResizeBatches Input [2]: [hash_partition_key#X, r_regionkey#X] -Arguments: X +Arguments: X, X (117) ColumnarExchange Input [2]: [hash_partition_key#X, r_regionkey#X] @@ -683,9 +683,9 @@ Input [5]: [o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(126) VeloxAppendBatches +(126) VeloxResizeBatches Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (127) ColumnarExchange Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] @@ -716,9 +716,9 @@ Input [3]: [o_year#X, sum(CASE WHEN (nation#X = BRAZIL) THEN volume#X ELSE 0.000 Input [2]: [o_year#X, mkt_share#X] Arguments: false -(134) VeloxAppendBatches +(134) VeloxResizeBatches Input [2]: [o_year#X, mkt_share#X] -Arguments: X +Arguments: X, X (135) ColumnarExchange Input [2]: [o_year#X, mkt_share#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/9.txt index 15fbf97a77f31..884d1d3563b76 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/9.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (155) +- ^ InputIteratorTransformer (103) +- ShuffleQueryStage (101), Statistics(X) +- ColumnarExchange (100) - +- VeloxAppendBatches (99) + +- VeloxResizeBatches (99) +- ^ RegularHashAggregateExecTransformer (97) +- ^ InputIteratorTransformer (96) +- ShuffleQueryStage (94), Statistics(X) +- ColumnarExchange (93) - +- VeloxAppendBatches (92) + +- VeloxResizeBatches (92) +- ^ ProjectExecTransformer (90) +- ^ FlushableHashAggregateExecTransformer (89) +- ^ ProjectExecTransformer (88) @@ -19,66 +19,66 @@ AdaptiveSparkPlan (155) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75), Statistics(X) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58), Statistics(X) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41), Statistics(X) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) - +- VeloxAppendBatches (82) + +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) +- ^ Scan parquet (78) @@ -152,9 +152,9 @@ Input [2]: [p_partkey#X, p_name#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -189,9 +189,9 @@ Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -220,9 +220,9 @@ Input [7]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -257,9 +257,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -288,9 +288,9 @@ Input [8]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -325,9 +325,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] @@ -356,9 +356,9 @@ Input [10]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpri Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] @@ -393,9 +393,9 @@ Input [2]: [o_orderkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] @@ -424,9 +424,9 @@ Input [8]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_natio Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] @@ -461,9 +461,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -503,9 +503,9 @@ Input [4]: [nation#X, o_year#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] Arguments: false -(92) VeloxAppendBatches +(92) VeloxResizeBatches Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (93) ColumnarExchange Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] @@ -532,9 +532,9 @@ Results [3]: [nation#X, o_year#X, sum(amount#X)#X AS sum_profit#X] Input [3]: [nation#X, o_year#X, sum_profit#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [3]: [nation#X, o_year#X, sum_profit#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [3]: [nation#X, o_year#X, sum_profit#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/1.txt index 1e53cd90e1b35..0c773785eafce 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/1.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (30) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ RegularHashAggregateExecTransformer (12) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9), Statistics(X) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ ProjectExecTransformer (5) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) @@ -58,9 +58,9 @@ Input [17]: [l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] @@ -87,9 +87,9 @@ Results [10]: [l_returnflag#X, l_linestatus#X, sum(l_quantity#X)#X AS sum_qty#X, Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/10.txt index d7376c740f934..3d4d5db424cd6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/10.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (100) +- ^ InputIteratorTransformer (62) +- ShuffleQueryStage (60), Statistics(X) +- ColumnarExchange (59) - +- VeloxAppendBatches (58) + +- VeloxResizeBatches (58) +- ^ ProjectExecTransformer (56) +- ^ FlushableHashAggregateExecTransformer (55) +- ^ ProjectExecTransformer (54) @@ -16,40 +16,40 @@ AdaptiveSparkPlan (100) :- ^ InputIteratorTransformer (43) : +- ShuffleQueryStage (41), Statistics(X) : +- ColumnarExchange (40) - : +- VeloxAppendBatches (39) + : +- VeloxResizeBatches (39) : +- ^ ProjectExecTransformer (37) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) : :- ^ InputIteratorTransformer (26) : : +- ShuffleQueryStage (24), Statistics(X) : : +- ColumnarExchange (23) - : : +- VeloxAppendBatches (22) + : : +- VeloxResizeBatches (22) : : +- ^ ProjectExecTransformer (20) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) : : :- ^ InputIteratorTransformer (9) : : : +- ShuffleQueryStage (7), Statistics(X) : : : +- ColumnarExchange (6) - : : : +- VeloxAppendBatches (5) + : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16), Statistics(X) : : +- ColumnarExchange (15) - : : +- VeloxAppendBatches (14) + : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ FilterExecTransformer (11) : : +- ^ Scan parquet (10) : +- ^ InputIteratorTransformer (35) : +- ShuffleQueryStage (33), Statistics(X) : +- ColumnarExchange (32) - : +- VeloxAppendBatches (31) + : +- VeloxResizeBatches (31) : +- ^ ProjectExecTransformer (29) : +- ^ FilterExecTransformer (28) : +- ^ Scan parquet (27) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50), Statistics(X) +- ColumnarExchange (49) - +- VeloxAppendBatches (48) + +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ FilterExecTransformer (45) +- ^ Scan parquet (44) @@ -107,9 +107,9 @@ Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] @@ -144,9 +144,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -176,9 +176,9 @@ Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] @@ -213,9 +213,9 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] @@ -245,9 +245,9 @@ Input [11]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acc Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] @@ -282,9 +282,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -325,9 +325,9 @@ Input [9]: [c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] Arguments: false -(58) VeloxAppendBatches +(58) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (59) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/11.txt index c9371ffbf2c3a..8dce41dc2898a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/11.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (82) +- ^ InputIteratorTransformer (53) +- ShuffleQueryStage (51), Statistics(X) +- ColumnarExchange (50) - +- VeloxAppendBatches (49) + +- VeloxResizeBatches (49) +- ^ FilterExecTransformer (47) +- ^ RegularHashAggregateExecTransformer (46) +- ^ InputIteratorTransformer (45) +- ShuffleQueryStage (43), Statistics(X) +- ColumnarExchange (42) - +- VeloxAppendBatches (41) + +- VeloxResizeBatches (41) +- ^ ProjectExecTransformer (39) +- ^ FlushableHashAggregateExecTransformer (38) +- ^ ProjectExecTransformer (37) @@ -20,27 +20,27 @@ AdaptiveSparkPlan (82) :- ^ InputIteratorTransformer (26) : +- ShuffleQueryStage (24), Statistics(X) : +- ColumnarExchange (23) - : +- VeloxAppendBatches (22) + : +- VeloxResizeBatches (22) : +- ^ ProjectExecTransformer (20) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) : +- ^ Scan parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ FilterExecTransformer (28) +- ^ Scan parquet (27) @@ -91,9 +91,9 @@ Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] @@ -128,9 +128,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -160,9 +160,9 @@ Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppke Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] @@ -197,9 +197,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -240,9 +240,9 @@ Input [3]: [ps_partkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] Arguments: false -(41) VeloxAppendBatches +(41) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (42) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] @@ -273,9 +273,9 @@ Arguments: (isnotnull(value#X) AND (cast(value#X as decimal(38,6)) > Subquery su Input [2]: [ps_partkey#X, value#X] Arguments: false -(49) VeloxAppendBatches +(49) VeloxResizeBatches Input [2]: [ps_partkey#X, value#X] -Arguments: X +Arguments: X, X (50) ColumnarExchange Input [2]: [ps_partkey#X, value#X] @@ -439,13 +439,13 @@ AdaptiveSparkPlan (136) :- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101), Statistics(X) : +- ColumnarExchange (100) - : +- VeloxAppendBatches (99) + : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96) : :- ^ InputIteratorTransformer (91) : : +- ShuffleQueryStage (89), Statistics(X) : : +- ColumnarExchange (88) - : : +- VeloxAppendBatches (87) + : : +- VeloxResizeBatches (87) : : +- ^ ProjectExecTransformer (85) : : +- ^ FilterExecTransformer (84) : : +- ^ Scan parquet (83) @@ -498,9 +498,9 @@ Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Input [4]: [hash_partition_key#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: false -(87) VeloxAppendBatches +(87) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (88) ColumnarExchange Input [4]: [hash_partition_key#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] @@ -543,9 +543,9 @@ Input [5]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppkey#X, s_nationk Input [4]: [hash_partition_key#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [4]: [hash_partition_key#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/12.txt index ce033f5468d1b..ffa8c4b4ac028 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/12.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (55) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) @@ -68,9 +68,9 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] @@ -105,9 +105,9 @@ Input [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipm Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] @@ -148,9 +148,9 @@ Input [3]: [l_shipmode#X, sum#X, sum#X] Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] @@ -177,9 +177,9 @@ Results [3]: [l_shipmode#X, sum(CASE WHEN ((o_orderpriority#X = 1-URGENT) OR (o_ Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/13.txt index c71d03b93e127..a2da9210750e1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/13.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (58) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35), Statistics(X) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ RegularHashAggregateExecTransformer (31) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ ProjectExecTransformer (22) @@ -22,13 +22,13 @@ AdaptiveSparkPlan (58) :- ^ InputIteratorTransformer (8) : +- ShuffleQueryStage (6), Statistics(X) : +- ColumnarExchange (5) - : +- VeloxAppendBatches (4) + : +- VeloxResizeBatches (4) : +- ^ ProjectExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15), Statistics(X) +- ColumnarExchange (14) - +- VeloxAppendBatches (13) + +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) +- ^ FilterExecTransformer (10) +- ^ Scan parquet (9) @@ -66,9 +66,9 @@ Input [1]: [c_custkey#X] Input [2]: [hash_partition_key#X, c_custkey#X] Arguments: false -(4) VeloxAppendBatches +(4) VeloxResizeBatches Input [2]: [hash_partition_key#X, c_custkey#X] -Arguments: X +Arguments: X, X (5) ColumnarExchange Input [2]: [hash_partition_key#X, c_custkey#X] @@ -103,9 +103,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(13) VeloxAppendBatches +(13) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (14) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -164,9 +164,9 @@ Input [2]: [c_count#X, count#X] Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] @@ -193,9 +193,9 @@ Results [2]: [c_count#X, count(1)#X AS custdist#X] Input [2]: [c_count#X, custdist#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [2]: [c_count#X, custdist#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/14.txt index 492d3f8b9d073..9994b8328915c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/14.txt @@ -10,14 +10,14 @@ AdaptiveSparkPlan (39) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) @@ -56,9 +56,9 @@ Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] @@ -93,9 +93,9 @@ Input [2]: [p_partkey#X, p_type#X] Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/15.txt index 129e4ad927e99..2c9849a2b1cf7 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/15.txt @@ -5,13 +5,13 @@ AdaptiveSparkPlan (47) +- AQEShuffleRead (29) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (23) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (47) +- ^ InputIteratorTransformer (20) +- ShuffleQueryStage (18), Statistics(X) +- ColumnarExchange (17) - +- VeloxAppendBatches (16) + +- VeloxResizeBatches (16) +- ^ ProjectExecTransformer (14) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) @@ -64,9 +64,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] @@ -112,9 +112,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(16) VeloxAppendBatches +(16) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (17) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] @@ -155,9 +155,9 @@ Input [6]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, supplier_no#X, total_ Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] @@ -269,7 +269,7 @@ AdaptiveSparkPlan (73) +- ^ InputIteratorTransformer (58) +- ShuffleQueryStage (56), Statistics(X) +- ColumnarExchange (55) - +- VeloxAppendBatches (54) + +- VeloxResizeBatches (54) +- ^ ProjectExecTransformer (52) +- ^ FlushableHashAggregateExecTransformer (51) +- ^ ProjectExecTransformer (50) @@ -316,9 +316,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(54) VeloxAppendBatches +(54) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (55) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/16.txt index 45b6041f8b4b0..4d7f8090a3b1c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/16.txt @@ -6,19 +6,19 @@ AdaptiveSparkPlan (71) +- ^ InputIteratorTransformer (44) +- ShuffleQueryStage (42), Statistics(X) +- ColumnarExchange (41) - +- VeloxAppendBatches (40) + +- VeloxResizeBatches (40) +- ^ RegularHashAggregateExecTransformer (38) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35), Statistics(X) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ ProjectExecTransformer (31) +- ^ FlushableHashAggregateExecTransformer (30) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -26,14 +26,14 @@ AdaptiveSparkPlan (71) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) @@ -82,9 +82,9 @@ Input [2]: [ps_partkey#X, ps_suppkey#X] Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] @@ -119,9 +119,9 @@ Input [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] @@ -162,9 +162,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] @@ -202,9 +202,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, count#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] @@ -231,9 +231,9 @@ Results [4]: [p_brand#X, p_type#X, p_size#X, count(ps_suppkey#X)#X AS supplier_c Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/17.txt index b46b3e3f27242..aba53c7c1f324 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/17.txt @@ -12,14 +12,14 @@ AdaptiveSparkPlan (63) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) : +- ^ Scan parquet (10) @@ -29,7 +29,7 @@ AdaptiveSparkPlan (63) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ FilterExecTransformer (22) @@ -78,9 +78,9 @@ Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] @@ -115,9 +115,9 @@ Input [3]: [p_partkey#X, p_brand#X, p_container#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -169,9 +169,9 @@ Input [3]: [l_partkey#X, sum#X, count#X] Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/18.txt index febb48962446b..740918c259fc0 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/18.txt @@ -10,26 +10,26 @@ AdaptiveSparkPlan (110) :- ^ InputIteratorTransformer (46) : +- ShuffleQueryStage (44), Statistics(X) : +- ColumnarExchange (43) - : +- VeloxAppendBatches (42) + : +- VeloxResizeBatches (42) : +- ^ ProjectExecTransformer (40) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (38) : +- ShuffleQueryStage (36), Statistics(X) : +- ColumnarExchange (35) - : +- VeloxAppendBatches (34) + : +- VeloxResizeBatches (34) : +- ^ ProjectExecTransformer (32) : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) : :- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16), Statistics(X) : : +- ColumnarExchange (15) - : : +- VeloxAppendBatches (14) + : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ FilterExecTransformer (11) : : +- ^ Scan parquet (10) @@ -39,7 +39,7 @@ AdaptiveSparkPlan (110) : +- ^ InputIteratorTransformer (27) : +- ShuffleQueryStage (25), Statistics(X) : +- ColumnarExchange (24) - : +- VeloxAppendBatches (23) + : +- VeloxResizeBatches (23) : +- ^ ProjectExecTransformer (21) : +- ^ FlushableHashAggregateExecTransformer (20) : +- ^ Scan parquet (19) @@ -47,7 +47,7 @@ AdaptiveSparkPlan (110) :- ^ InputIteratorTransformer (55) : +- ShuffleQueryStage (53), Statistics(X) : +- ColumnarExchange (52) - : +- VeloxAppendBatches (51) + : +- VeloxResizeBatches (51) : +- ^ ProjectExecTransformer (49) : +- ^ FilterExecTransformer (48) : +- ^ Scan parquet (47) @@ -118,9 +118,9 @@ Input [2]: [c_custkey#X, c_name#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] @@ -155,9 +155,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] @@ -194,9 +194,9 @@ Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] @@ -241,9 +241,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(34) VeloxAppendBatches +(34) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (35) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] @@ -273,9 +273,9 @@ Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_ Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(42) VeloxAppendBatches +(42) VeloxResizeBatches Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (43) ColumnarExchange Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] @@ -310,9 +310,9 @@ Input [2]: [l_orderkey#X, l_quantity#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] Arguments: false -(51) VeloxAppendBatches +(51) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] -Arguments: X +Arguments: X, X (52) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/19.txt index fa78645313e48..1acbbcee646f0 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/19.txt @@ -9,14 +9,14 @@ AdaptiveSparkPlan (38) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) @@ -55,9 +55,9 @@ Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipin Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -92,9 +92,9 @@ Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/20.txt index bb9987fc32c1b..54b461c23e0f4 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/20.txt @@ -5,52 +5,52 @@ AdaptiveSparkPlan (143) +- AQEShuffleRead (92) +- ShuffleQueryStage (91), Statistics(X) +- ColumnarExchange (90) - +- VeloxAppendBatches (89) + +- VeloxResizeBatches (89) +- ^ ProjectExecTransformer (87) +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (86) :- ^ InputIteratorTransformer (76) : +- ShuffleQueryStage (74), Statistics(X) : +- ColumnarExchange (73) - : +- VeloxAppendBatches (72) + : +- VeloxResizeBatches (72) : +- ^ ProjectExecTransformer (70) : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (69) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (68) : +- ShuffleQueryStage (66), Statistics(X) : +- ColumnarExchange (65) - : +- VeloxAppendBatches (64) + : +- VeloxResizeBatches (64) : +- ^ ProjectExecTransformer (62) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (61) : :- ^ InputIteratorTransformer (35) : : +- ShuffleQueryStage (33), Statistics(X) : : +- ColumnarExchange (32) - : : +- VeloxAppendBatches (31) + : : +- VeloxResizeBatches (31) : : +- ^ ProjectExecTransformer (29) : : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (28) : : :- ^ InputIteratorTransformer (18) : : : +- ShuffleQueryStage (16), Statistics(X) : : : +- ColumnarExchange (15) - : : : +- VeloxAppendBatches (14) + : : : +- VeloxResizeBatches (14) : : : +- ^ ProjectExecTransformer (12) : : : +- ^ FilterExecTransformer (11) : : : +- ^ Scan parquet (10) : : +- ^ InputIteratorTransformer (27) : : +- ShuffleQueryStage (25), Statistics(X) : : +- ColumnarExchange (24) - : : +- VeloxAppendBatches (23) + : : +- VeloxResizeBatches (23) : : +- ^ ProjectExecTransformer (21) : : +- ^ FilterExecTransformer (20) : : +- ^ Scan parquet (19) : +- ^ InputIteratorTransformer (60) : +- ShuffleQueryStage (58), Statistics(X) : +- ColumnarExchange (57) - : +- VeloxAppendBatches (56) + : +- VeloxResizeBatches (56) : +- ^ ProjectExecTransformer (54) : +- ^ FilterExecTransformer (53) : +- ^ ProjectExecTransformer (52) @@ -60,7 +60,7 @@ AdaptiveSparkPlan (143) : :- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42), Statistics(X) : : +- ColumnarExchange (41) - : : +- VeloxAppendBatches (40) + : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) : : +- ^ FilterExecTransformer (37) : : +- ^ Scan parquet (36) @@ -70,7 +70,7 @@ AdaptiveSparkPlan (143) +- ^ InputIteratorTransformer (85) +- ShuffleQueryStage (83), Statistics(X) +- ColumnarExchange (82) - +- VeloxAppendBatches (81) + +- VeloxResizeBatches (81) +- ^ ProjectExecTransformer (79) +- ^ FilterExecTransformer (78) +- ^ Scan parquet (77) @@ -145,9 +145,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] @@ -182,9 +182,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] @@ -219,9 +219,9 @@ Input [2]: [p_partkey#X, p_name#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -251,9 +251,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] @@ -288,9 +288,9 @@ Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] @@ -355,9 +355,9 @@ Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] @@ -387,9 +387,9 @@ Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: false -(64) VeloxAppendBatches +(64) VeloxResizeBatches Input [2]: [hash_partition_key#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (65) ColumnarExchange Input [2]: [hash_partition_key#X, ps_suppkey#X] @@ -419,9 +419,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(72) VeloxAppendBatches +(72) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (73) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] @@ -456,9 +456,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(81) VeloxAppendBatches +(81) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (82) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -488,9 +488,9 @@ Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] Input [2]: [s_name#X, s_address#X] Arguments: false -(89) VeloxAppendBatches +(89) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] -Arguments: X +Arguments: X, X (90) ColumnarExchange Input [2]: [s_name#X, s_address#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/21.txt index 5e8c9ad9f92ac..e752cdadeec94 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/21.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (137) +- ^ InputIteratorTransformer (88) +- ShuffleQueryStage (86), Statistics(X) +- ColumnarExchange (85) - +- VeloxAppendBatches (84) + +- VeloxResizeBatches (84) +- ^ ProjectExecTransformer (82) +- ^ FlushableHashAggregateExecTransformer (81) +- ^ ProjectExecTransformer (80) @@ -14,60 +14,60 @@ AdaptiveSparkPlan (137) :- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (62) : :- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (45) : : :- ^ InputIteratorTransformer (9) : : : +- ShuffleQueryStage (7), Statistics(X) : : : +- ColumnarExchange (6) - : : : +- VeloxAppendBatches (5) + : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42), Statistics(X) : : +- ColumnarExchange (41) - : : +- VeloxAppendBatches (40) + : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) : : +- ^ ShuffledHashJoinExecTransformer LeftAnti BuildRight (37) : : :- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (27) : : : :- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (26) : : : +- ShuffleQueryStage (24), Statistics(X) : : : +- ColumnarExchange (23) - : : : +- VeloxAppendBatches (22) + : : : +- VeloxResizeBatches (22) : : : +- ^ ProjectExecTransformer (20) : : : +- ^ Scan parquet (19) : : +- ^ InputIteratorTransformer (36) : : +- ShuffleQueryStage (34), Statistics(X) : : +- ColumnarExchange (33) - : : +- VeloxAppendBatches (32) + : : +- VeloxResizeBatches (32) : : +- ^ ProjectExecTransformer (30) : : +- ^ FilterExecTransformer (29) : : +- ^ Scan parquet (28) : +- ^ InputIteratorTransformer (61) : +- ShuffleQueryStage (59), Statistics(X) : +- ColumnarExchange (58) - : +- VeloxAppendBatches (57) + : +- VeloxResizeBatches (57) : +- ^ ProjectExecTransformer (55) : +- ^ FilterExecTransformer (54) : +- ^ Scan parquet (53) +- ^ InputIteratorTransformer (78) +- ShuffleQueryStage (76), Statistics(X) +- ColumnarExchange (75) - +- VeloxAppendBatches (74) + +- VeloxResizeBatches (74) +- ^ ProjectExecTransformer (72) +- ^ FilterExecTransformer (71) +- ^ Scan parquet (70) @@ -138,9 +138,9 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] @@ -175,9 +175,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -207,9 +207,9 @@ Input [2]: [l_orderkey#X, l_suppkey#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -250,9 +250,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(32) VeloxAppendBatches +(32) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (33) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -282,9 +282,9 @@ Input [2]: [l_orderkey#X, l_suppkey#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -314,9 +314,9 @@ Input [5]: [s_suppkey#X, s_name#X, s_nationkey#X, l_orderkey#X, l_suppkey#X] Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] @@ -351,9 +351,9 @@ Input [2]: [o_orderkey#X, o_orderstatus#X] Input [2]: [hash_partition_key#X, o_orderkey#X] Arguments: false -(57) VeloxAppendBatches +(57) VeloxResizeBatches Input [2]: [hash_partition_key#X, o_orderkey#X] -Arguments: X +Arguments: X, X (58) ColumnarExchange Input [2]: [hash_partition_key#X, o_orderkey#X] @@ -383,9 +383,9 @@ Input [4]: [s_name#X, s_nationkey#X, l_orderkey#X, o_orderkey#X] Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] @@ -420,9 +420,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(74) VeloxAppendBatches +(74) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (75) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -463,9 +463,9 @@ Input [2]: [s_name#X, count#X] Input [3]: [hash_partition_key#X, s_name#X, count#X] Arguments: false -(84) VeloxAppendBatches +(84) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, count#X] -Arguments: X +Arguments: X, X (85) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/22.txt index 50f1c1bdef30b..f4597ae85ae23 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/22.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (52) +- ^ InputIteratorTransformer (34) +- ShuffleQueryStage (32), Statistics(X) +- ColumnarExchange (31) - +- VeloxAppendBatches (30) + +- VeloxResizeBatches (30) +- ^ RegularHashAggregateExecTransformer (28) +- ^ InputIteratorTransformer (27) +- ShuffleQueryStage (25), Statistics(X) +- ColumnarExchange (24) - +- VeloxAppendBatches (23) + +- VeloxResizeBatches (23) +- ^ ProjectExecTransformer (21) +- ^ FlushableHashAggregateExecTransformer (20) +- ^ ProjectExecTransformer (19) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (52) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15), Statistics(X) +- ColumnarExchange (14) - +- VeloxAppendBatches (13) + +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) +- ^ Scan parquet (10) +- == Initial Plan == @@ -65,9 +65,9 @@ Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] @@ -97,9 +97,9 @@ Input [1]: [o_custkey#X] Input [2]: [hash_partition_key#X, o_custkey#X] Arguments: false -(13) VeloxAppendBatches +(13) VeloxResizeBatches Input [2]: [hash_partition_key#X, o_custkey#X] -Arguments: X +Arguments: X, X (14) ColumnarExchange Input [2]: [hash_partition_key#X, o_custkey#X] @@ -140,9 +140,9 @@ Input [4]: [cntrycode#X, count#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] @@ -169,9 +169,9 @@ Results [3]: [cntrycode#X, count(1)#X AS numcust#X, sum(c_acctbal#X)#X AS totacc Input [3]: [cntrycode#X, numcust#X, totacctbal#X] Arguments: false -(30) VeloxAppendBatches +(30) VeloxResizeBatches Input [3]: [cntrycode#X, numcust#X, totacctbal#X] -Arguments: X +Arguments: X, X (31) ColumnarExchange Input [3]: [cntrycode#X, numcust#X, totacctbal#X] @@ -281,7 +281,7 @@ AdaptiveSparkPlan (72) +- ^ InputIteratorTransformer (62) +- ShuffleQueryStage (60), Statistics(X) +- ColumnarExchange (59) - +- VeloxAppendBatches (58) + +- VeloxResizeBatches (58) +- ^ FlushableHashAggregateExecTransformer (56) +- ^ ProjectExecTransformer (55) +- ^ FilterExecTransformer (54) @@ -321,9 +321,9 @@ Results [2]: [sum#X, count#X] Input [2]: [sum#X, count#X] Arguments: false -(58) VeloxAppendBatches +(58) VeloxResizeBatches Input [2]: [sum#X, count#X] -Arguments: X +Arguments: X, X (59) ColumnarExchange Input [2]: [sum#X, count#X] @@ -398,7 +398,7 @@ AdaptiveSparkPlan (72) +- ^ InputIteratorTransformer (62) +- ShuffleQueryStage (60), Statistics(X) +- ColumnarExchange (59) - +- VeloxAppendBatches (58) + +- VeloxResizeBatches (58) +- ^ FlushableHashAggregateExecTransformer (56) +- ^ ProjectExecTransformer (55) +- ^ FilterExecTransformer (54) diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/3.txt index 50ad3b59c3471..6ae748ed5b026 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/3.txt @@ -11,27 +11,27 @@ AdaptiveSparkPlan (67) :- ^ InputIteratorTransformer (26) : +- ShuffleQueryStage (24), Statistics(X) : +- ColumnarExchange (23) - : +- VeloxAppendBatches (22) + : +- VeloxResizeBatches (22) : +- ^ ProjectExecTransformer (20) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) : +- ^ Scan parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ FilterExecTransformer (28) +- ^ Scan parquet (27) @@ -80,9 +80,9 @@ Input [2]: [c_custkey#X, c_mktsegment#X] Input [2]: [hash_partition_key#X, c_custkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, c_custkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, c_custkey#X] @@ -117,9 +117,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] @@ -149,9 +149,9 @@ Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriorit Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] @@ -186,9 +186,9 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/4.txt index 2db46753e9fcf..a51a10913f363 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/4.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (56) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (56) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) @@ -69,9 +69,9 @@ Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] @@ -106,9 +106,9 @@ Input [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Input [2]: [hash_partition_key#X, l_orderkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [2]: [hash_partition_key#X, l_orderkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [2]: [hash_partition_key#X, l_orderkey#X] @@ -149,9 +149,9 @@ Input [2]: [o_orderpriority#X, count#X] Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] @@ -178,9 +178,9 @@ Results [2]: [o_orderpriority#X, count(1)#X AS order_count#X] Input [2]: [o_orderpriority#X, order_count#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [2]: [o_orderpriority#X, order_count#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [2]: [o_orderpriority#X, order_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/5.txt index 07a5c86709f4e..15425399ceb8e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/5.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (156) +- ^ InputIteratorTransformer (103) +- ShuffleQueryStage (101), Statistics(X) +- ColumnarExchange (100) - +- VeloxAppendBatches (99) + +- VeloxResizeBatches (99) +- ^ RegularHashAggregateExecTransformer (97) +- ^ InputIteratorTransformer (96) +- ShuffleQueryStage (94), Statistics(X) +- ColumnarExchange (93) - +- VeloxAppendBatches (92) + +- VeloxResizeBatches (92) +- ^ ProjectExecTransformer (90) +- ^ FlushableHashAggregateExecTransformer (89) +- ^ ProjectExecTransformer (88) @@ -19,66 +19,66 @@ AdaptiveSparkPlan (156) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75), Statistics(X) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58), Statistics(X) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41), Statistics(X) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) - +- VeloxAppendBatches (82) + +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) +- ^ Scan parquet (78) @@ -153,9 +153,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -190,9 +190,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -222,9 +222,9 @@ Input [4]: [c_custkey#X, c_nationkey#X, o_orderkey#X, o_custkey#X] Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] @@ -259,9 +259,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -291,9 +291,9 @@ Input [6]: [c_nationkey#X, o_orderkey#X, l_orderkey#X, l_suppkey#X, l_extendedpr Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -328,9 +328,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -360,9 +360,9 @@ Input [6]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppk Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -397,9 +397,9 @@ Input [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] @@ -429,9 +429,9 @@ Input [6]: [l_extendedprice#X, l_discount#X, s_nationkey#X, n_nationkey#X, n_nam Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] @@ -466,9 +466,9 @@ Input [2]: [r_regionkey#X, r_name#X] Input [2]: [hash_partition_key#X, r_regionkey#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [2]: [hash_partition_key#X, r_regionkey#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [2]: [hash_partition_key#X, r_regionkey#X] @@ -509,9 +509,9 @@ Input [3]: [n_name#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] Arguments: false -(92) VeloxAppendBatches +(92) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (93) ColumnarExchange Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] @@ -538,9 +538,9 @@ Results [2]: [n_name#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X AS reven Input [2]: [n_name#X, revenue#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [2]: [n_name#X, revenue#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [2]: [n_name#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/6.txt index ddc921e22d0f6..9463fa1da9d93 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/6.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (20) +- ^ InputIteratorTransformer (10) +- ShuffleQueryStage (8), Statistics(X) +- ColumnarExchange (7) - +- VeloxAppendBatches (6) + +- VeloxResizeBatches (6) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) @@ -46,9 +46,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(6) VeloxAppendBatches +(6) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (7) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/7.txt index b27398e415d31..2960110d5b70e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/7.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (149) +- ^ InputIteratorTransformer (98) +- ShuffleQueryStage (96), Statistics(X) +- ColumnarExchange (95) - +- VeloxAppendBatches (94) + +- VeloxResizeBatches (94) +- ^ RegularHashAggregateExecTransformer (92) +- ^ InputIteratorTransformer (91) +- ShuffleQueryStage (89), Statistics(X) +- ColumnarExchange (88) - +- VeloxAppendBatches (87) + +- VeloxResizeBatches (87) +- ^ ProjectExecTransformer (85) +- ^ FlushableHashAggregateExecTransformer (84) +- ^ ProjectExecTransformer (83) @@ -19,59 +19,59 @@ AdaptiveSparkPlan (149) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75), Statistics(X) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58), Statistics(X) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41), Statistics(X) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) : +- ^ Scan parquet (61) @@ -147,9 +147,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -184,9 +184,9 @@ Input [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipda Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] @@ -216,9 +216,9 @@ Input [7]: [s_suppkey#X, s_nationkey#X, l_orderkey#X, l_suppkey#X, l_extendedpri Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] @@ -253,9 +253,9 @@ Input [2]: [o_orderkey#X, o_custkey#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -285,9 +285,9 @@ Input [7]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_ship Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] @@ -322,9 +322,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -354,9 +354,9 @@ Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_cust Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] @@ -391,9 +391,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -423,9 +423,9 @@ Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nati Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] @@ -479,9 +479,9 @@ Input [5]: [supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Arguments: false -(87) VeloxAppendBatches +(87) VeloxResizeBatches Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (88) ColumnarExchange Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] @@ -508,9 +508,9 @@ Results [4]: [supp_nation#X, cust_nation#X, l_year#X, sum(volume#X)#X AS revenue Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] Arguments: false -(94) VeloxAppendBatches +(94) VeloxResizeBatches Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] -Arguments: X +Arguments: X, X (95) ColumnarExchange Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/8.txt index fa2a2789f4d37..3cacef9cdd5e1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/8.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (207) +- ^ InputIteratorTransformer (138) +- ShuffleQueryStage (136), Statistics(X) +- ColumnarExchange (135) - +- VeloxAppendBatches (134) + +- VeloxResizeBatches (134) +- ^ ProjectExecTransformer (132) +- ^ RegularHashAggregateExecTransformer (131) +- ^ InputIteratorTransformer (130) +- ShuffleQueryStage (128), Statistics(X) +- ColumnarExchange (127) - +- VeloxAppendBatches (126) + +- VeloxResizeBatches (126) +- ^ ProjectExecTransformer (124) +- ^ FlushableHashAggregateExecTransformer (123) +- ^ ProjectExecTransformer (122) @@ -20,92 +20,92 @@ AdaptiveSparkPlan (207) :- ^ InputIteratorTransformer (111) : +- ShuffleQueryStage (109), Statistics(X) : +- ColumnarExchange (108) - : +- VeloxAppendBatches (107) + : +- VeloxResizeBatches (107) : +- ^ ProjectExecTransformer (105) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (104) : :- ^ InputIteratorTransformer (94) : : +- ShuffleQueryStage (92), Statistics(X) : : +- ColumnarExchange (91) - : : +- VeloxAppendBatches (90) + : : +- VeloxResizeBatches (90) : : +- ^ ProjectExecTransformer (88) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (87) : : :- ^ InputIteratorTransformer (77) : : : +- ShuffleQueryStage (75), Statistics(X) : : : +- ColumnarExchange (74) - : : : +- VeloxAppendBatches (73) + : : : +- VeloxResizeBatches (73) : : : +- ^ ProjectExecTransformer (71) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (70) : : : :- ^ InputIteratorTransformer (60) : : : : +- ShuffleQueryStage (58), Statistics(X) : : : : +- ColumnarExchange (57) - : : : : +- VeloxAppendBatches (56) + : : : : +- VeloxResizeBatches (56) : : : : +- ^ ProjectExecTransformer (54) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (53) : : : : :- ^ InputIteratorTransformer (43) : : : : : +- ShuffleQueryStage (41), Statistics(X) : : : : : +- ColumnarExchange (40) - : : : : : +- VeloxAppendBatches (39) + : : : : : +- VeloxResizeBatches (39) : : : : : +- ^ ProjectExecTransformer (37) : : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : : : :- ^ InputIteratorTransformer (26) : : : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : : : +- ColumnarExchange (23) - : : : : : : +- VeloxAppendBatches (22) + : : : : : : +- VeloxResizeBatches (22) : : : : : : +- ^ ProjectExecTransformer (20) : : : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : : : :- ^ InputIteratorTransformer (9) : : : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : : : +- ColumnarExchange (6) - : : : : : : : +- VeloxAppendBatches (5) + : : : : : : : +- VeloxResizeBatches (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ FilterExecTransformer (2) : : : : : : : +- ^ Scan parquet (1) : : : : : : +- ^ InputIteratorTransformer (18) : : : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : : : +- ColumnarExchange (15) - : : : : : : +- VeloxAppendBatches (14) + : : : : : : +- VeloxResizeBatches (14) : : : : : : +- ^ ProjectExecTransformer (12) : : : : : : +- ^ FilterExecTransformer (11) : : : : : : +- ^ Scan parquet (10) : : : : : +- ^ InputIteratorTransformer (35) : : : : : +- ShuffleQueryStage (33), Statistics(X) : : : : : +- ColumnarExchange (32) - : : : : : +- VeloxAppendBatches (31) + : : : : : +- VeloxResizeBatches (31) : : : : : +- ^ ProjectExecTransformer (29) : : : : : +- ^ FilterExecTransformer (28) : : : : : +- ^ Scan parquet (27) : : : : +- ^ InputIteratorTransformer (52) : : : : +- ShuffleQueryStage (50), Statistics(X) : : : : +- ColumnarExchange (49) - : : : : +- VeloxAppendBatches (48) + : : : : +- VeloxResizeBatches (48) : : : : +- ^ ProjectExecTransformer (46) : : : : +- ^ FilterExecTransformer (45) : : : : +- ^ Scan parquet (44) : : : +- ^ InputIteratorTransformer (69) : : : +- ShuffleQueryStage (67), Statistics(X) : : : +- ColumnarExchange (66) - : : : +- VeloxAppendBatches (65) + : : : +- VeloxResizeBatches (65) : : : +- ^ ProjectExecTransformer (63) : : : +- ^ FilterExecTransformer (62) : : : +- ^ Scan parquet (61) : : +- ^ InputIteratorTransformer (86) : : +- ShuffleQueryStage (84), Statistics(X) : : +- ColumnarExchange (83) - : : +- VeloxAppendBatches (82) + : : +- VeloxResizeBatches (82) : : +- ^ ProjectExecTransformer (80) : : +- ^ FilterExecTransformer (79) : : +- ^ Scan parquet (78) : +- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101), Statistics(X) : +- ColumnarExchange (100) - : +- VeloxAppendBatches (99) + : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ FilterExecTransformer (96) : +- ^ Scan parquet (95) +- ^ InputIteratorTransformer (120) +- ShuffleQueryStage (118), Statistics(X) +- ColumnarExchange (117) - +- VeloxAppendBatches (116) + +- VeloxResizeBatches (116) +- ^ ProjectExecTransformer (114) +- ^ FilterExecTransformer (113) +- ^ Scan parquet (112) @@ -196,9 +196,9 @@ Input [2]: [p_partkey#X, p_type#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -233,9 +233,9 @@ Input [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discoun Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -265,9 +265,9 @@ Input [6]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -302,9 +302,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -334,9 +334,9 @@ Input [6]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppke Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -371,9 +371,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] @@ -403,9 +403,9 @@ Input [7]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orde Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] @@ -440,9 +440,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -472,9 +472,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_order Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] @@ -509,9 +509,9 @@ Input [2]: [n_nationkey#X, n_regionkey#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] @@ -541,9 +541,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nat Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] Arguments: false -(90) VeloxAppendBatches +(90) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] -Arguments: X +Arguments: X, X (91) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] @@ -578,9 +578,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -610,9 +610,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_reg Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] Arguments: false -(107) VeloxAppendBatches +(107) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] -Arguments: X +Arguments: X, X (108) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] @@ -647,9 +647,9 @@ Input [2]: [r_regionkey#X, r_name#X] Input [2]: [hash_partition_key#X, r_regionkey#X] Arguments: false -(116) VeloxAppendBatches +(116) VeloxResizeBatches Input [2]: [hash_partition_key#X, r_regionkey#X] -Arguments: X +Arguments: X, X (117) ColumnarExchange Input [2]: [hash_partition_key#X, r_regionkey#X] @@ -690,9 +690,9 @@ Input [5]: [o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(126) VeloxAppendBatches +(126) VeloxResizeBatches Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (127) ColumnarExchange Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] @@ -723,9 +723,9 @@ Input [3]: [o_year#X, sum(CASE WHEN (nation#X = BRAZIL) THEN volume#X ELSE 0.000 Input [2]: [o_year#X, mkt_share#X] Arguments: false -(134) VeloxAppendBatches +(134) VeloxResizeBatches Input [2]: [o_year#X, mkt_share#X] -Arguments: X +Arguments: X, X (135) ColumnarExchange Input [2]: [o_year#X, mkt_share#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/9.txt index 3000cbae7a6d4..61b6401046abf 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/9.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (155) +- ^ InputIteratorTransformer (103) +- ShuffleQueryStage (101), Statistics(X) +- ColumnarExchange (100) - +- VeloxAppendBatches (99) + +- VeloxResizeBatches (99) +- ^ RegularHashAggregateExecTransformer (97) +- ^ InputIteratorTransformer (96) +- ShuffleQueryStage (94), Statistics(X) +- ColumnarExchange (93) - +- VeloxAppendBatches (92) + +- VeloxResizeBatches (92) +- ^ ProjectExecTransformer (90) +- ^ FlushableHashAggregateExecTransformer (89) +- ^ ProjectExecTransformer (88) @@ -19,66 +19,66 @@ AdaptiveSparkPlan (155) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75), Statistics(X) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58), Statistics(X) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41), Statistics(X) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) - +- VeloxAppendBatches (82) + +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) +- ^ Scan parquet (78) @@ -152,9 +152,9 @@ Input [2]: [p_partkey#X, p_name#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -189,9 +189,9 @@ Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -221,9 +221,9 @@ Input [7]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -258,9 +258,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -290,9 +290,9 @@ Input [8]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -327,9 +327,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] @@ -359,9 +359,9 @@ Input [10]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpri Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] @@ -396,9 +396,9 @@ Input [2]: [o_orderkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] @@ -428,9 +428,9 @@ Input [8]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_natio Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] @@ -465,9 +465,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -508,9 +508,9 @@ Input [4]: [nation#X, o_year#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] Arguments: false -(92) VeloxAppendBatches +(92) VeloxResizeBatches Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (93) ColumnarExchange Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] @@ -537,9 +537,9 @@ Results [3]: [nation#X, o_year#X, sum(amount#X)#X AS sum_profit#X] Input [3]: [nation#X, o_year#X, sum_profit#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [3]: [nation#X, o_year#X, sum_profit#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [3]: [nation#X, o_year#X, sum_profit#X] diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/TestOperator.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/TestOperator.scala index 86a62a4471a5e..7eb9df7be6cad 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/TestOperator.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/TestOperator.scala @@ -894,16 +894,18 @@ class TestOperator extends VeloxWholeStageTransformerSuite with AdaptiveSparkPla test("combine small batches before shuffle") { val minBatchSize = 15 + val maxBatchSize = 100 withSQLConf( - "spark.gluten.sql.columnar.backend.velox.coalesceBatchesBeforeShuffle" -> "true", + "spark.gluten.sql.columnar.backend.velox.resizeBatches.shuffleInput" -> "true", "spark.gluten.sql.columnar.maxBatchSize" -> "2", - "spark.gluten.sql.columnar.backend.velox.minBatchSizeForShuffle" -> s"$minBatchSize" + "spark.gluten.sql.columnar.backend.velox.resizeBatches.shuffleInput.range" -> + s"$minBatchSize~$maxBatchSize" ) { val df = runQueryAndCompare( "select l_orderkey, sum(l_partkey) as sum from lineitem " + "where l_orderkey < 100 group by l_orderkey") { _ => } checkLengthAndPlan(df, 27) - val ops = collect(df.queryExecution.executedPlan) { case p: VeloxAppendBatchesExec => p } + val ops = collect(df.queryExecution.executedPlan) { case p: VeloxResizeBatchesExec => p } assert(ops.size == 1) val op = ops.head assert(op.minOutputBatchSize == minBatchSize) @@ -915,6 +917,31 @@ class TestOperator extends VeloxWholeStageTransformerSuite with AdaptiveSparkPla } } + test("split small batches before shuffle") { + val minBatchSize = 1 + val maxBatchSize = 4 + withSQLConf( + "spark.gluten.sql.columnar.backend.velox.resizeBatches.shuffleInput" -> "true", + "spark.gluten.sql.columnar.maxBatchSize" -> "100", + "spark.gluten.sql.columnar.backend.velox.resizeBatches.shuffleInput.range" -> + s"$minBatchSize~$maxBatchSize" + ) { + val df = runQueryAndCompare( + "select l_orderkey, sum(l_partkey) as sum from lineitem " + + "where l_orderkey < 100 group by l_orderkey") { _ => } + checkLengthAndPlan(df, 27) + val ops = collect(df.queryExecution.executedPlan) { case p: VeloxResizeBatchesExec => p } + assert(ops.size == 1) + val op = ops.head + assert(op.minOutputBatchSize == minBatchSize) + val metrics = op.metrics + assert(metrics("numInputRows").value == 27) + assert(metrics("numInputBatches").value == 1) + assert(metrics("numOutputRows").value == 27) + assert(metrics("numOutputBatches").value == 7) + } + } + test("test OneRowRelation") { val df = sql("SELECT 1") checkAnswer(df, Row(1)) diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxTPCHSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxTPCHSuite.scala index 792ab407836c4..798cea8cb94f9 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxTPCHSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxTPCHSuite.scala @@ -70,6 +70,7 @@ abstract class VeloxTPCHSuite extends VeloxTPCHTableSupport { // for unexpected blank .replaceAll("Scan parquet ", "Scan parquet") // Spark QueryStageExec will take it's id as argument, replace it with X + .replaceAll("Arguments: [0-9]+, [0-9]+", "Arguments: X, X") .replaceAll("Arguments: [0-9]+", "Arguments: X") // mask PullOutPostProject and PullOutPreProject id .replaceAll("_pre_[0-9]*", "_pre_X") diff --git a/cpp/velox/CMakeLists.txt b/cpp/velox/CMakeLists.txt index 4c37d1c28fbe5..f1aa96277cb85 100644 --- a/cpp/velox/CMakeLists.txt +++ b/cpp/velox/CMakeLists.txt @@ -527,7 +527,7 @@ set(VELOX_SRCS utils/VeloxArrowUtils.cc utils/ConfigExtractor.cc utils/Common.cc - utils/VeloxBatchAppender.cc) + utils/VeloxBatchResizer.cc) if(ENABLE_HDFS) list(APPEND VELOX_SRCS utils/HdfsUtils.cc) diff --git a/cpp/velox/jni/VeloxJniWrapper.cc b/cpp/velox/jni/VeloxJniWrapper.cc index f9387188bc69d..cb49abd7d4668 100644 --- a/cpp/velox/jni/VeloxJniWrapper.cc +++ b/cpp/velox/jni/VeloxJniWrapper.cc @@ -31,7 +31,7 @@ #include "memory/VeloxMemoryManager.h" #include "substrait/SubstraitToVeloxPlanValidator.h" #include "utils/ObjectStore.h" -#include "utils/VeloxBatchAppender.h" +#include "utils/VeloxBatchResizer.h" #include "velox/common/base/BloomFilter.h" #include @@ -244,17 +244,18 @@ JNIEXPORT jbyteArray JNICALL Java_org_apache_gluten_utils_VeloxBloomFilterJniWra JNI_METHOD_END(nullptr) } -JNIEXPORT jlong JNICALL Java_org_apache_gluten_utils_VeloxBatchAppenderJniWrapper_create( // NOLINT +JNIEXPORT jlong JNICALL Java_org_apache_gluten_utils_VeloxBatchResizerJniWrapper_create( // NOLINT JNIEnv* env, jobject wrapper, jint minOutputBatchSize, + jint maxOutputBatchSize, jobject jIter) { JNI_METHOD_START auto ctx = gluten::getRuntime(env, wrapper); auto pool = dynamic_cast(ctx->memoryManager())->getLeafMemoryPool(); auto iter = gluten::makeJniColumnarBatchIterator(env, jIter, ctx, nullptr); auto appender = std::make_shared( - std::make_unique(pool.get(), minOutputBatchSize, std::move(iter))); + std::make_unique(pool.get(), minOutputBatchSize, maxOutputBatchSize, std::move(iter))); return ctx->saveObject(appender); JNI_METHOD_END(gluten::kInvalidObjectHandle) } diff --git a/cpp/velox/utils/VeloxBatchAppender.cc b/cpp/velox/utils/VeloxBatchAppender.cc deleted file mode 100644 index 8fa1ade217e02..0000000000000 --- a/cpp/velox/utils/VeloxBatchAppender.cc +++ /dev/null @@ -1,59 +0,0 @@ -/* - * 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. - */ - -#include "VeloxBatchAppender.h" - -namespace gluten { - -gluten::VeloxBatchAppender::VeloxBatchAppender( - facebook::velox::memory::MemoryPool* pool, - int32_t minOutputBatchSize, - std::unique_ptr in) - : pool_(pool), minOutputBatchSize_(minOutputBatchSize), in_(std::move(in)) {} - -std::shared_ptr VeloxBatchAppender::next() { - auto cb = in_->next(); - if (cb == nullptr) { - // Input iterator was drained. - return nullptr; - } - if (cb->numRows() >= minOutputBatchSize_) { - // Fast flush path. - return cb; - } - - auto vb = VeloxColumnarBatch::from(pool_, cb); - auto rv = vb->getRowVector(); - auto buffer = facebook::velox::RowVector::createEmpty(rv->type(), pool_); - buffer->append(rv.get()); - - for (auto nextCb = in_->next(); nextCb != nullptr; nextCb = in_->next()) { - auto nextVb = VeloxColumnarBatch::from(pool_, nextCb); - auto nextRv = nextVb->getRowVector(); - buffer->append(nextRv.get()); - if (buffer->size() >= minOutputBatchSize_) { - // Buffer is full. - break; - } - } - return std::make_shared(buffer); -} - -int64_t VeloxBatchAppender::spillFixedSize(int64_t size) { - return in_->spillFixedSize(size); -} -} // namespace gluten diff --git a/cpp/velox/utils/VeloxBatchResizer.cc b/cpp/velox/utils/VeloxBatchResizer.cc new file mode 100644 index 0000000000000..7b51463068c94 --- /dev/null +++ b/cpp/velox/utils/VeloxBatchResizer.cc @@ -0,0 +1,111 @@ +/* + * 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. + */ + +#include "VeloxBatchResizer.h" + +namespace gluten { + +namespace { +class SliceRowVector : public ColumnarBatchIterator { + public: + SliceRowVector(int32_t maxOutputBatchSize, facebook::velox::RowVectorPtr in) + : maxOutputBatchSize_(maxOutputBatchSize), in_(in) { + GLUTEN_CHECK(in->size() > maxOutputBatchSize, "Invalid state"); + } + + std::shared_ptr next() override { + int32_t remainingLength = in_->size() - cursor_; + GLUTEN_CHECK(remainingLength >= 0, "Invalid state"); + if (remainingLength == 0) { + return nullptr; + } + int32_t sliceLength = std::min(maxOutputBatchSize_, remainingLength); + auto out = std::dynamic_pointer_cast(in_->slice(cursor_, sliceLength)); + cursor_ += sliceLength; + GLUTEN_CHECK(out != nullptr, "Invalid state"); + return std::make_shared(out); + } + + private: + int32_t maxOutputBatchSize_; + facebook::velox::RowVectorPtr in_; + int32_t cursor_ = 0; +}; +} // namespace + +gluten::VeloxBatchResizer::VeloxBatchResizer( + facebook::velox::memory::MemoryPool* pool, + int32_t minOutputBatchSize, + int32_t maxOutputBatchSize, + std::unique_ptr in) + : pool_(pool), + minOutputBatchSize_(minOutputBatchSize), + maxOutputBatchSize_(maxOutputBatchSize), + in_(std::move(in)) {} + +std::shared_ptr VeloxBatchResizer::next() { + if (next_) { + auto next = next_->next(); + if (next != nullptr) { + return next; + } + // Cached output was drained. Continue reading data from input iterator. + next_ = nullptr; + } + + auto cb = in_->next(); + if (cb == nullptr) { + // Input iterator was drained. + return nullptr; + } + + if (cb->numRows() < minOutputBatchSize_) { + auto vb = VeloxColumnarBatch::from(pool_, cb); + auto rv = vb->getRowVector(); + auto buffer = facebook::velox::RowVector::createEmpty(rv->type(), pool_); + buffer->append(rv.get()); + + for (auto nextCb = in_->next(); nextCb != nullptr; nextCb = in_->next()) { + auto nextVb = VeloxColumnarBatch::from(pool_, nextCb); + auto nextRv = nextVb->getRowVector(); + buffer->append(nextRv.get()); + if (buffer->size() >= minOutputBatchSize_) { + // Buffer is full. + break; + } + } + return std::make_shared(buffer); + } + + if (cb->numRows() > maxOutputBatchSize_) { + auto vb = VeloxColumnarBatch::from(pool_, cb); + auto rv = vb->getRowVector(); + GLUTEN_CHECK(next_ == nullptr, "Invalid state"); + next_ = std::make_unique(maxOutputBatchSize_, rv); + auto next = next_->next(); + GLUTEN_CHECK(next != nullptr, "Invalid state"); + return next; + } + + // Fast flush path. + return cb; +} + +int64_t VeloxBatchResizer::spillFixedSize(int64_t size) { + return in_->spillFixedSize(size); +} +} // namespace gluten diff --git a/cpp/velox/utils/VeloxBatchAppender.h b/cpp/velox/utils/VeloxBatchResizer.h similarity index 87% rename from cpp/velox/utils/VeloxBatchAppender.h rename to cpp/velox/utils/VeloxBatchResizer.h index 3698381d0add5..09cdf4d51170f 100644 --- a/cpp/velox/utils/VeloxBatchAppender.h +++ b/cpp/velox/utils/VeloxBatchResizer.h @@ -22,11 +22,12 @@ #include "velox/vector/ComplexVector.h" namespace gluten { -class VeloxBatchAppender : public ColumnarBatchIterator { +class VeloxBatchResizer : public ColumnarBatchIterator { public: - VeloxBatchAppender( + VeloxBatchResizer( facebook::velox::memory::MemoryPool* pool, int32_t minOutputBatchSize, + int32_t maxOutputBatchSize, std::unique_ptr in); std::shared_ptr next() override; @@ -36,6 +37,9 @@ class VeloxBatchAppender : public ColumnarBatchIterator { private: facebook::velox::memory::MemoryPool* pool_; const int32_t minOutputBatchSize_; + const int32_t maxOutputBatchSize_; std::unique_ptr in_; + + std::unique_ptr next_ = nullptr; }; } // namespace gluten diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/errors/GlutenQueryExecutionErrorsSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/errors/GlutenQueryExecutionErrorsSuite.scala index c25e65cf0b686..8c661f86023a7 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/errors/GlutenQueryExecutionErrorsSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/errors/GlutenQueryExecutionErrorsSuite.scala @@ -27,7 +27,7 @@ class GlutenQueryExecutionErrorsSuite // Disables VeloxAppendBatches in which GeneralOutIterator wraps vanilla Spark's exceptions // with GlutenException. super.sparkConf - .set("spark.gluten.sql.columnar.backend.velox.coalesceBatchesBeforeShuffle", "false") + .set("spark.gluten.sql.columnar.backend.velox.resizeBatches.shuffleInput", "false") } override protected def getResourceParquetFilePath(name: String): String = { diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index eb32a25e6a5b1..e3f6f1d984ed0 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -306,14 +306,32 @@ class GlutenConfig(conf: SQLConf) extends Logging { def veloxBloomFilterMaxNumBits: Long = conf.getConf(COLUMNAR_VELOX_BLOOM_FILTER_MAX_NUM_BITS) - def veloxCoalesceBatchesBeforeShuffle: Boolean = - conf.getConf(COLUMNAR_VELOX_COALESCE_BATCHES_BEFORE_SHUFFLE) + case class ResizeRange(min: Int, max: Int) { + assert(max >= min) + assert(min > 0, "Min batch size should be larger than 0") + assert(max > 0, "Max batch size should be larger than 0") + } + + private object ResizeRange { + def parse(pattern: String): ResizeRange = { + assert(pattern.count(_ == '~') == 1, s"Invalid range pattern for batch resizing: $pattern") + val splits = pattern.split('~') + assert(splits.length == 2) + ResizeRange(splits(0).toInt, splits(1).toInt) + } + } - def veloxMinBatchSizeForShuffle: Int = { - val defaultSize: Int = (0.8 * conf.getConf(COLUMNAR_MAX_BATCH_SIZE)).toInt.max(1) + def veloxResizeBatchesShuffleInput: Boolean = + conf.getConf(COLUMNAR_VELOX_RESIZE_BATCHES_SHUFFLE_INPUT) + + def veloxResizeBatchesShuffleInputRange: ResizeRange = { + val standardSize = conf.getConf(COLUMNAR_MAX_BATCH_SIZE) + val defaultRange: ResizeRange = + ResizeRange((0.25 * standardSize).toInt.max(1), 4 * standardSize) conf - .getConf(COLUMNAR_VELOX_MIN_BATCH_SIZE_FOR_SHUFFLE) - .getOrElse(defaultSize) + .getConf(COLUMNAR_VELOX_RESIZE_BATCHES_SHUFFLE_INPUT_RANGE) + .map(ResizeRange.parse) + .getOrElse(defaultRange) } def chColumnarShuffleSpillThreshold: Long = { @@ -1453,21 +1471,25 @@ object GlutenConfig { .checkValue(_ > 0, "must be a positive number") .createWithDefault(10000) - val COLUMNAR_VELOX_COALESCE_BATCHES_BEFORE_SHUFFLE = - buildConf("spark.gluten.sql.columnar.backend.velox.coalesceBatchesBeforeShuffle") + val COLUMNAR_VELOX_RESIZE_BATCHES_SHUFFLE_INPUT = + buildConf("spark.gluten.sql.columnar.backend.velox.resizeBatches.shuffleInput") .internal() .doc(s"If true, combine small columnar batches together before sending to shuffle. " + s"The default minimum output batch size is equal to 0.8 * $GLUTEN_MAX_BATCH_SIZE_KEY") .booleanConf .createWithDefault(true) - val COLUMNAR_VELOX_MIN_BATCH_SIZE_FOR_SHUFFLE = - buildConf("spark.gluten.sql.columnar.backend.velox.minBatchSizeForShuffle") + val COLUMNAR_VELOX_RESIZE_BATCHES_SHUFFLE_INPUT_RANGE = + buildConf("spark.gluten.sql.columnar.backend.velox.resizeBatches.shuffleInput.range") .internal() - .doc(s"The minimum batch size for shuffle. If the batch size is smaller than this value, " + - s"it will be combined with other batches before sending to shuffle. Only functions when " + - s"${COLUMNAR_VELOX_COALESCE_BATCHES_BEFORE_SHUFFLE.key} is set to true.") - .intConf + .doc( + s"The minimum and maximum batch sizes for shuffle. If the batch size is " + + s"smaller / bigger than minimum / maximum value, it will be combined with other " + + s"batches / split before sending to shuffle. Only functions when " + + s"${COLUMNAR_VELOX_RESIZE_BATCHES_SHUFFLE_INPUT.key} is set to true. " + + s"A valid value for the option is min~max. " + + s"E.g., s.g.s.c.b.v.resizeBatches.shuffleInput.range=100~10000") + .stringConf .createOptional val COLUMNAR_CH_SHUFFLE_SPILL_THRESHOLD = From b7dd9a83ca8956bb8d597b9643b247528fd95cf6 Mon Sep 17 00:00:00 2001 From: Mingliang Zhu Date: Fri, 2 Aug 2024 16:52:37 +0800 Subject: [PATCH 54/61] [VL] Eliminate pre local sort after offload date type range frame window (#6667) --- .../org/apache/gluten/execution/TestOperator.scala | 12 +++++++++--- .../extension/columnar/EliminateLocalSort.scala | 3 +++ .../columnar/enumerated/EnumeratedApplier.scala | 2 +- .../columnar/heuristic/HeuristicApplier.scala | 2 +- 4 files changed, 14 insertions(+), 5 deletions(-) diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/TestOperator.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/TestOperator.scala index 7eb9df7be6cad..5ca5087d9ef4a 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/TestOperator.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/TestOperator.scala @@ -351,14 +351,20 @@ class TestOperator extends VeloxWholeStageTransformerSuite with AdaptiveSparkPla } test("window expression") { - Seq("sort", "streaming").foreach { - windowType => + Seq(("sort", 0), ("streaming", 1)).foreach { + case (windowType, localSortSize) => withSQLConf("spark.gluten.sql.columnar.backend.velox.window.type" -> windowType) { runQueryAndCompare( "select max(l_partkey) over" + " (partition by l_suppkey order by l_commitdate" + " RANGE BETWEEN 1 PRECEDING AND CURRENT ROW) from lineitem ") { - checkSparkOperatorMatch[WindowExecTransformer] + df => + checkSparkOperatorMatch[WindowExecTransformer](df) + assert( + getExecutedPlan(df).collect { + case s: SortExecTransformer if !s.global => s + }.size == localSortSize + ) } runQueryAndCompare( diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/EliminateLocalSort.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/EliminateLocalSort.scala index 6a5c195e5da15..03e7e4eb73cea 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/EliminateLocalSort.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/EliminateLocalSort.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.execution.{ProjectExec, SortExec, SparkPlan, UnaryEx * - Offload SortAggregate to native hash aggregate * - Offload WindowGroupLimit to native TopNRowNumber * - The columnar window type is `sort` + * - Offload Window which has date type range frame */ object EliminateLocalSort extends Rule[SparkPlan] { private def canEliminateLocalSort(p: SparkPlan): Boolean = p match { @@ -37,6 +38,8 @@ object EliminateLocalSort extends Rule[SparkPlan] { case _: ShuffledHashJoinExecTransformerBase => true case _: WindowGroupLimitExecTransformer => true case _: WindowExecTransformer => true + case s: SortExec if s.global == false => true + case s: SortExecTransformer if s.global == false => true case _ => false } diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedApplier.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedApplier.scala index 3df0282f8e642..5cf3961c548ba 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedApplier.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedApplier.scala @@ -102,8 +102,8 @@ class EnumeratedApplier(session: SparkSession) List( (_: SparkSession) => RemoveNativeWriteFilesSortAndProject(), (spark: SparkSession) => RewriteTransformer(spark), - (_: SparkSession) => EliminateLocalSort, (_: SparkSession) => EnsureLocalSortRequirements, + (_: SparkSession) => EliminateLocalSort, (_: SparkSession) => CollapseProjectExecTransformer ) ::: BackendsApiManager.getSparkPlanExecApiInstance.genExtendedColumnarTransformRules() ::: diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicApplier.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicApplier.scala index 738d67f4b77bb..f776a1dcc3cdf 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicApplier.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicApplier.scala @@ -114,8 +114,8 @@ class HeuristicApplier(session: SparkSession) List( (_: SparkSession) => RemoveNativeWriteFilesSortAndProject(), (spark: SparkSession) => RewriteTransformer(spark), - (_: SparkSession) => EliminateLocalSort, (_: SparkSession) => EnsureLocalSortRequirements, + (_: SparkSession) => EliminateLocalSort, (_: SparkSession) => CollapseProjectExecTransformer ) ::: BackendsApiManager.getSparkPlanExecApiInstance.genExtendedColumnarTransformRules() ::: From 90c25fba97847ba05dff15a91b0f5f6ae299a644 Mon Sep 17 00:00:00 2001 From: Chang chen Date: Sat, 3 Aug 2024 20:07:39 +0800 Subject: [PATCH 55/61] [GLUTEN-6701][CH] fix: Performace regression at 20240802 daily build (#6702) This reverts commit e472e5dac31343caf167f74c9e95f89c146fd049. --- cpp-ch/local-engine/Shuffle/PartitionWriter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp b/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp index 58be564213cb0..a2ef0888aeff5 100644 --- a/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp +++ b/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp @@ -324,7 +324,7 @@ PartitionWriter::PartitionWriter(CachedShuffleWriter * shuffle_writer_, LoggerPt partition_block_buffer[partition_id] = std::make_shared(options->split_size); partition_buffer[partition_id] = std::make_shared(); } - settings = MemoryConfig::loadFromContext(SerializedPlanParser::global_context); + settings.loadFromContext(SerializedPlanParser::global_context); } size_t PartitionWriter::bytes() const From cb0ad202442c698fb06611ffebfb113764c21948 Mon Sep 17 00:00:00 2001 From: Kyligence Git Date: Sat, 3 Aug 2024 08:49:04 -0500 Subject: [PATCH 56/61] [GLUTEN-1632][CH]Daily Update Clickhouse Version (20240803) (#6700) * [GLUTEN-1632][CH]Daily Update Clickhouse Version (20240803) * fix build due to https://github.com/ClickHouse/ClickHouse/pull/65751 * Fix build due to https://github.com/apache/incubator-gluten/pull/6558 --------- Co-authored-by: kyligence-git Co-authored-by: Chang Chen --- cpp-ch/clickhouse.version | 4 ++-- .../ObjectStorages/CompactObjectStorageDiskTransaction.cpp | 4 ++-- .../Disks/ObjectStorages/GlutenHDFSObjectStorage.cpp | 2 +- .../Disks/ObjectStorages/GlutenHDFSObjectStorage.h | 2 +- cpp-ch/local-engine/Storages/Mergetree/MetaDataHelper.cpp | 2 +- cpp-ch/local-engine/tests/benchmark_local_engine.cpp | 2 +- 6 files changed, 8 insertions(+), 8 deletions(-) diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version index dd638c97424a1..d73e34fb425b6 100644 --- a/cpp-ch/clickhouse.version +++ b/cpp-ch/clickhouse.version @@ -1,4 +1,4 @@ CH_ORG=Kyligence -CH_BRANCH=rebase_ch/20240801 -CH_COMMIT=768e193bd4d +CH_BRANCH=rebase_ch/20240803 +CH_COMMIT=b3cfc38f967 diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.cpp b/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.cpp index 66c4470101a19..3a4d156876c6e 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.cpp +++ b/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.cpp @@ -60,8 +60,8 @@ void CompactObjectStorageDiskTransaction::commit() std::filesystem::path meta_path = std::filesystem::path(prefix_path) / "meta.bin"; auto object_storage = disk.getObjectStorage(); - auto data_key = object_storage->generateObjectKeyForPath(data_path); - auto meta_key = object_storage->generateObjectKeyForPath(meta_path); + auto data_key = object_storage->generateObjectKeyForPath(data_path, std::nullopt); + auto meta_key = object_storage->generateObjectKeyForPath(meta_path, std::nullopt); disk.createDirectories(prefix_path); auto data_write_buffer = object_storage->writeObject(DB::StoredObject(data_key.serialize(), data_path), DB::WriteMode::Rewrite); diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/GlutenHDFSObjectStorage.cpp b/cpp-ch/local-engine/Disks/ObjectStorages/GlutenHDFSObjectStorage.cpp index cab87d66d8848..38db8a2dfb1e3 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/GlutenHDFSObjectStorage.cpp +++ b/cpp-ch/local-engine/Disks/ObjectStorages/GlutenHDFSObjectStorage.cpp @@ -33,7 +33,7 @@ std::unique_ptr GlutenHDFSObjectStorage::readObject( /// return std::make_unique(hdfs_uri, hdfs_path, config, HDFSObjectStorage::patchSettings(read_settings)); } -DB::ObjectStorageKey local_engine::GlutenHDFSObjectStorage::generateObjectKeyForPath(const std::string & path) const +DB::ObjectStorageKey local_engine::GlutenHDFSObjectStorage::generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const { initializeHDFSFS(); /// what ever data_source_description.description value is, consider that key as relative key diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/GlutenHDFSObjectStorage.h b/cpp-ch/local-engine/Disks/ObjectStorages/GlutenHDFSObjectStorage.h index da37e1d782dbc..82f3003eb3d78 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/GlutenHDFSObjectStorage.h +++ b/cpp-ch/local-engine/Disks/ObjectStorages/GlutenHDFSObjectStorage.h @@ -41,7 +41,7 @@ class GlutenHDFSObjectStorage final : public DB::HDFSObjectStorage const DB::ReadSettings & read_settings = DB::ReadSettings{}, std::optional read_hint = {}, std::optional file_size = {}) const override; - DB::ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override; + DB::ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional & key_prefix) const override; hdfsFS getHDFSFS() const { return hdfs_fs.get(); } }; #endif diff --git a/cpp-ch/local-engine/Storages/Mergetree/MetaDataHelper.cpp b/cpp-ch/local-engine/Storages/Mergetree/MetaDataHelper.cpp index ebc01544fd1f6..bcb1302591a28 100644 --- a/cpp-ch/local-engine/Storages/Mergetree/MetaDataHelper.cpp +++ b/cpp-ch/local-engine/Storages/Mergetree/MetaDataHelper.cpp @@ -105,7 +105,7 @@ void restoreMetaData(CustomStorageMergeTreePtr & storage, const MergeTreeTable & return; else metadata_disk->createDirectories(part_path); - auto key = s3->generateObjectKeyForPath(metadata_file_path.generic_string()); + auto key = s3->generateObjectKeyForPath(metadata_file_path.generic_string(), std::nullopt); StoredObject metadata_object(key.serialize()); auto part_metadata = extractPartMetaData(*s3->readObject(metadata_object)); for (const auto & item : part_metadata) diff --git a/cpp-ch/local-engine/tests/benchmark_local_engine.cpp b/cpp-ch/local-engine/tests/benchmark_local_engine.cpp index 3402d5026284d..fba8ef7327012 100644 --- a/cpp-ch/local-engine/tests/benchmark_local_engine.cpp +++ b/cpp-ch/local-engine/tests/benchmark_local_engine.cpp @@ -35,7 +35,7 @@ #include #include #include -#include + #include #include #include From 6f00a4549654fa85a3d50d3df92ae66dd8407ebf Mon Sep 17 00:00:00 2001 From: LiuNeng <1398775315@qq.com> Date: Sun, 4 Aug 2024 13:50:46 +0800 Subject: [PATCH 57/61] [CH] Support CACHE DATA command for MergeTree table (#6621) [CH] Support CACHE DATA command for MergeTree table --------- Co-authored-by: liuneng1994 Co-authored-by: Zhichao Zhang --- backends-clickhouse/pom.xml | 16 + .../sql/parser/GlutenClickhouseSqlBase.g4 | 232 ++++++++++ .../parser/GlutenClickhouseSqlParser.scala | 61 +++ .../apache/spark/sql/delta/DeltaAdapter.scala | 10 + .../parser/GlutenClickhouseSqlParser.scala | 65 +++ .../apache/spark/sql/delta/DeltaAdapter.scala | 10 + .../parser/GlutenClickhouseSqlParser.scala | 65 +++ .../apache/spark/sql/delta/DeltaAdapter.scala | 11 + .../execution/CHNativeCacheManager.java | 21 +- .../clickhouse/CHSparkPlanExecApi.scala | 7 + .../GlutenClickhouseSqlParserBase.scala | 276 ++++++++++++ .../spark/rpc/GlutenDriverEndpoint.scala | 4 +- .../spark/rpc/GlutenExecutorEndpoint.scala | 17 +- .../apache/spark/rpc/GlutenRpcMessages.scala | 4 + .../spark/sql/delta/DeltaAdapterTrait.scala | 9 + .../commands/GlutenCHCacheDataCommand.scala | 287 +++++++++++++ .../v2/clickhouse/metadata/AddFileTags.scala | 11 +- ...enClickHouseMergeTreeCacheDataSSuite.scala | 401 ++++++++++++++++++ ...lutenClickHouseNativeWriteTableSuite.scala | 2 +- cpp-ch/local-engine/CMakeLists.txt | 1 + cpp-ch/local-engine/Common/CHUtil.cpp | 3 + cpp-ch/local-engine/Common/MergeTreeTool.cpp | 32 ++ cpp-ch/local-engine/Common/MergeTreeTool.h | 1 + .../Parser/MergeTreeRelParser.cpp | 14 +- .../local-engine/Parser/MergeTreeRelParser.h | 2 - .../Storages/Cache/CacheManager.cpp | 143 +++++++ .../Storages/Cache/CacheManager.h | 44 ++ .../Storages/CustomMergeTreeSink.h | 43 -- .../Storages/StorageMergeTreeFactory.cpp | 27 +- .../Storages/StorageMergeTreeFactory.h | 12 +- cpp-ch/local-engine/local_engine_jni.cpp | 16 + .../gluten/backendsapi/SparkPlanExecApi.scala | 4 + .../extension/OthersExtensionOverrides.scala | 3 + .../GlutenSessionExtensionSuite.scala | 8 +- .../GlutenSessionExtensionSuite.scala | 8 +- .../GlutenSessionExtensionSuite.scala | 8 +- .../GlutenSessionExtensionSuite.scala | 8 +- pom.xml | 10 + 38 files changed, 1806 insertions(+), 90 deletions(-) create mode 100644 backends-clickhouse/src/main/antlr4/org/apache/gluten/sql/parser/GlutenClickhouseSqlBase.g4 create mode 100644 backends-clickhouse/src/main/delta-20/org/apache/gluten/parser/GlutenClickhouseSqlParser.scala create mode 100644 backends-clickhouse/src/main/delta-23/org/apache/gluten/parser/GlutenClickhouseSqlParser.scala create mode 100644 backends-clickhouse/src/main/delta-32/org/apache/gluten/parser/GlutenClickhouseSqlParser.scala rename cpp-ch/local-engine/Storages/CustomMergeTreeSink.cpp => backends-clickhouse/src/main/java/org/apache/gluten/execution/CHNativeCacheManager.java (57%) create mode 100644 backends-clickhouse/src/main/scala/org/apache/gluten/parser/GlutenClickhouseSqlParserBase.scala create mode 100644 backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/commands/GlutenCHCacheDataCommand.scala create mode 100644 backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeCacheDataSSuite.scala create mode 100644 cpp-ch/local-engine/Storages/Cache/CacheManager.cpp create mode 100644 cpp-ch/local-engine/Storages/Cache/CacheManager.h delete mode 100644 cpp-ch/local-engine/Storages/CustomMergeTreeSink.h diff --git a/backends-clickhouse/pom.xml b/backends-clickhouse/pom.xml index 5672056b41607..f2ec45a5169a4 100644 --- a/backends-clickhouse/pom.xml +++ b/backends-clickhouse/pom.xml @@ -365,6 +365,22 @@ + + org.antlr + antlr4-maven-plugin + + + + antlr4 + + + + + true + ../backends-clickhouse/src/main/antlr4 + true + + diff --git a/backends-clickhouse/src/main/antlr4/org/apache/gluten/sql/parser/GlutenClickhouseSqlBase.g4 b/backends-clickhouse/src/main/antlr4/org/apache/gluten/sql/parser/GlutenClickhouseSqlBase.g4 new file mode 100644 index 0000000000000..ac4f66a4fabae --- /dev/null +++ b/backends-clickhouse/src/main/antlr4/org/apache/gluten/sql/parser/GlutenClickhouseSqlBase.g4 @@ -0,0 +1,232 @@ +/* + * 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. + */ + +grammar GlutenClickhouseSqlBase; + +@members { + /** + * Verify whether current token is a valid decimal token (which contains dot). + * Returns true if the character that follows the token is not a digit or letter or underscore. + * + * For example: + * For char stream "2.3", "2." is not a valid decimal token, because it is followed by digit '3'. + * For char stream "2.3_", "2.3" is not a valid decimal token, because it is followed by '_'. + * For char stream "2.3W", "2.3" is not a valid decimal token, because it is followed by 'W'. + * For char stream "12.0D 34.E2+0.12 " 12.0D is a valid decimal token because it is folllowed + * by a space. 34.E2 is a valid decimal token because it is followed by symbol '+' + * which is not a digit or letter or underscore. + */ + public boolean isValidDecimal() { + int nextChar = _input.LA(1); + if (nextChar >= 'A' && nextChar <= 'Z' || nextChar >= '0' && nextChar <= '9' || + nextChar == '_') { + return false; + } else { + return true; + } + } +} + +tokens { + DELIMITER +} + +singleStatement + : statement ';'* EOF + ; + +statement + : CACHE META? DATA ASYNC? SELECT selectedColumns=selectedColumnNames + FROM (path=STRING | table=qualifiedName) (AFTER filter=filterClause)? + (CACHEPROPERTIES cacheProps=propertyList)? #cacheData + | .*? #passThrough + ; + +qualifiedName + : identifier (DOT identifier)* + ; + +selectedColumnNames + : ASTERISK + | identifier (COMMA identifier)* + ; + +filterClause + : TIMESTAMP AS OF timestamp=STRING + | datepartition=identifier AS OF datetime=STRING + ; + +propertyList + : LEFT_PAREN property (COMMA property)* RIGHT_PAREN + ; + +property + : key=propertyKey (EQ? value=propertyValue)? + ; + +propertyKey + : identifier (DOT identifier)* + | stringLit + ; + +propertyValue + : INTEGER_VALUE + | DECIMAL_VALUE + | booleanValue + | identifier LEFT_PAREN stringLit COMMA stringLit RIGHT_PAREN + | value=stringLit + ; + +stringLit + : STRING + | DOUBLEQUOTED_STRING + ; + +booleanValue + : TRUE | FALSE + ; + +identifier + : IDENTIFIER #unquotedIdentifier + | quotedIdentifier #quotedIdentifierAlternative + | nonReserved #unquotedIdentifier + ; + +quotedIdentifier + : BACKQUOTED_IDENTIFIER + ; + +// Add keywords here so that people's queries don't break if they have a column name as one of +// these tokens +nonReserved + : CACHE | META | ASYNC | DATA + | SELECT | FOR | AFTER | CACHEPROPERTIES + | TIMESTAMP | AS | OF | DATE_PARTITION + ; + +// Define how the keywords above should appear in a user's SQL statement. +CACHE: 'CACHE'; +META: 'META'; +ASYNC: 'ASYNC'; +DATA: 'DATA'; +SELECT: 'SELECT'; +COMMA: ','; +FOR: 'FOR'; +FROM: 'FROM'; +AFTER: 'AFTER'; +CACHEPROPERTIES: 'CACHEPROPERTIES'; +DOT: '.'; +ASTERISK: '*'; +TIMESTAMP: 'TIMESTAMP'; +AS: 'AS'; +OF: 'OF'; +DATE_PARTITION: 'DATE_PARTITION'; +LEFT_PAREN: '('; +RIGHT_PAREN: ')'; +TRUE: 'TRUE'; +FALSE: 'FALSE'; + +EQ : '=' | '=='; +NSEQ: '<=>'; +NEQ : '<>'; +NEQJ: '!='; +LTE : '<=' | '!>'; +GTE : '>=' | '!<'; +CONCAT_PIPE: '||'; + +STRING + : '\'' ( ~('\''|'\\') | ('\\' .) )* '\'' + | '"' ( ~('"'|'\\') | ('\\' .) )* '"' + ; + +DOUBLEQUOTED_STRING + :'"' ( ~('"'|'\\') | ('\\' .) )* '"' + ; + +BIGINT_LITERAL + : DIGIT+ 'L' + ; + +SMALLINT_LITERAL + : DIGIT+ 'S' + ; + +TINYINT_LITERAL + : DIGIT+ 'Y' + ; + +INTEGER_VALUE + : DIGIT+ + ; + +DECIMAL_VALUE + : DIGIT+ EXPONENT + | DECIMAL_DIGITS EXPONENT? {isValidDecimal()}? + ; + +DOUBLE_LITERAL + : DIGIT+ EXPONENT? 'D' + | DECIMAL_DIGITS EXPONENT? 'D' {isValidDecimal()}? + ; + +BIGDECIMAL_LITERAL + : DIGIT+ EXPONENT? 'BD' + | DECIMAL_DIGITS EXPONENT? 'BD' {isValidDecimal()}? + ; + +IDENTIFIER + : (LETTER | DIGIT | '_')+ + ; + +BACKQUOTED_IDENTIFIER + : '`' ( ~'`' | '``' )* '`' + ; + +fragment DECIMAL_DIGITS + : DIGIT+ '.' DIGIT* + | '.' DIGIT+ + ; + +fragment EXPONENT + : 'E' [+-]? DIGIT+ + ; + +fragment DIGIT + : [0-9] + ; + +fragment LETTER + : [A-Z] + ; + +SIMPLE_COMMENT + : '--' ~[\r\n]* '\r'? '\n'? -> channel(HIDDEN) + ; + +BRACKETED_COMMENT + : '/*' .*? '*/' -> channel(HIDDEN) + ; + +WS : [ \r\n\t]+ -> channel(HIDDEN) + ; + +// Catch-all for anything we can't recognize. +// We use this to be able to ignore and recover all the text +// when splitting statements with DelimiterLexer +UNRECOGNIZED + : . + ; diff --git a/backends-clickhouse/src/main/delta-20/org/apache/gluten/parser/GlutenClickhouseSqlParser.scala b/backends-clickhouse/src/main/delta-20/org/apache/gluten/parser/GlutenClickhouseSqlParser.scala new file mode 100644 index 0000000000000..cc4f0bd9ffcfd --- /dev/null +++ b/backends-clickhouse/src/main/delta-20/org/apache/gluten/parser/GlutenClickhouseSqlParser.scala @@ -0,0 +1,61 @@ +/* + * 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.gluten.parser + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.parser.ParserInterface +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.types.{DataType, StructType} + +class GlutenClickhouseSqlParser(spark: SparkSession, delegate: ParserInterface) + extends GlutenClickhouseSqlParserBase { + + override def parsePlan(sqlText: String): LogicalPlan = + parse(sqlText) { + parser => + astBuilder.visit(parser.singleStatement()) match { + case plan: LogicalPlan => plan + case _ => delegate.parsePlan(sqlText) + } + } + + override def parseExpression(sqlText: String): Expression = { + delegate.parseExpression(sqlText) + } + + override def parseTableIdentifier(sqlText: String): TableIdentifier = { + delegate.parseTableIdentifier(sqlText) + } + + override def parseFunctionIdentifier(sqlText: String): FunctionIdentifier = { + delegate.parseFunctionIdentifier(sqlText) + } + + override def parseMultipartIdentifier(sqlText: String): Seq[String] = { + delegate.parseMultipartIdentifier(sqlText) + } + + override def parseTableSchema(sqlText: String): StructType = { + delegate.parseTableSchema(sqlText) + } + + override def parseDataType(sqlText: String): DataType = { + delegate.parseDataType(sqlText) + } +} diff --git a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/DeltaAdapter.scala b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/DeltaAdapter.scala index b6d4c04844c43..4ffa2e8415a22 100644 --- a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/DeltaAdapter.scala +++ b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/DeltaAdapter.scala @@ -15,7 +15,17 @@ * limitations under the License. */ package org.apache.spark.sql.delta +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.delta.stats.DeltaScan object DeltaAdapter extends DeltaAdapterTrait { override def snapshot(deltaLog: DeltaLog): Snapshot = deltaLog.snapshot + + override def snapshotFilesForScan( + snapshot: Snapshot, + projection: Seq[Attribute], + filters: Seq[Expression], + keepNumRecords: Boolean): DeltaScan = { + snapshot.filesForScan(projection, filters, keepNumRecords) + } } diff --git a/backends-clickhouse/src/main/delta-23/org/apache/gluten/parser/GlutenClickhouseSqlParser.scala b/backends-clickhouse/src/main/delta-23/org/apache/gluten/parser/GlutenClickhouseSqlParser.scala new file mode 100644 index 0000000000000..1f2dfe00767a4 --- /dev/null +++ b/backends-clickhouse/src/main/delta-23/org/apache/gluten/parser/GlutenClickhouseSqlParser.scala @@ -0,0 +1,65 @@ +/* + * 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.gluten.parser + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.parser.ParserInterface +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.types.{DataType, StructType} + +class GlutenClickhouseSqlParser(spark: SparkSession, delegate: ParserInterface) + extends GlutenClickhouseSqlParserBase { + + override def parsePlan(sqlText: String): LogicalPlan = + parse(sqlText) { + parser => + astBuilder.visit(parser.singleStatement()) match { + case plan: LogicalPlan => plan + case _ => delegate.parsePlan(sqlText) + } + } + + override def parseExpression(sqlText: String): Expression = { + delegate.parseExpression(sqlText) + } + + override def parseTableIdentifier(sqlText: String): TableIdentifier = { + delegate.parseTableIdentifier(sqlText) + } + + override def parseFunctionIdentifier(sqlText: String): FunctionIdentifier = { + delegate.parseFunctionIdentifier(sqlText) + } + + override def parseMultipartIdentifier(sqlText: String): Seq[String] = { + delegate.parseMultipartIdentifier(sqlText) + } + + override def parseTableSchema(sqlText: String): StructType = { + delegate.parseTableSchema(sqlText) + } + + override def parseDataType(sqlText: String): DataType = { + delegate.parseDataType(sqlText) + } + + override def parseQuery(sqlText: String): LogicalPlan = { + delegate.parseQuery(sqlText) + } +} diff --git a/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/DeltaAdapter.scala b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/DeltaAdapter.scala index 8a9c5585e888b..58d59aa9dec50 100644 --- a/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/DeltaAdapter.scala +++ b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/DeltaAdapter.scala @@ -15,7 +15,17 @@ * limitations under the License. */ package org.apache.spark.sql.delta +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.delta.stats.DeltaScan object DeltaAdapter extends DeltaAdapterTrait { override def snapshot(deltaLog: DeltaLog): Snapshot = deltaLog.unsafeVolatileSnapshot + + override def snapshotFilesForScan( + snapshot: Snapshot, + projection: Seq[Attribute], + filters: Seq[Expression], + keepNumRecords: Boolean): DeltaScan = { + snapshot.filesForScan(filters, keepNumRecords) + } } diff --git a/backends-clickhouse/src/main/delta-32/org/apache/gluten/parser/GlutenClickhouseSqlParser.scala b/backends-clickhouse/src/main/delta-32/org/apache/gluten/parser/GlutenClickhouseSqlParser.scala new file mode 100644 index 0000000000000..1f2dfe00767a4 --- /dev/null +++ b/backends-clickhouse/src/main/delta-32/org/apache/gluten/parser/GlutenClickhouseSqlParser.scala @@ -0,0 +1,65 @@ +/* + * 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.gluten.parser + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.parser.ParserInterface +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.types.{DataType, StructType} + +class GlutenClickhouseSqlParser(spark: SparkSession, delegate: ParserInterface) + extends GlutenClickhouseSqlParserBase { + + override def parsePlan(sqlText: String): LogicalPlan = + parse(sqlText) { + parser => + astBuilder.visit(parser.singleStatement()) match { + case plan: LogicalPlan => plan + case _ => delegate.parsePlan(sqlText) + } + } + + override def parseExpression(sqlText: String): Expression = { + delegate.parseExpression(sqlText) + } + + override def parseTableIdentifier(sqlText: String): TableIdentifier = { + delegate.parseTableIdentifier(sqlText) + } + + override def parseFunctionIdentifier(sqlText: String): FunctionIdentifier = { + delegate.parseFunctionIdentifier(sqlText) + } + + override def parseMultipartIdentifier(sqlText: String): Seq[String] = { + delegate.parseMultipartIdentifier(sqlText) + } + + override def parseTableSchema(sqlText: String): StructType = { + delegate.parseTableSchema(sqlText) + } + + override def parseDataType(sqlText: String): DataType = { + delegate.parseDataType(sqlText) + } + + override def parseQuery(sqlText: String): LogicalPlan = { + delegate.parseQuery(sqlText) + } +} diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/DeltaAdapter.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/DeltaAdapter.scala index 8a9c5585e888b..f414ab8f28561 100644 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/DeltaAdapter.scala +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/DeltaAdapter.scala @@ -16,6 +16,17 @@ */ package org.apache.spark.sql.delta +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.delta.stats.DeltaScan + object DeltaAdapter extends DeltaAdapterTrait { override def snapshot(deltaLog: DeltaLog): Snapshot = deltaLog.unsafeVolatileSnapshot + + override def snapshotFilesForScan( + snapshot: Snapshot, + projection: Seq[Attribute], + filters: Seq[Expression], + keepNumRecords: Boolean): DeltaScan = { + snapshot.filesForScan(filters, keepNumRecords) + } } diff --git a/cpp-ch/local-engine/Storages/CustomMergeTreeSink.cpp b/backends-clickhouse/src/main/java/org/apache/gluten/execution/CHNativeCacheManager.java similarity index 57% rename from cpp-ch/local-engine/Storages/CustomMergeTreeSink.cpp rename to backends-clickhouse/src/main/java/org/apache/gluten/execution/CHNativeCacheManager.java index 4ec946c94f95e..f5f75dc1dca6d 100644 --- a/cpp-ch/local-engine/Storages/CustomMergeTreeSink.cpp +++ b/backends-clickhouse/src/main/java/org/apache/gluten/execution/CHNativeCacheManager.java @@ -14,17 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#include "CustomMergeTreeSink.h" +package org.apache.gluten.execution; -void local_engine::CustomMergeTreeSink::consume(Chunk chunk) -{ - auto block = metadata_snapshot->getSampleBlock().cloneWithColumns(chunk.detachColumns()); - DB::BlockWithPartition block_with_partition(Block(block), DB::Row{}); - auto part = storage.writer.writeTempPart(block_with_partition, metadata_snapshot, context); - MergeTreeData::Transaction transaction(storage, NO_TRANSACTION_RAW); - { - auto lock = storage.lockParts(); - storage.renameTempPartAndAdd(part.part, transaction, lock); - transaction.commit(&lock); - } +import java.util.Set; + +public class CHNativeCacheManager { + public static void cacheParts(String table, Set columns, boolean async) { + nativeCacheParts(table, String.join(",", columns), async); + } + + private static native void nativeCacheParts(String table, String columns, boolean async); } diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala index 3069c4a3f6505..bba5525edb955 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala @@ -25,6 +25,7 @@ import org.apache.gluten.extension.{CountDistinctWithoutExpand, FallbackBroadcas import org.apache.gluten.extension.columnar.AddFallbackTagRule import org.apache.gluten.extension.columnar.MiscColumnarRules.TransformPreOverrides import org.apache.gluten.extension.columnar.transition.Convention +import org.apache.gluten.parser.GlutenClickhouseSqlParser import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.gluten.substrait.expression.{ExpressionBuilder, ExpressionNode, WindowFunctionNode} import org.apache.gluten.utils.{CHJoinValidateUtil, UnknownJoinStrategy} @@ -40,6 +41,7 @@ import org.apache.spark.sql.catalyst.{CHAggregateFunctionRewriteRule, EqualToRew import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, CollectList, CollectSet} import org.apache.spark.sql.catalyst.optimizer.BuildSide +import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, HashPartitioning, Partitioning, RangePartitioning} @@ -612,6 +614,11 @@ class CHSparkPlanExecApi extends SparkPlanExecApi { override def genExtendedStrategies(): List[SparkSession => Strategy] = List() + override def genInjectExtendedParser() + : List[(SparkSession, ParserInterface) => ParserInterface] = { + List((spark, parserInterface) => new GlutenClickhouseSqlParser(spark, parserInterface)) + } + /** Define backend specfic expression mappings. */ override def extraExpressionMappings: Seq[Sig] = { List( diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/parser/GlutenClickhouseSqlParserBase.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/parser/GlutenClickhouseSqlParserBase.scala new file mode 100644 index 0000000000000..18fc102bec3d1 --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/parser/GlutenClickhouseSqlParserBase.scala @@ -0,0 +1,276 @@ +/* + * 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.gluten.parser + +import org.apache.gluten.sql.parser.{GlutenClickhouseSqlBaseBaseListener, GlutenClickhouseSqlBaseBaseVisitor, GlutenClickhouseSqlBaseLexer, GlutenClickhouseSqlBaseParser} +import org.apache.gluten.sql.parser.GlutenClickhouseSqlBaseParser._ + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.parser.{ParseErrorListener, ParseException, ParserInterface} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.trees.Origin +import org.apache.spark.sql.execution.commands.GlutenCHCacheDataCommand +import org.apache.spark.sql.internal.VariableSubstitution + +import org.antlr.v4.runtime._ +import org.antlr.v4.runtime.atn.PredictionMode +import org.antlr.v4.runtime.misc.{Interval, ParseCancellationException} +import org.antlr.v4.runtime.tree.TerminalNodeImpl + +import java.util.Locale + +import scala.collection.JavaConverters._ + +trait GlutenClickhouseSqlParserBase extends ParserInterface { + + protected val astBuilder = new GlutenClickhouseSqlAstBuilder + protected val substitution = new VariableSubstitution + + protected def parse[T](command: String)(toResult: GlutenClickhouseSqlBaseParser => T): T = { + val lexer = new GlutenClickhouseSqlBaseLexer( + new UpperCaseCharStream(CharStreams.fromString(substitution.substitute(command)))) + lexer.removeErrorListeners() + lexer.addErrorListener(ParseErrorListener) + + val tokenStream = new CommonTokenStream(lexer) + val parser = new GlutenClickhouseSqlBaseParser(tokenStream) + parser.addParseListener(PostProcessor) + parser.removeErrorListeners() + parser.addErrorListener(ParseErrorListener) + + try { + try { + // first, try parsing with potentially faster SLL mode + parser.getInterpreter.setPredictionMode(PredictionMode.SLL) + toResult(parser) + } catch { + case e: ParseCancellationException => + // if we fail, parse with LL mode + tokenStream.seek(0) // rewind input stream + parser.reset() + + // Try Again. + parser.getInterpreter.setPredictionMode(PredictionMode.LL) + toResult(parser) + } + } catch { + case e: ParseException if e.command.isDefined => + throw e + case e: ParseException => + throw e.withCommand(command) + case e: AnalysisException => + val position = Origin(e.line, e.startPosition) + throw new ParseException( + command = Option(command), + message = e.message, + start = position, + stop = position, + errorClass = Some("GLUTEN_CH_PARSING_ANALYSIS_ERROR")) + } + } +} + +class GlutenClickhouseSqlAstBuilder extends GlutenClickhouseSqlBaseBaseVisitor[AnyRef] { + + import org.apache.spark.sql.catalyst.parser.ParserUtils._ + + /** Convert a property list into a key-value map. */ + override def visitPropertyList(ctx: PropertyListContext): Map[String, String] = withOrigin(ctx) { + val properties = ctx.property.asScala.map { + property => + val key = visitPropertyKey(property.key) + val value = visitPropertyValue(property.value) + key -> value + } + // Check for duplicate property names. + checkDuplicateKeys(properties.toSeq, ctx) + properties.toMap + } + + /** + * A property key can either be String or a collection of dot separated elements. This function + * extracts the property key based on whether its a string literal or a property identifier. + */ + override def visitPropertyKey(key: PropertyKeyContext): String = { + if (key.stringLit() != null) { + string(visitStringLit(key.stringLit())) + } else { + key.getText + } + } + + /** + * A property value can be String, Integer, Boolean or Decimal. This function extracts the + * property value based on whether its a string, integer, boolean or decimal literal. + */ + override def visitPropertyValue(value: PropertyValueContext): String = { + if (value == null) { + null + } else if (value.identifier != null) { + value.identifier.getText + } else if (value.value != null) { + string(visitStringLit(value.value)) + } else if (value.booleanValue != null) { + value.getText.toLowerCase(Locale.ROOT) + } else { + value.getText + } + } + + def visitPropertyKeyValues(ctx: PropertyListContext): Map[String, String] = { + val props = visitPropertyList(ctx) + val badKeys = props.collect { case (key, null) => key } + if (badKeys.nonEmpty) { + operationNotAllowed( + s"Values must be specified for key(s): ${badKeys.mkString("[", ",", "]")}", + ctx) + } + props + } + + override def visitStringLit(ctx: StringLitContext): Token = { + if (ctx != null) { + if (ctx.STRING != null) { + ctx.STRING.getSymbol + } else { + ctx.DOUBLEQUOTED_STRING.getSymbol + } + } else { + null + } + } + + override def visitSingleStatement( + ctx: GlutenClickhouseSqlBaseParser.SingleStatementContext): AnyRef = withOrigin(ctx) { + visit(ctx.statement).asInstanceOf[LogicalPlan] + } + + override def visitCacheData(ctx: GlutenClickhouseSqlBaseParser.CacheDataContext): AnyRef = + withOrigin(ctx) { + val onlyMetaCache = ctx.META != null + val asynExecute = ctx.ASYNC != null + val (tsfilter, partitionColumn, partitionValue) = if (ctx.AFTER != null) { + if (ctx.filter.TIMESTAMP != null) { + (Some(string(ctx.filter.timestamp)), None, None) + } else if (ctx.filter.datepartition != null && ctx.filter.datetime != null) { + (None, Some(ctx.filter.datepartition.getText), Some(string(ctx.filter.datetime))) + } else { + throw new ParseException(s"Illegal filter value ${ctx.getText}", ctx) + } + } else { + (None, None, None) + } + val selectedColuman = visitSelectedColumnNames(ctx.selectedColumns) + val tablePropertyOverrides = Option(ctx.cacheProps) + .map(visitPropertyKeyValues) + .getOrElse(Map.empty[String, String]) + + GlutenCHCacheDataCommand( + onlyMetaCache, + asynExecute, + selectedColuman, + Option(ctx.path).map(string), + Option(ctx.table).map(visitTableIdentifier), + tsfilter, + partitionColumn, + partitionValue, + tablePropertyOverrides + ) + } + + override def visitPassThrough(ctx: GlutenClickhouseSqlBaseParser.PassThroughContext): AnyRef = + null + + protected def visitTableIdentifier(ctx: QualifiedNameContext): TableIdentifier = withOrigin(ctx) { + ctx.identifier.asScala.toSeq match { + case Seq(tbl) => TableIdentifier(tbl.getText) + case Seq(db, tbl) => TableIdentifier(tbl.getText, Some(db.getText)) + // TODO: Spark 3.5 supports catalog parameter + // case Seq(catalog, db, tbl) => + // TableIdentifier(tbl.getText, Some(db.getText), Some(catalog.getText)) + case _ => throw new ParseException(s"Illegal table name ${ctx.getText}", ctx) + } + } + + override def visitSelectedColumnNames(ctx: SelectedColumnNamesContext): Option[Seq[String]] = + withOrigin(ctx) { + if (ctx != null) { + if (ctx.ASTERISK != null) { + // It means select all columns + None + } else if (ctx.identifier != null && !(ctx.identifier).isEmpty) { + Some(ctx.identifier.asScala.map(_.getText).toSeq) + } else { + throw new ParseException(s"Illegal selected column.", ctx) + } + } else { + throw new ParseException(s"Illegal selected column.", ctx) + } + } +} + +case object PostProcessor extends GlutenClickhouseSqlBaseBaseListener { + + /** Remove the back ticks from an Identifier. */ + override def exitQuotedIdentifier(ctx: QuotedIdentifierContext): Unit = { + replaceTokenByIdentifier(ctx, 1) { + token => + // Remove the double back ticks in the string. + token.setText(token.getText.replace("``", "`")) + token + } + } + + /** Treat non-reserved keywords as Identifiers. */ + override def exitNonReserved(ctx: NonReservedContext): Unit = { + replaceTokenByIdentifier(ctx, 0)(identity) + } + + private def replaceTokenByIdentifier(ctx: ParserRuleContext, stripMargins: Int)( + f: CommonToken => CommonToken = identity): Unit = { + val parent = ctx.getParent + parent.removeLastChild() + val token = ctx.getChild(0).getPayload.asInstanceOf[Token] + val newToken = new CommonToken( + new org.antlr.v4.runtime.misc.Pair(token.getTokenSource, token.getInputStream), + GlutenClickhouseSqlBaseParser.IDENTIFIER, + token.getChannel, + token.getStartIndex + stripMargins, + token.getStopIndex - stripMargins + ) + parent.addChild(new TerminalNodeImpl(f(newToken))) + } +} + +class UpperCaseCharStream(wrapped: CodePointCharStream) extends CharStream { + override def consume(): Unit = wrapped.consume + override def getSourceName(): String = wrapped.getSourceName + override def index(): Int = wrapped.index + override def mark(): Int = wrapped.mark + override def release(marker: Int): Unit = wrapped.release(marker) + override def seek(where: Int): Unit = wrapped.seek(where) + override def size(): Int = wrapped.size + + override def getText(interval: Interval): String = wrapped.getText(interval) + + override def LA(i: Int): Int = { + val la = wrapped.LA(i) + if (la == 0 || la == IntStream.EOF) la + else Character.toUpperCase(la) + } +} diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/rpc/GlutenDriverEndpoint.scala b/backends-clickhouse/src/main/scala/org/apache/spark/rpc/GlutenDriverEndpoint.scala index 319381f894b85..a061a620d209d 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/rpc/GlutenDriverEndpoint.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/rpc/GlutenDriverEndpoint.scala @@ -103,7 +103,7 @@ object GlutenDriverEndpoint extends Logging with RemovalListener[String, util.Se var glutenDriverEndpointRef: RpcEndpointRef = _ // keep executorRef on memory - private val executorDataMap = new ConcurrentHashMap[String, ExecutorData] + val executorDataMap = new ConcurrentHashMap[String, ExecutorData] // If spark.scheduler.listenerbus.eventqueue.capacity is set too small, // the listener may lose messages. @@ -131,4 +131,4 @@ object GlutenDriverEndpoint extends Logging with RemovalListener[String, util.Se } } -private class ExecutorData(val executorEndpointRef: RpcEndpointRef) {} +class ExecutorData(val executorEndpointRef: RpcEndpointRef) {} diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/rpc/GlutenExecutorEndpoint.scala b/backends-clickhouse/src/main/scala/org/apache/spark/rpc/GlutenExecutorEndpoint.scala index f05933ef78e33..4d90ab6533ba7 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/rpc/GlutenExecutorEndpoint.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/rpc/GlutenExecutorEndpoint.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.rpc -import org.apache.gluten.execution.CHBroadcastBuildSideCache +import org.apache.gluten.execution.{CHBroadcastBuildSideCache, CHNativeCacheManager} import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.internal.{config, Logging} @@ -64,10 +64,25 @@ class GlutenExecutorEndpoint(val executorId: String, val conf: SparkConf) hashIds.forEach( resource_id => CHBroadcastBuildSideCache.invalidateBroadcastHashtable(resource_id)) } + case GlutenMergeTreeCacheLoad(mergeTreeTable, columns) => + CHNativeCacheManager.cacheParts(mergeTreeTable, columns, true) case e => logError(s"Received unexpected message. $e") } + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case GlutenMergeTreeCacheLoad(mergeTreeTable, columns) => + try { + CHNativeCacheManager.cacheParts(mergeTreeTable, columns, false) + context.reply(CacheLoadResult(true)) + } catch { + case _: Exception => + context.reply(CacheLoadResult(false, s"executor: $executorId cache data failed.")) + } + case e => + logError(s"Received unexpected message. $e") + } } object GlutenExecutorEndpoint { var executorEndpoint: GlutenExecutorEndpoint = _ diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/rpc/GlutenRpcMessages.scala b/backends-clickhouse/src/main/scala/org/apache/spark/rpc/GlutenRpcMessages.scala index 43a0b7bd42b23..d675d705f10a2 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/rpc/GlutenRpcMessages.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/rpc/GlutenRpcMessages.scala @@ -35,4 +35,8 @@ object GlutenRpcMessages { case class GlutenCleanExecutionResource(executionId: String, broadcastHashIds: util.Set[String]) extends GlutenRpcMessage + case class GlutenMergeTreeCacheLoad(mergeTreeTable: String, columns: util.Set[String]) + extends GlutenRpcMessage + + case class CacheLoadResult(success: Boolean, reason: String = "") extends GlutenRpcMessage } diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/DeltaAdapterTrait.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/DeltaAdapterTrait.scala index 3ea4af4ae7087..6f3bb37050a4a 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/DeltaAdapterTrait.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/DeltaAdapterTrait.scala @@ -16,7 +16,16 @@ */ package org.apache.spark.sql.delta +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.delta.stats.DeltaScan + trait DeltaAdapterTrait { def snapshot(deltaLog: DeltaLog): Snapshot + + def snapshotFilesForScan( + snapshot: Snapshot, + projection: Seq[Attribute], + filters: Seq[Expression], + keepNumRecords: Boolean): DeltaScan } diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/commands/GlutenCHCacheDataCommand.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/commands/GlutenCHCacheDataCommand.scala new file mode 100644 index 0000000000000..1e6b024063b6d --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/commands/GlutenCHCacheDataCommand.scala @@ -0,0 +1,287 @@ +/* + * 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.spark.sql.execution.commands + +import org.apache.gluten.exception.GlutenException +import org.apache.gluten.expression.ConverterUtils +import org.apache.gluten.substrait.rel.ExtensionTableBuilder + +import org.apache.spark.affinity.CHAffinity +import org.apache.spark.rpc.GlutenDriverEndpoint +import org.apache.spark.rpc.GlutenRpcMessages.{CacheLoadResult, GlutenMergeTreeCacheLoad} +import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, GreaterThanOrEqual, IsNotNull, Literal} +import org.apache.spark.sql.delta._ +import org.apache.spark.sql.execution.command.LeafRunnableCommand +import org.apache.spark.sql.execution.commands.GlutenCHCacheDataCommand.toExecutorId +import org.apache.spark.sql.execution.datasources.v2.clickhouse.metadata.AddMergeTreeParts +import org.apache.spark.sql.types.{BooleanType, StringType} +import org.apache.spark.util.ThreadUtils + +import org.apache.hadoop.fs.Path + +import java.net.URI +import java.util.{ArrayList => JList} + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.Future +import scala.concurrent.duration.Duration + +case class GlutenCHCacheDataCommand( + onlyMetaCache: Boolean, + asynExecute: Boolean, + selectedColuman: Option[Seq[String]], + path: Option[String], + table: Option[TableIdentifier], + tsfilter: Option[String], + partitionColumn: Option[String], + partitionValue: Option[String], + tablePropertyOverrides: Map[String, String] +) extends LeafRunnableCommand { + + override def output: Seq[Attribute] = Seq( + AttributeReference("result", BooleanType, nullable = false)(), + AttributeReference("reason", StringType, nullable = false)()) + + override def run(sparkSession: SparkSession): Seq[Row] = { + val pathToCache = + if (path.nonEmpty) { + new Path(path.get) + } else if (table.nonEmpty) { + DeltaTableIdentifier(sparkSession, table.get) match { + case Some(id) if id.path.nonEmpty => + new Path(id.path.get) + case _ => + new Path(sparkSession.sessionState.catalog.getTableMetadata(table.get).location) + } + } else { + throw DeltaErrors.missingTableIdentifierException("CACHE DATA") + } + + val baseDeltaPath = DeltaTableUtils.findDeltaTableRoot(sparkSession, pathToCache) + if (baseDeltaPath.isDefined) { + if (baseDeltaPath.get != pathToCache) { + throw DeltaErrors.vacuumBasePathMissingException(baseDeltaPath.get) + } + } + + val deltaLog = DeltaLog.forTable(sparkSession, pathToCache) + if (!deltaLog.tableExists) { + throw DeltaErrors.notADeltaTableException( + "CACHE DATA", + DeltaTableIdentifier(path = Some(pathToCache.toString))) + } + + val snapshot = deltaLog.update() + + require( + snapshot.version >= 0, + "No state defined for this table. Is this really " + + "a Delta table? Refusing to garbage collect.") + + val allColumns = snapshot.dataSchema.fieldNames.toSeq + val selectedColumns = if (selectedColuman.nonEmpty) { + selectedColuman.get + .filter(allColumns.contains(_)) + .map(ConverterUtils.normalizeColName) + .toSeq + } else { + allColumns.map(ConverterUtils.normalizeColName) + } + + val selectedAddFiles = if (tsfilter.isDefined) { + val allParts = DeltaAdapter.snapshotFilesForScan(snapshot, Seq.empty, Seq.empty, false) + allParts.files.filter(_.modificationTime >= tsfilter.get.toLong).toSeq + } else if (partitionColumn.isDefined && partitionValue.isDefined) { + val partitionColumns = snapshot.metadata.partitionSchema.fieldNames + require( + partitionColumns.contains(partitionColumn.get), + s"the partition column ${partitionColumn.get} is invalid.") + val partitionColumnField = snapshot.metadata.partitionSchema(partitionColumn.get) + + val partitionColumnAttr = AttributeReference( + ConverterUtils.normalizeColName(partitionColumn.get), + partitionColumnField.dataType, + partitionColumnField.nullable)() + val isNotNullExpr = IsNotNull(partitionColumnAttr) + val greaterThanOrEqual = GreaterThanOrEqual(partitionColumnAttr, Literal(partitionValue.get)) + DeltaAdapter + .snapshotFilesForScan( + snapshot, + Seq(partitionColumnAttr), + Seq(isNotNullExpr, greaterThanOrEqual), + false) + .files + } else { + DeltaAdapter.snapshotFilesForScan(snapshot, Seq.empty, Seq.empty, false).files + } + + val executorIdsToAddFiles = + scala.collection.mutable.Map[String, ArrayBuffer[AddMergeTreeParts]]() + val executorIdsToParts = scala.collection.mutable.Map[String, String]() + executorIdsToAddFiles.put( + GlutenCHCacheDataCommand.ALL_EXECUTORS, + new ArrayBuffer[AddMergeTreeParts]()) + selectedAddFiles.foreach( + addFile => { + val mergeTreePart = addFile.asInstanceOf[AddMergeTreeParts] + val partName = mergeTreePart.name + val tableUri = URI.create(mergeTreePart.tablePath) + val relativeTablePath = if (tableUri.getPath.startsWith("/")) { + tableUri.getPath.substring(1) + } else tableUri.getPath + + val locations = CHAffinity.getNativeMergeTreePartLocations(partName, relativeTablePath) + + if (locations.isEmpty) { + // non soft affinity + executorIdsToAddFiles + .get(GlutenCHCacheDataCommand.ALL_EXECUTORS) + .get + .append(mergeTreePart) + } else { + locations.foreach( + executor => { + if (!executorIdsToAddFiles.contains(executor)) { + executorIdsToAddFiles.put(executor, new ArrayBuffer[AddMergeTreeParts]()) + } + executorIdsToAddFiles.get(executor).get.append(mergeTreePart) + }) + } + }) + + executorIdsToAddFiles.foreach( + value => { + val parts = value._2 + val executorId = value._1 + if (parts.nonEmpty) { + val onePart = parts(0) + val partNameList = parts.map(_.name).toSeq + // starts and lengths is useless for write + val partRanges = Seq.range(0L, partNameList.length).map(_ => long2Long(0L)).asJava + + val extensionTableNode = ExtensionTableBuilder.makeExtensionTable( + -1, + -1, + onePart.database, + onePart.table, + ClickhouseSnapshot.genSnapshotId(snapshot), + onePart.tablePath, + pathToCache.toString, + snapshot.metadata.configuration.getOrElse("orderByKey", ""), + snapshot.metadata.configuration.getOrElse("lowCardKey", ""), + snapshot.metadata.configuration.getOrElse("minmaxIndexKey", ""), + snapshot.metadata.configuration.getOrElse("bloomfilterIndexKey", ""), + snapshot.metadata.configuration.getOrElse("setIndexKey", ""), + snapshot.metadata.configuration.getOrElse("primaryKey", ""), + partNameList.asJava, + partRanges, + partRanges, + ConverterUtils.convertNamedStructJson(snapshot.metadata.schema), + snapshot.metadata.configuration.asJava, + new JList[String]() + ) + + executorIdsToParts.put(executorId, extensionTableNode.getExtensionTableStr) + } + }) + + // send rpc call + if (executorIdsToParts.contains(GlutenCHCacheDataCommand.ALL_EXECUTORS)) { + // send all parts to all executors + val tableMessage = executorIdsToParts.get(GlutenCHCacheDataCommand.ALL_EXECUTORS).get + if (asynExecute) { + GlutenDriverEndpoint.executorDataMap.forEach( + (executorId, executor) => { + executor.executorEndpointRef.send( + GlutenMergeTreeCacheLoad(tableMessage, selectedColumns.toSet.asJava)) + }) + Seq(Row(true, "")) + } else { + val futureList = ArrayBuffer[Future[CacheLoadResult]]() + val resultList = ArrayBuffer[CacheLoadResult]() + GlutenDriverEndpoint.executorDataMap.forEach( + (executorId, executor) => { + futureList.append( + executor.executorEndpointRef.ask[CacheLoadResult]( + GlutenMergeTreeCacheLoad(tableMessage, selectedColumns.toSet.asJava) + )) + }) + futureList.foreach( + f => { + resultList.append(ThreadUtils.awaitResult(f, Duration.Inf)) + }) + if (resultList.exists(!_.success)) { + Seq(Row(false, resultList.filter(!_.success).map(_.reason).mkString(";"))) + } else { + Seq(Row(true, "")) + } + } + } else { + if (asynExecute) { + executorIdsToParts.foreach( + value => { + val executorData = GlutenDriverEndpoint.executorDataMap.get(toExecutorId(value._1)) + if (executorData != null) { + executorData.executorEndpointRef.send( + GlutenMergeTreeCacheLoad(value._2, selectedColumns.toSet.asJava)) + } else { + throw new GlutenException( + s"executor ${value._1} not found," + + s" all executors are ${GlutenDriverEndpoint.executorDataMap.toString}") + } + }) + Seq(Row(true, "")) + } else { + val futureList = ArrayBuffer[Future[CacheLoadResult]]() + val resultList = ArrayBuffer[CacheLoadResult]() + executorIdsToParts.foreach( + value => { + val executorData = GlutenDriverEndpoint.executorDataMap.get(toExecutorId(value._1)) + if (executorData != null) { + futureList.append( + executorData.executorEndpointRef.ask[CacheLoadResult]( + GlutenMergeTreeCacheLoad(value._2, selectedColumns.toSet.asJava) + )) + } else { + throw new GlutenException( + s"executor ${value._1} not found," + + s" all executors are ${GlutenDriverEndpoint.executorDataMap.toString}") + } + }) + futureList.foreach( + f => { + resultList.append(ThreadUtils.awaitResult(f, Duration.Inf)) + }) + if (resultList.exists(!_.success)) { + Seq(Row(false, resultList.filter(!_.success).map(_.reason).mkString(";"))) + } else { + Seq(Row(true, "")) + } + } + } + } +} + +object GlutenCHCacheDataCommand { + val ALL_EXECUTORS = "allExecutors" + + private def toExecutorId(executorId: String): String = + executorId.split("_").last +} diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala index 8acc23aec2070..71d5c54318348 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala @@ -33,7 +33,7 @@ class AddMergeTreeParts( val database: String, val table: String, val engine: String, // default is "MergeTree" - override val path: String, // table path + val tablePath: String, // table path val targetNode: String, // the node which the current part is generated val name: String, // part name val uuid: String, @@ -98,7 +98,7 @@ object AddFileTags { database: String, table: String, engine: String, - path: String, + tablePath: String, targetNode: String, name: String, uuid: String, @@ -125,7 +125,7 @@ object AddFileTags { "database" -> database, "table" -> table, "engine" -> engine, - "path" -> path, + "path" -> tablePath, "targetNode" -> targetNode, "partition" -> partition, "uuid" -> uuid, @@ -161,7 +161,7 @@ object AddFileTags { addFile.tags.get("database").get, addFile.tags.get("table").get, addFile.tags.get("engine").get, - addFile.path, + addFile.tags.get("path").get, addFile.tags.get("targetNode").get, addFile.path, addFile.tags.get("uuid").get, @@ -199,6 +199,7 @@ object AddFileTags { mapper.readValue(returnedMetrics, new TypeReference[JList[WriteReturnedMetric]]() {}) var addFiles = new ArrayBuffer[AddFile]() val path = new Path(originPathStr) + val modificationTime = System.currentTimeMillis() addFiles.appendAll(values.asScala.map { value => AddFileTags.partsInfoToAddFile( @@ -213,7 +214,7 @@ object AddFileTags { value.getDiskSize, -1L, -1L, - -1L, + modificationTime, "", -1L, -1L, diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeCacheDataSSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeCacheDataSSuite.scala new file mode 100644 index 0000000000000..960c921789932 --- /dev/null +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeCacheDataSSuite.scala @@ -0,0 +1,401 @@ +/* + * 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.gluten.execution + +import org.apache.spark.SparkConf +import org.apache.spark.sql.delta.files.TahoeFileIndex +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.datasources.v2.clickhouse.metadata.AddMergeTreeParts + +import org.apache.commons.io.FileUtils +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileSystem + +import java.io.File + +import scala.concurrent.duration.DurationInt + +// Some sqls' line length exceeds 100 +// scalastyle:off line.size.limit + +class GlutenClickHouseMergeTreeCacheDataSSuite + extends GlutenClickHouseTPCHAbstractSuite + with AdaptiveSparkPlanHelper { + + override protected val needCopyParquetToTablePath = true + + override protected val tablesPath: String = basePath + "/tpch-data" + override protected val tpchQueries: String = rootPath + "queries/tpch-queries-ch" + override protected val queriesResults: String = rootPath + "mergetree-queries-output" + + override protected def createTPCHNotNullTables(): Unit = { + createNotNullTPCHTablesInParquet(tablesPath) + } + + override protected def sparkConf: SparkConf = { + super.sparkConf + .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + .set("spark.io.compression.codec", "LZ4") + .set("spark.sql.shuffle.partitions", "5") + .set("spark.sql.autoBroadcastJoinThreshold", "10MB") + .set("spark.sql.adaptive.enabled", "true") + .set("spark.gluten.sql.columnar.backend.ch.runtime_config.logger.level", "error") + .set("spark.gluten.soft-affinity.enabled", "true") + .set( + "spark.gluten.sql.columnar.backend.ch.runtime_settings.mergetree.merge_after_insert", + "false") + } + + override protected def beforeEach(): Unit = { + super.beforeEach() + val conf = new Configuration + conf.set("fs.defaultFS", HDFS_URL) + val fs = FileSystem.get(conf) + fs.delete(new org.apache.hadoop.fs.Path(HDFS_URL), true) + FileUtils.deleteDirectory(new File(HDFS_METADATA_PATH)) + FileUtils.forceMkdir(new File(HDFS_METADATA_PATH)) + FileUtils.deleteDirectory(new File(HDFS_CACHE_PATH)) + FileUtils.forceMkdir(new File(HDFS_CACHE_PATH)) + } + + def countFiles(directory: File): Int = { + if (directory.exists && directory.isDirectory) { + val files = directory.listFiles + val count = files + .count(_.isFile) + files.filter(_.isDirectory).map(countFiles).sum + count + } else { + 0 + } + } + + test("test cache mergetree data sync") { + spark.sql(s""" + |DROP TABLE IF EXISTS lineitem_mergetree_hdfs; + |""".stripMargin) + + spark.sql(s""" + |CREATE TABLE IF NOT EXISTS lineitem_mergetree_hdfs + |( + | l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity double, + | l_extendedprice double, + | l_discount double, + | l_tax double, + | l_returnflag string, + | l_linestatus string, + | l_shipdate date, + | l_commitdate date, + | l_receiptdate date, + | l_shipinstruct string, + | l_shipmode string, + | l_comment string + |) + |USING clickhouse + |PARTITIONED BY (l_shipdate) + |LOCATION '$HDFS_URL/test/lineitem_mergetree_hdfs' + |TBLPROPERTIES (storage_policy='__hdfs_main', + | orderByKey='l_linenumber,l_orderkey') + |""".stripMargin) + + spark.sql(s""" + | insert into table lineitem_mergetree_hdfs + | select * from lineitem a + | where a.l_shipdate between date'1995-01-01' and date'1995-01-31' + |""".stripMargin) + FileUtils.deleteDirectory(new File(HDFS_METADATA_PATH)) + FileUtils.forceMkdir(new File(HDFS_METADATA_PATH)) + val dataPath = new File(HDFS_CACHE_PATH) + val initial_cache_files = countFiles(dataPath) + + val res = spark + .sql(s""" + |cache data + | select l_orderkey, l_partkey from lineitem_mergetree_hdfs + | after l_shipdate AS OF '1995-01-10' + | CACHEPROPERTIES(storage_policy='__hdfs_main', + | aaa='ccc')""".stripMargin) + .collect() + assertResult(true)(res(0).getBoolean(0)) + val metaPath = new File(HDFS_METADATA_PATH + s"$sparkVersion/test/lineitem_mergetree_hdfs") + assertResult(true)(metaPath.exists() && metaPath.isDirectory) + assertResult(22)(metaPath.list().length) + assert(countFiles(dataPath) > initial_cache_files) + + val first_cache_files = countFiles(dataPath) + val res1 = spark.sql(s"cache data select * from lineitem_mergetree_hdfs").collect() + assertResult(true)(res1(0).getBoolean(0)) + assertResult(31)(metaPath.list().length) + assert(countFiles(dataPath) > first_cache_files) + + val sqlStr = + s""" + |SELECT + | l_returnflag, + | l_linestatus, + | sum(l_quantity) AS sum_qty, + | sum(l_extendedprice) AS sum_base_price, + | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, + | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, + | avg(l_quantity) AS avg_qty, + | avg(l_extendedprice) AS avg_price, + | avg(l_discount) AS avg_disc, + | count(*) AS count_order + |FROM + | lineitem_mergetree_hdfs + |WHERE + | l_shipdate >= date'1995-01-10' + |GROUP BY + | l_returnflag, + | l_linestatus + |ORDER BY + | l_returnflag, + | l_linestatus; + | + |""".stripMargin + runSql(sqlStr)( + df => { + val scanExec = collect(df.queryExecution.executedPlan) { + case f: FileSourceScanExecTransformer => f + } + assertResult(1)(scanExec.size) + + val mergetreeScan = scanExec.head + assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + + val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] + val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddMergeTreeParts]) + assertResult(7898)(addFiles.map(_.rows).sum) + }) + spark.sql("drop table lineitem_mergetree_hdfs purge") + } + + test("test cache mergetree data async") { + spark.sql(s""" + |DROP TABLE IF EXISTS lineitem_mergetree_hdfs; + |""".stripMargin) + + spark.sql(s""" + |CREATE TABLE IF NOT EXISTS lineitem_mergetree_hdfs + |( + | l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity double, + | l_extendedprice double, + | l_discount double, + | l_tax double, + | l_returnflag string, + | l_linestatus string, + | l_shipdate date, + | l_commitdate date, + | l_receiptdate date, + | l_shipinstruct string, + | l_shipmode string, + | l_comment string + |) + |USING clickhouse + |PARTITIONED BY (l_shipdate) + |LOCATION '$HDFS_URL/test/lineitem_mergetree_hdfs' + |TBLPROPERTIES (storage_policy='__hdfs_main', + | orderByKey='l_linenumber,l_orderkey') + |""".stripMargin) + + spark.sql(s""" + | insert into table lineitem_mergetree_hdfs + | select * from lineitem a + | where a.l_shipdate between date'1995-01-01' and date'1995-01-31' + |""".stripMargin) + FileUtils.deleteDirectory(new File(HDFS_METADATA_PATH)) + FileUtils.forceMkdir(new File(HDFS_METADATA_PATH)) + val dataPath = new File(HDFS_CACHE_PATH) + val initial_cache_files = countFiles(dataPath) + + val res = spark + .sql(s""" + |cache data async + | select * from lineitem_mergetree_hdfs + | after l_shipdate AS OF '1995-01-10' + | CACHEPROPERTIES(storage_policy='__hdfs_main', + | aaa='ccc')""".stripMargin) + .collect() + assertResult(true)(res(0).getBoolean(0)) + val metaPath = new File(HDFS_METADATA_PATH + s"$sparkVersion/test/lineitem_mergetree_hdfs") + assertResult(true)(metaPath.exists() && metaPath.isDirectory) + eventually(timeout(60.seconds), interval(2.seconds)) { + assertResult(22)(metaPath.list().length) + assert(countFiles(dataPath) > initial_cache_files) + } + + val first_cache_files = countFiles(dataPath) + val res1 = spark.sql(s"cache data async select * from lineitem_mergetree_hdfs").collect() + assertResult(true)(res1(0).getBoolean(0)) + eventually(timeout(60.seconds), interval(2.seconds)) { + assertResult(31)(metaPath.list().length) + assert(countFiles(dataPath) > first_cache_files) + } + + val sqlStr = + s""" + |SELECT + | l_returnflag, + | l_linestatus, + | sum(l_quantity) AS sum_qty, + | sum(l_extendedprice) AS sum_base_price, + | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, + | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, + | avg(l_quantity) AS avg_qty, + | avg(l_extendedprice) AS avg_price, + | avg(l_discount) AS avg_disc, + | count(*) AS count_order + |FROM + | lineitem_mergetree_hdfs + |WHERE + | l_shipdate >= date'1995-01-10' + |GROUP BY + | l_returnflag, + | l_linestatus + |ORDER BY + | l_returnflag, + | l_linestatus; + | + |""".stripMargin + runSql(sqlStr)( + df => { + val scanExec = collect(df.queryExecution.executedPlan) { + case f: FileSourceScanExecTransformer => f + } + assertResult(1)(scanExec.size) + + val mergetreeScan = scanExec.head + assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + + val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] + val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddMergeTreeParts]) + assertResult(7898)(addFiles.map(_.rows).sum) + }) + spark.sql("drop table lineitem_mergetree_hdfs purge") + } + + test("test cache mergetree data with the path") { + spark.sql(s""" + |DROP TABLE IF EXISTS lineitem_mergetree_hdfs; + |""".stripMargin) + + spark.sql(s""" + |CREATE TABLE IF NOT EXISTS lineitem_mergetree_hdfs + |( + | l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity double, + | l_extendedprice double, + | l_discount double, + | l_tax double, + | l_returnflag string, + | l_linestatus string, + | l_shipdate date, + | l_commitdate date, + | l_receiptdate date, + | l_shipinstruct string, + | l_shipmode string, + | l_comment string + |) + |USING clickhouse + |PARTITIONED BY (l_shipdate) + |LOCATION '$HDFS_URL/test/lineitem_mergetree_hdfs' + |TBLPROPERTIES (storage_policy='__hdfs_main', + | orderByKey='l_linenumber,l_orderkey') + |""".stripMargin) + + spark.sql(s""" + | insert into table lineitem_mergetree_hdfs + | select * from lineitem a + | where a.l_shipdate between date'1995-01-01' and date'1995-01-31' + |""".stripMargin) + FileUtils.deleteDirectory(new File(HDFS_METADATA_PATH)) + FileUtils.forceMkdir(new File(HDFS_METADATA_PATH)) + val dataPath = new File(HDFS_CACHE_PATH) + val initial_cache_files = countFiles(dataPath) + + val res = spark + .sql(s""" + |cache data + | select * from '$HDFS_URL/test/lineitem_mergetree_hdfs' + | after l_shipdate AS OF '1995-01-10' + | CACHEPROPERTIES(storage_policy='__hdfs_main', + | aaa='ccc')""".stripMargin) + .collect() + assertResult(true)(res(0).getBoolean(0)) + val metaPath = new File(HDFS_METADATA_PATH + s"$sparkVersion/test/lineitem_mergetree_hdfs") + assertResult(true)(metaPath.exists() && metaPath.isDirectory) + assertResult(22)(metaPath.list().length) + assert(countFiles(dataPath) > initial_cache_files) + val first_cache_files = countFiles(dataPath) + val res1 = spark.sql(s"cache data select * from lineitem_mergetree_hdfs").collect() + assertResult(true)(res1(0).getBoolean(0)) + assertResult(31)(metaPath.list().length) + assert(countFiles(dataPath) > first_cache_files) + + val sqlStr = + s""" + |SELECT + | l_returnflag, + | l_linestatus, + | sum(l_quantity) AS sum_qty, + | sum(l_extendedprice) AS sum_base_price, + | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, + | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, + | avg(l_quantity) AS avg_qty, + | avg(l_extendedprice) AS avg_price, + | avg(l_discount) AS avg_disc, + | count(*) AS count_order + |FROM + | lineitem_mergetree_hdfs + |WHERE + | l_shipdate >= date'1995-01-10' + |GROUP BY + | l_returnflag, + | l_linestatus + |ORDER BY + | l_returnflag, + | l_linestatus; + | + |""".stripMargin + runSql(sqlStr)( + df => { + val scanExec = collect(df.queryExecution.executedPlan) { + case f: FileSourceScanExecTransformer => f + } + assertResult(1)(scanExec.size) + + val mergetreeScan = scanExec.head + assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + + val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] + val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddMergeTreeParts]) + assertResult(7898)(addFiles.map(_.rows).sum) + }) + spark.sql("drop table lineitem_mergetree_hdfs purge") + } +} +// scalastyle:off line.size.limit diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeWriteTableSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeWriteTableSuite.scala index 0f642dfa86647..2fec68a492163 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeWriteTableSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeWriteTableSuite.scala @@ -65,7 +65,7 @@ class GlutenClickHouseNativeWriteTableSuite // TODO: support default ANSI policy .set("spark.sql.storeAssignmentPolicy", "legacy") .set("spark.sql.warehouse.dir", getWarehouseDir) - .set("spark.gluten.sql.columnar.backend.ch.runtime_config.logger.level", "debug") + .set("spark.gluten.sql.columnar.backend.ch.runtime_config.logger.level", "error") .setMaster("local[1]") } diff --git a/cpp-ch/local-engine/CMakeLists.txt b/cpp-ch/local-engine/CMakeLists.txt index 93ee4b8218af1..2bf99a4949fc7 100644 --- a/cpp-ch/local-engine/CMakeLists.txt +++ b/cpp-ch/local-engine/CMakeLists.txt @@ -53,6 +53,7 @@ add_headers_and_sources(storages Storages/Output) add_headers_and_sources(storages Storages/Serializations) add_headers_and_sources(storages Storages/IO) add_headers_and_sources(storages Storages/Mergetree) +add_headers_and_sources(storages Storages/Cache) add_headers_and_sources(common Common) add_headers_and_sources(external External) add_headers_and_sources(shuffle Shuffle) diff --git a/cpp-ch/local-engine/Common/CHUtil.cpp b/cpp-ch/local-engine/Common/CHUtil.cpp index 88867e290abb0..12bf7ed599392 100644 --- a/cpp-ch/local-engine/Common/CHUtil.cpp +++ b/cpp-ch/local-engine/Common/CHUtil.cpp @@ -71,6 +71,7 @@ #include #include #include +#include namespace DB { @@ -975,6 +976,8 @@ void BackendInitializerUtil::init(const std::string_view plan) // Init the table metadata cache map StorageMergeTreeFactory::init_cache_map(); + CacheManager::initialize(SerializedPlanParser::global_context); + std::call_once( init_flag, [&] diff --git a/cpp-ch/local-engine/Common/MergeTreeTool.cpp b/cpp-ch/local-engine/Common/MergeTreeTool.cpp index d3b7d7b229a12..31994170f76df 100644 --- a/cpp-ch/local-engine/Common/MergeTreeTool.cpp +++ b/cpp-ch/local-engine/Common/MergeTreeTool.cpp @@ -231,4 +231,36 @@ RangesInDataParts MergeTreeTable::extractRange(DataPartsVector parts_vector) con }); return ranges_in_data_parts; } + +bool sameColumns(const substrait::NamedStruct & left, const substrait::NamedStruct & right) +{ + if (left.names_size() != right.names_size()) + return false; + std::unordered_map map; + for (size_t i = 0; i < left.names_size(); i++) + map.emplace(left.names(i), left.struct_().types(i).kind_case()); + for (size_t i = 0; i < right.names_size(); i++) + { + if (!map.contains(right.names(i)) || map[right.names(i)] != right.struct_().types(i).kind_case()) + return false; + } + return true; +} + +bool MergeTreeTable::sameStructWith(const MergeTreeTable & other) +{ + return database == other.database && + table == other.table && + snapshot_id == other.snapshot_id && + sameColumns(schema, other.schema) && + order_by_key == other.order_by_key && + low_card_key == other.low_card_key && + minmax_index_key == other.minmax_index_key && + bf_index_key == other.bf_index_key && + set_index_key == other.set_index_key && + primary_key == other.primary_key && + relative_path == other.relative_path && + absolute_path == other.absolute_path && + table_configs.storage_policy == other.table_configs.storage_policy; +} } \ No newline at end of file diff --git a/cpp-ch/local-engine/Common/MergeTreeTool.h b/cpp-ch/local-engine/Common/MergeTreeTool.h index 0f0a1c1c77dd0..fc312eba9e3c5 100644 --- a/cpp-ch/local-engine/Common/MergeTreeTool.h +++ b/cpp-ch/local-engine/Common/MergeTreeTool.h @@ -67,6 +67,7 @@ struct MergeTreeTable std::vector parts; std::unordered_set getPartNames() const; RangesInDataParts extractRange(DataPartsVector parts_vector) const; + bool sameStructWith(const MergeTreeTable& other); }; std::shared_ptr buildMetaData(const DB::NamesAndTypesList &columns, ContextPtr context, const MergeTreeTable &); diff --git a/cpp-ch/local-engine/Parser/MergeTreeRelParser.cpp b/cpp-ch/local-engine/Parser/MergeTreeRelParser.cpp index 967f8ba70c9ac..b1b024ce5e6d8 100644 --- a/cpp-ch/local-engine/Parser/MergeTreeRelParser.cpp +++ b/cpp-ch/local-engine/Parser/MergeTreeRelParser.cpp @@ -78,6 +78,7 @@ MergeTreeRelParser::parseStorage(const MergeTreeTable & merge_tree_table, Contex auto global_storage = StorageMergeTreeFactory::getStorage( StorageID(merge_tree_table.database, merge_tree_table.table), merge_tree_table.snapshot_id, + merge_tree_table, [&]() -> CustomStorageMergeTreePtr { auto custom_storage_merge_tree = std::make_shared( @@ -98,13 +99,6 @@ MergeTreeRelParser::parseStorage(const MergeTreeTable & merge_tree_table, Contex return global_storage; } -CustomStorageMergeTreePtr -MergeTreeRelParser::parseStorage(const substrait::ReadRel::ExtensionTable & extension_table, ContextMutablePtr context) -{ - auto merge_tree_table = parseMergeTreeTable(extension_table); - return parseStorage(merge_tree_table, context, true); -} - CustomStorageMergeTreePtr MergeTreeRelParser::copyToDefaultPolicyStorage(MergeTreeTable merge_tree_table, ContextMutablePtr context) { @@ -131,7 +125,9 @@ DB::QueryPlanPtr MergeTreeRelParser::parseReadRel( DB::QueryPlanPtr query_plan, const substrait::ReadRel & rel, const substrait::ReadRel::ExtensionTable & extension_table) { auto merge_tree_table = parseMergeTreeTable(extension_table); - auto storage = parseStorage(extension_table, global_context); + // ignore snapshot id for query + merge_tree_table.snapshot_id = ""; + auto storage = parseStorage(merge_tree_table, global_context, true); DB::Block input; if (rel.has_base_schema() && rel.base_schema().names_size()) @@ -392,6 +388,8 @@ String MergeTreeRelParser::filterRangesOnDriver(const substrait::ReadRel & read_ google::protobuf::StringValue table; table.ParseFromString(read_rel.advanced_extension().enhancement().value()); auto merge_tree_table = parseMergeTreeTableString(table.value()); + // ignore snapshot id for query + merge_tree_table.snapshot_id = ""; auto custom_storage_mergetree = parseStorage(merge_tree_table, global_context, true); auto input = TypeParser::buildBlockFromNamedStruct(read_rel.base_schema()); diff --git a/cpp-ch/local-engine/Parser/MergeTreeRelParser.h b/cpp-ch/local-engine/Parser/MergeTreeRelParser.h index 1c9ea736cd43a..b26239dc4d36b 100644 --- a/cpp-ch/local-engine/Parser/MergeTreeRelParser.h +++ b/cpp-ch/local-engine/Parser/MergeTreeRelParser.h @@ -40,8 +40,6 @@ using namespace DB; class MergeTreeRelParser : public RelParser { public: - static CustomStorageMergeTreePtr - parseStorage(const substrait::ReadRel::ExtensionTable & extension_table, ContextMutablePtr context); static CustomStorageMergeTreePtr parseStorage( const MergeTreeTable & merge_tree_table, ContextMutablePtr context, bool restore = false); diff --git a/cpp-ch/local-engine/Storages/Cache/CacheManager.cpp b/cpp-ch/local-engine/Storages/Cache/CacheManager.cpp new file mode 100644 index 0000000000000..d2c7b06810db6 --- /dev/null +++ b/cpp-ch/local-engine/Storages/Cache/CacheManager.cpp @@ -0,0 +1,143 @@ +/* + * 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. + */ +#include "CacheManager.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ +extern const int INVALID_STATE; +} +} + +namespace CurrentMetrics +{ +extern const Metric LocalThread; +extern const Metric LocalThreadActive; +extern const Metric LocalThreadScheduled; +} + +namespace local_engine +{ +CacheManager & CacheManager::instance() +{ + static CacheManager cache_manager; + return cache_manager; +} + +void CacheManager::initialize(DB::ContextMutablePtr context_) +{ + auto & manager = instance(); + manager.context = context_; + manager.thread_pool = std::make_unique( + CurrentMetrics::LocalThread, + CurrentMetrics::LocalThreadActive, + CurrentMetrics::LocalThreadScheduled, + manager.context->getConfigRef().getInt("cache_sync_max_threads", 10), + 0, + 0); +} + +struct CacheJobContext +{ + MergeTreeTable table; +}; + +void CacheManager::cachePart(const MergeTreeTable& table, const MergeTreePart& part, const std::unordered_set & columns, std::shared_ptr latch) +{ + CacheJobContext job_context{table}; + job_context.table.parts.clear(); + job_context.table.parts.push_back(part); + job_context.table.snapshot_id = ""; + auto job = [job_detail = job_context, context = this->context, read_columns = columns, latch = latch]() + { + try + { + SCOPE_EXIT({ if (latch) latch->count_down();}); + auto storage = MergeTreeRelParser::parseStorage(job_detail.table, context, true); + auto storage_snapshot = std::make_shared(*storage, storage->getInMemoryMetadataPtr()); + NamesAndTypesList names_and_types_list; + auto meta_columns = storage->getInMemoryMetadata().getColumns(); + for (const auto & column : meta_columns) + { + if (read_columns.contains(column.name)) + names_and_types_list.push_back(NameAndTypePair(column.name, column.type)); + } + auto query_info = buildQueryInfo(names_and_types_list); + std::vector selected_parts + = StorageMergeTreeFactory::getDataPartsByNames(storage->getStorageID(), "", {job_detail.table.parts.front().name}); + auto read_step = storage->reader.readFromParts( + selected_parts, + /* alter_conversions = */ + {}, + names_and_types_list.getNames(), + storage_snapshot, + *query_info, + context, + context->getSettingsRef().max_block_size, + 1); + QueryPlan plan; + plan.addStep(std::move(read_step)); + auto pipeline_builder = plan.buildQueryPipeline({}, {}); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*pipeline_builder.get())); + PullingPipelineExecutor executor(pipeline); + while (true) + { + Chunk chunk; + if (!executor.pull(chunk)) + break; + } + LOG_INFO(getLogger("CacheManager"), "Load cache of table {}.{} part {} success.", job_detail.table.database, job_detail.table.table, job_detail.table.parts.front().name); + } + catch (std::exception& e) + { + LOG_ERROR(getLogger("CacheManager"), "Load cache of table {}.{} part {} failed.\n {}", job_detail.table.database, job_detail.table.table, job_detail.table.parts.front().name, e.what()); + } + }; + LOG_INFO(getLogger("CacheManager"), "Loading cache of table {}.{} part {}", job_context.table.database, job_context.table.table, job_context.table.parts.front().name); + thread_pool->scheduleOrThrowOnError(std::move(job)); +} + +void CacheManager::cacheParts(const String& table_def, const std::unordered_set& columns, bool async) +{ + auto table = parseMergeTreeTableString(table_def); + std::shared_ptr latch = nullptr; + if (!async) latch = std::make_shared(table.parts.size()); + for (const auto & part : table.parts) + { + cachePart(table, part, columns, latch); + } + if (latch) + latch->wait(); +} +} \ No newline at end of file diff --git a/cpp-ch/local-engine/Storages/Cache/CacheManager.h b/cpp-ch/local-engine/Storages/Cache/CacheManager.h new file mode 100644 index 0000000000000..a303b7b7fc63e --- /dev/null +++ b/cpp-ch/local-engine/Storages/Cache/CacheManager.h @@ -0,0 +1,44 @@ +/* + * 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. + */ +#pragma once +#include +#include + + +namespace local_engine +{ +struct MergeTreePart; +struct MergeTreeTable; +/*** + * Manage the cache of the MergeTree, mainly including meta.bin, data.bin, metadata.gluten + */ +class CacheManager { +public: + static CacheManager & instance(); + static void initialize(DB::ContextMutablePtr context); + void cachePart(const MergeTreeTable& table, const MergeTreePart& part, const std::unordered_set& columns, std::shared_ptr latch = nullptr); + void cacheParts(const String& table_def, const std::unordered_set& columns, bool async = true); +private: + CacheManager() = default; + + std::unique_ptr thread_pool; + DB::ContextMutablePtr context; + std::unordered_map policy_to_disk; + std::unordered_map disk_to_metadisk; + std::unordered_map policy_to_cache; +}; +} \ No newline at end of file diff --git a/cpp-ch/local-engine/Storages/CustomMergeTreeSink.h b/cpp-ch/local-engine/Storages/CustomMergeTreeSink.h deleted file mode 100644 index acb9702c375c5..0000000000000 --- a/cpp-ch/local-engine/Storages/CustomMergeTreeSink.h +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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. - */ -#pragma once - -#include -#include -#include -#include "CustomStorageMergeTree.h" - -namespace local_engine -{ -class CustomMergeTreeSink : public ISink -{ -public: - CustomMergeTreeSink(CustomStorageMergeTree & storage_, const StorageMetadataPtr metadata_snapshot_, ContextPtr context_) - : ISink(metadata_snapshot_->getSampleBlock()), storage(storage_), metadata_snapshot(metadata_snapshot_), context(context_) - { - } - - String getName() const override { return "CustomMergeTreeSink"; } - void consume(Chunk chunk) override; - -private: - CustomStorageMergeTree & storage; - StorageMetadataPtr metadata_snapshot; - ContextPtr context; -}; - -} diff --git a/cpp-ch/local-engine/Storages/StorageMergeTreeFactory.cpp b/cpp-ch/local-engine/Storages/StorageMergeTreeFactory.cpp index c59d6ddb4bd41..3f7aac8724a09 100644 --- a/cpp-ch/local-engine/Storages/StorageMergeTreeFactory.cpp +++ b/cpp-ch/local-engine/Storages/StorageMergeTreeFactory.cpp @@ -18,6 +18,9 @@ #include +#include +#include + namespace local_engine { @@ -55,14 +58,24 @@ void StorageMergeTreeFactory::freeStorage(const StorageID & id, const String & s } } + CustomStorageMergeTreePtr -StorageMergeTreeFactory::getStorage(StorageID id, const String & snapshot_id, std::function creator) +StorageMergeTreeFactory::getStorage(const StorageID& id, const String & snapshot_id, MergeTreeTable merge_tree_table, std::function creator) { auto table_name = getTableName(id, snapshot_id); std::lock_guard lock(storage_map_mutex); + + merge_tree_table.parts.clear(); + if (storage_map->has(table_name) && !storage_map->get(table_name)->second.sameStructWith(merge_tree_table)) + { + freeStorage(id); + std::lock_guard lock_datapart(datapart_mutex); + if (datapart_map->has(table_name)) + datapart_map->remove(table_name); + } if (!storage_map->has(table_name)) - storage_map->add(table_name, creator()); - return *(storage_map->get(table_name)); + storage_map->add(table_name, {creator(), merge_tree_table}); + return storage_map->get(table_name)->first; } DataPartsVector StorageMergeTreeFactory::getDataPartsByNames(const StorageID & id, const String & snapshot_id, std::unordered_set part_name) @@ -96,7 +109,7 @@ DataPartsVector StorageMergeTreeFactory::getDataPartsByNames(const StorageID & i CustomStorageMergeTreePtr storage_merge_tree; { std::lock_guard storage_lock(storage_map_mutex); - storage_merge_tree = *(storage_map->get(table_name)); + storage_merge_tree = storage_map->get(table_name)->first; } auto missing_parts = storage_merge_tree->loadDataPartsWithNames(missing_names); for (const auto & part : missing_parts) @@ -108,9 +121,9 @@ DataPartsVector StorageMergeTreeFactory::getDataPartsByNames(const StorageID & i return res; } // will be inited in native init phase -std::unique_ptr> StorageMergeTreeFactory::storage_map = nullptr; +std::unique_ptr>> StorageMergeTreeFactory::storage_map = nullptr; std::unique_ptr>>> StorageMergeTreeFactory::datapart_map = nullptr; -std::mutex StorageMergeTreeFactory::storage_map_mutex; -std::mutex StorageMergeTreeFactory::datapart_mutex; +std::recursive_mutex StorageMergeTreeFactory::storage_map_mutex; +std::recursive_mutex StorageMergeTreeFactory::datapart_mutex; } diff --git a/cpp-ch/local-engine/Storages/StorageMergeTreeFactory.h b/cpp-ch/local-engine/Storages/StorageMergeTreeFactory.h index f372175bb02ce..3fa8c6285bbeb 100644 --- a/cpp-ch/local-engine/Storages/StorageMergeTreeFactory.h +++ b/cpp-ch/local-engine/Storages/StorageMergeTreeFactory.h @@ -16,11 +16,13 @@ */ #pragma once #include +#include #include #include #include #include + namespace local_engine { using CustomStorageMergeTreePtr = std::shared_ptr; @@ -31,7 +33,7 @@ class StorageMergeTreeFactory static StorageMergeTreeFactory & instance(); static void freeStorage(const StorageID & id, const String & snapshot_id = ""); static CustomStorageMergeTreePtr - getStorage(StorageID id, const String & snapshot_id, std::function creator); + getStorage(const StorageID& id, const String & snapshot_id, MergeTreeTable merge_tree_table, std::function creator); static DataPartsVector getDataPartsByNames(const StorageID & id, const String & snapshot_id, std::unordered_set part_name); static void init_cache_map() { @@ -39,7 +41,7 @@ class StorageMergeTreeFactory auto & storage_map_v = storage_map; if (!storage_map_v) { - storage_map_v = std::make_unique>(config.table_metadata_cache_max_count); + storage_map_v = std::make_unique>>(config.table_metadata_cache_max_count); } else { @@ -65,10 +67,10 @@ class StorageMergeTreeFactory static String getTableName(const StorageID & id, const String & snapshot_id); private: - static std::unique_ptr> storage_map; + static std::unique_ptr>> storage_map; static std::unique_ptr>>> datapart_map; - static std::mutex storage_map_mutex; - static std::mutex datapart_mutex; + static std::recursive_mutex storage_map_mutex; + static std::recursive_mutex datapart_mutex; }; struct TempStorageFreer diff --git a/cpp-ch/local-engine/local_engine_jni.cpp b/cpp-ch/local-engine/local_engine_jni.cpp index 3a18771cdd520..8807a0f639911 100644 --- a/cpp-ch/local-engine/local_engine_jni.cpp +++ b/cpp-ch/local-engine/local_engine_jni.cpp @@ -55,6 +55,7 @@ #include #include #include +#include #ifdef __cplusplus @@ -1252,6 +1253,21 @@ JNIEXPORT void Java_org_apache_gluten_utils_TestExceptionUtils_generateNativeExc +JNIEXPORT void Java_org_apache_gluten_execution_CHNativeCacheManager_nativeCacheParts(JNIEnv * env, jobject, jstring table_, jstring columns_, jboolean async_) +{ + LOCAL_ENGINE_JNI_METHOD_START + auto table_def = jstring2string(env, table_); + auto columns = jstring2string(env, columns_); + Poco::StringTokenizer tokenizer(columns, ","); + std::unordered_set column_set; + for (const auto & col : tokenizer) + { + column_set.insert(col); + } + local_engine::CacheManager::instance().cacheParts(table_def, column_set, async_); + LOCAL_ENGINE_JNI_METHOD_END(env, ); +} + #ifdef __cplusplus } diff --git a/gluten-core/src/main/scala/org/apache/gluten/backendsapi/SparkPlanExecApi.scala b/gluten-core/src/main/scala/org/apache/gluten/backendsapi/SparkPlanExecApi.scala index 273443f647ab6..8f24afae1da48 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/backendsapi/SparkPlanExecApi.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/backendsapi/SparkPlanExecApi.scala @@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.optimizer.BuildSide +import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, Partitioning} @@ -468,6 +469,9 @@ trait SparkPlanExecApi { def genInjectPostHocResolutionRules(): List[SparkSession => Rule[LogicalPlan]] + def genInjectExtendedParser(): List[(SparkSession, ParserInterface) => ParserInterface] = + List.empty + def genGetStructFieldTransformer( substraitExprName: String, childTransformer: ExpressionTransformer, diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/OthersExtensionOverrides.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/OthersExtensionOverrides.scala index 0897f411fce51..f2ccf6e81ca17 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/OthersExtensionOverrides.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/OthersExtensionOverrides.scala @@ -23,6 +23,9 @@ import org.apache.spark.sql.SparkSessionExtensions object OthersExtensionOverrides extends GlutenSparkExtensionsInjector { override def inject(extensions: SparkSessionExtensions): Unit = { + BackendsApiManager.getSparkPlanExecApiInstance + .genInjectExtendedParser() + .foreach(extensions.injectParser) BackendsApiManager.getSparkPlanExecApiInstance .genExtendedAnalyzers() .foreach(extensions.injectResolutionRule) diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala index 4df9c63b3ef6d..6816534094f32 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.extension import org.apache.gluten.extension.ColumnarOverrideRules +import org.apache.gluten.utils.BackendTestUtils import org.apache.spark.SparkConf import org.apache.spark.sql._ @@ -37,7 +38,12 @@ class GlutenSessionExtensionSuite extends GlutenSQLTestsTrait { assert(spark.sessionState.analyzer.postHocResolutionRules.contains(MyRule(spark))) assert(spark.sessionState.analyzer.extendedCheckRules.contains(MyCheckRule(spark))) assert(spark.sessionState.optimizer.batches.flatMap(_.rules).contains(MyRule(spark))) - assert(spark.sessionState.sqlParser.isInstanceOf[MyParser]) + if (BackendTestUtils.isCHBackendLoaded()) { + assert( + spark.sessionState.sqlParser.getClass.getSimpleName.equals("GlutenClickhouseSqlParser")) + } else { + assert(spark.sessionState.sqlParser.isInstanceOf[MyParser]) + } assert( spark.sessionState.functionRegistry .lookupFunction(MyExtensions.myFunction._1) diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala index 4df9c63b3ef6d..6816534094f32 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.extension import org.apache.gluten.extension.ColumnarOverrideRules +import org.apache.gluten.utils.BackendTestUtils import org.apache.spark.SparkConf import org.apache.spark.sql._ @@ -37,7 +38,12 @@ class GlutenSessionExtensionSuite extends GlutenSQLTestsTrait { assert(spark.sessionState.analyzer.postHocResolutionRules.contains(MyRule(spark))) assert(spark.sessionState.analyzer.extendedCheckRules.contains(MyCheckRule(spark))) assert(spark.sessionState.optimizer.batches.flatMap(_.rules).contains(MyRule(spark))) - assert(spark.sessionState.sqlParser.isInstanceOf[MyParser]) + if (BackendTestUtils.isCHBackendLoaded()) { + assert( + spark.sessionState.sqlParser.getClass.getSimpleName.equals("GlutenClickhouseSqlParser")) + } else { + assert(spark.sessionState.sqlParser.isInstanceOf[MyParser]) + } assert( spark.sessionState.functionRegistry .lookupFunction(MyExtensions.myFunction._1) diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala index 4df9c63b3ef6d..6816534094f32 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.extension import org.apache.gluten.extension.ColumnarOverrideRules +import org.apache.gluten.utils.BackendTestUtils import org.apache.spark.SparkConf import org.apache.spark.sql._ @@ -37,7 +38,12 @@ class GlutenSessionExtensionSuite extends GlutenSQLTestsTrait { assert(spark.sessionState.analyzer.postHocResolutionRules.contains(MyRule(spark))) assert(spark.sessionState.analyzer.extendedCheckRules.contains(MyCheckRule(spark))) assert(spark.sessionState.optimizer.batches.flatMap(_.rules).contains(MyRule(spark))) - assert(spark.sessionState.sqlParser.isInstanceOf[MyParser]) + if (BackendTestUtils.isCHBackendLoaded()) { + assert( + spark.sessionState.sqlParser.getClass.getSimpleName.equals("GlutenClickhouseSqlParser")) + } else { + assert(spark.sessionState.sqlParser.isInstanceOf[MyParser]) + } assert( spark.sessionState.functionRegistry .lookupFunction(MyExtensions.myFunction._1) diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala index 4df9c63b3ef6d..6816534094f32 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.extension import org.apache.gluten.extension.ColumnarOverrideRules +import org.apache.gluten.utils.BackendTestUtils import org.apache.spark.SparkConf import org.apache.spark.sql._ @@ -37,7 +38,12 @@ class GlutenSessionExtensionSuite extends GlutenSQLTestsTrait { assert(spark.sessionState.analyzer.postHocResolutionRules.contains(MyRule(spark))) assert(spark.sessionState.analyzer.extendedCheckRules.contains(MyCheckRule(spark))) assert(spark.sessionState.optimizer.batches.flatMap(_.rules).contains(MyRule(spark))) - assert(spark.sessionState.sqlParser.isInstanceOf[MyParser]) + if (BackendTestUtils.isCHBackendLoaded()) { + assert( + spark.sessionState.sqlParser.getClass.getSimpleName.equals("GlutenClickhouseSqlParser")) + } else { + assert(spark.sessionState.sqlParser.isInstanceOf[MyParser]) + } assert( spark.sessionState.functionRegistry .lookupFunction(MyExtensions.myFunction._1) diff --git a/pom.xml b/pom.xml index cbec5befba87c..6f6b2cd57f673 100644 --- a/pom.xml +++ b/pom.xml @@ -73,6 +73,7 @@ 2.7.4 2.0.7 2.20.0 + 4.9.3 UTF-8 UTF-8 spark-sql-columnar @@ -261,6 +262,7 @@ delta-core 2.0.1 20 + 4.8 @@ -275,6 +277,7 @@ delta-core 2.3.0 23 + 4.8 @@ -288,6 +291,7 @@ delta-core 2.4.0 24 + 4.9.3 @@ -303,6 +307,7 @@ 32 2.15.1 3.3.4 + 4.9.3 @@ -989,6 +994,11 @@ protobuf-maven-plugin 0.5.1 + + org.antlr + antlr4-maven-plugin + ${antlr4.version} + From 944c92623ae07430d48b7569ae52ec14eefcfb0a Mon Sep 17 00:00:00 2001 From: Nicholas Jiang Date: Sun, 4 Aug 2024 14:37:03 +0800 Subject: [PATCH 58/61] [GLUTEN-6695][CH] Introduce shuffleWallTime in CHMetricsApi to calculate the overall shuffle write time (#6696) --- .../apache/gluten/backendsapi/clickhouse/CHMetricsApi.scala | 1 + .../spark/shuffle/CHCelebornColumnarShuffleWriter.scala | 4 ++++ 2 files changed, 5 insertions(+) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHMetricsApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHMetricsApi.scala index 85b298fa48354..0ff53e1c58178 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHMetricsApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHMetricsApi.scala @@ -222,6 +222,7 @@ class CHMetricsApi extends MetricsApi with Logging with LogLevelUtil { "spillTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time to spill"), "compressTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time to compress"), "prepareTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time to prepare"), + "shuffleWallTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "shuffle wall time"), "avgReadBatchNumRows" -> SQLMetrics .createAverageMetric(sparkContext, "avg read batch num rows"), "numInputRows" -> SQLMetrics.createMetric(sparkContext, "number of input rows"), diff --git a/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornColumnarShuffleWriter.scala b/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornColumnarShuffleWriter.scala index ae22a08908193..c7d7957c15b6b 100644 --- a/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornColumnarShuffleWriter.scala +++ b/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornColumnarShuffleWriter.scala @@ -63,7 +63,9 @@ class CHCelebornColumnarShuffleWriter[K, V]( } else { initShuffleWriter(cb) val col = cb.column(0).asInstanceOf[CHColumnVector] + val startTime = System.nanoTime() jniWrapper.split(nativeShuffleWriter, col.getBlockAddress) + dep.metrics("shuffleWallTime").add(System.nanoTime() - startTime) dep.metrics("numInputRows").add(cb.numRows) dep.metrics("inputBatches").add(1) // This metric is important, AQE use it to decide if EliminateLimit @@ -77,8 +79,10 @@ class CHCelebornColumnarShuffleWriter[K, V]( return } + val startTime = System.nanoTime() splitResult = jniWrapper.stop(nativeShuffleWriter) + dep.metrics("shuffleWallTime").add(System.nanoTime() - startTime) dep.metrics("splitTime").add(splitResult.getSplitTime) dep.metrics("IOTime").add(splitResult.getDiskWriteTime) dep.metrics("serializeTime").add(splitResult.getSerializationTime) From 5d6c6f3a04bdcdf94157d140dbee3c1db512de34 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Sun, 4 Aug 2024 18:03:21 +0800 Subject: [PATCH 59/61] [GLUTEN-6588][CH] Cast columns if necessary before finally writing to ORC/Parquet files during native inserting (#6691) * [GLUTEN-6588][CH] Cast columns if necessary before finally writing to ORC/Parquet files during native inserting * fix style * fix style * fix conflicts and remove examples * fix style * fix style --- .../datasources/CHDatasourceJniWrapper.java | 2 +- .../v1/CHFormatWriterInjects.scala | 14 +++++- ...lutenClickHouseNativeWriteTableSuite.scala | 10 ++--- cpp-ch/local-engine/Common/CHUtil.cpp | 5 ++- cpp-ch/local-engine/Common/CHUtil.h | 3 +- .../Storages/Output/FileWriterWrappers.cpp | 28 +++++++++--- .../Storages/Output/FileWriterWrappers.h | 12 +++--- .../Storages/Output/ORCOutputFormatFile.cpp | 6 +-- .../Storages/Output/ORCOutputFormatFile.h | 4 +- .../Storages/Output/OutputFormatFile.cpp | 43 +++++++++---------- .../Storages/Output/OutputFormatFile.h | 10 +++-- .../Output/ParquetOutputFormatFile.cpp | 8 ++-- .../Storages/Output/ParquetOutputFormatFile.h | 3 +- .../Storages/SubstraitSource/FormatFile.h | 9 +--- .../SubstraitSource/ORCFormatFile.cpp | 1 + cpp-ch/local-engine/local_engine_jni.cpp | 32 +++++++++----- 16 files changed, 107 insertions(+), 83 deletions(-) diff --git a/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/CHDatasourceJniWrapper.java b/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/CHDatasourceJniWrapper.java index 2bb3d44e0ff16..f19c5d39df1d8 100644 --- a/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/CHDatasourceJniWrapper.java +++ b/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/CHDatasourceJniWrapper.java @@ -19,7 +19,7 @@ public class CHDatasourceJniWrapper { public native long nativeInitFileWriterWrapper( - String filePath, String[] preferredColumnNames, String formatHint); + String filePath, byte[] preferredSchema, String formatHint); public native long nativeInitMergeTreeWriterWrapper( byte[] plan, diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHFormatWriterInjects.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHFormatWriterInjects.scala index 06d5b152716d3..547904d7e0373 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHFormatWriterInjects.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHFormatWriterInjects.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.datasources.v1 import org.apache.gluten.execution.datasource.GlutenRowSplitter +import org.apache.gluten.expression.ConverterUtils import org.apache.gluten.memory.CHThreadGroup import org.apache.gluten.vectorized.CHColumnVector @@ -26,6 +27,7 @@ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.orc.OrcUtils import org.apache.spark.sql.types.StructType +import io.substrait.proto.{NamedStruct, Type} import org.apache.hadoop.fs.FileStatus import org.apache.hadoop.mapreduce.TaskAttemptContext @@ -39,10 +41,20 @@ trait CHFormatWriterInjects extends GlutenFormatWriterInjectsBase { val originPath = path val datasourceJniWrapper = new CHDatasourceJniWrapper(); CHThreadGroup.registerNewThreadGroup() + + val namedStructBuilder = NamedStruct.newBuilder + val structBuilder = Type.Struct.newBuilder + for (field <- dataSchema.fields) { + namedStructBuilder.addNames(field.name) + structBuilder.addTypes(ConverterUtils.getTypeNode(field.dataType, field.nullable).toProtobuf) + } + namedStructBuilder.setStruct(structBuilder.build) + var namedStruct = namedStructBuilder.build + val instance = datasourceJniWrapper.nativeInitFileWriterWrapper( path, - dataSchema.fieldNames, + namedStruct.toByteArray, getFormatName()); new OutputWriter { diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeWriteTableSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeWriteTableSuite.scala index 2fec68a492163..1f99947e5b96f 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeWriteTableSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseNativeWriteTableSuite.scala @@ -913,12 +913,10 @@ class GlutenClickHouseNativeWriteTableSuite (table_name, create_sql, insert_sql) }, (table_name, _) => - if (isSparkVersionGE("3.5")) { - compareResultsAgainstVanillaSpark( - s"select * from $table_name", - compareResult = true, - _ => {}) - } + compareResultsAgainstVanillaSpark( + s"select * from $table_name", + compareResult = true, + _ => {}) ) } } diff --git a/cpp-ch/local-engine/Common/CHUtil.cpp b/cpp-ch/local-engine/Common/CHUtil.cpp index 12bf7ed599392..d32eed92340a4 100644 --- a/cpp-ch/local-engine/Common/CHUtil.cpp +++ b/cpp-ch/local-engine/Common/CHUtil.cpp @@ -16,11 +16,12 @@ */ #include "CHUtil.h" + #include -#include #include #include #include + #include #include #include @@ -1009,7 +1010,7 @@ void BackendInitializerUtil::init(const std::string_view plan) }); } -void BackendInitializerUtil::updateConfig(const DB::ContextMutablePtr & context, const std::string_view plan) +void BackendInitializerUtil::updateConfig(const DB::ContextMutablePtr & context, std::string_view plan) { std::map backend_conf_map = getBackendConfMap(plan); diff --git a/cpp-ch/local-engine/Common/CHUtil.h b/cpp-ch/local-engine/Common/CHUtil.h index f528128033357..785d5d6c00565 100644 --- a/cpp-ch/local-engine/Common/CHUtil.h +++ b/cpp-ch/local-engine/Common/CHUtil.h @@ -161,8 +161,7 @@ class BackendInitializerUtil /// 1. global level resources like global_context/shared_context, notice that they can only be initialized once in process lifetime /// 2. session level resources like settings/configs, they can be initialized multiple times following the lifetime of executor/driver static void init(const std::string_view plan); - static void updateConfig(const DB::ContextMutablePtr &, const std::string_view); - + static void updateConfig(const DB::ContextMutablePtr &, std::string_view); // use excel text parser inline static const std::string USE_EXCEL_PARSER = "use_excel_serialization"; diff --git a/cpp-ch/local-engine/Storages/Output/FileWriterWrappers.cpp b/cpp-ch/local-engine/Storages/Output/FileWriterWrappers.cpp index fc4b3a72f75b8..46edb7f30d5b8 100644 --- a/cpp-ch/local-engine/Storages/Output/FileWriterWrappers.cpp +++ b/cpp-ch/local-engine/Storages/Output/FileWriterWrappers.cpp @@ -15,7 +15,6 @@ * limitations under the License. */ #include "FileWriterWrappers.h" -#include namespace local_engine { @@ -28,7 +27,6 @@ NormalFileWriter::NormalFileWriter(const OutputFormatFilePtr & file_, const DB:: { } - void NormalFileWriter::consume(DB::Block & block) { if (!writer) [[unlikely]] @@ -39,6 +37,22 @@ void NormalFileWriter::consume(DB::Block & block) writer = std::make_unique(*pipeline); } + /// In case input block didn't have the same types as the preferred schema, we cast the input block to the preferred schema. + /// Notice that preferred_schema is the actual file schema, which is also the data schema of current inserted table. + /// Refer to issue: https://github.com/apache/incubator-gluten/issues/6588 + size_t index = 0; + const auto & preferred_schema = file->getPreferredSchema(); + for (auto & column : block) + { + if (column.name.starts_with("__bucket_value__")) + continue; + + const auto & preferred_column = preferred_schema.getByPosition(index++); + column.column = DB::castColumn(column, preferred_column.type); + column.name = preferred_column.name; + column.type = preferred_column.type; + } + /// Although gluten will append MaterializingTransform to the end of the pipeline before native insert in most cases, there are some cases in which MaterializingTransform won't be appended. /// e.g. https://github.com/oap-project/gluten/issues/2900 /// So we need to do materialize here again to make sure all blocks passed to native writer are all materialized. @@ -54,8 +68,8 @@ void NormalFileWriter::close() writer->finish(); } -OutputFormatFilePtr create_output_format_file( - const DB::ContextPtr & context, const std::string & file_uri, const DB::Names & preferred_column_names, const std::string & format_hint) +OutputFormatFilePtr createOutputFormatFile( + const DB::ContextPtr & context, const std::string & file_uri, const DB::Block & preferred_schema, const std::string & format_hint) { // the passed in file_uri is exactly what is expected to see in the output folder // e.g /xxx/中文/timestamp_field=2023-07-13 03%3A00%3A17.622/abc.parquet @@ -64,13 +78,13 @@ OutputFormatFilePtr create_output_format_file( Poco::URI::encode(file_uri, "", encoded); // encode the space and % seen in the file_uri Poco::URI poco_uri(encoded); auto write_buffer_builder = WriteBufferBuilderFactory::instance().createBuilder(poco_uri.getScheme(), context); - return OutputFormatFileUtil::createFile(context, write_buffer_builder, encoded, preferred_column_names, format_hint); + return OutputFormatFileUtil::createFile(context, write_buffer_builder, encoded, preferred_schema, format_hint); } std::unique_ptr createFileWriterWrapper( - const DB::ContextPtr & context, const std::string & file_uri, const DB::Names & preferred_column_names, const std::string & format_hint) + const DB::ContextPtr & context, const std::string & file_uri, const DB::Block & preferred_schema, const std::string & format_hint) { - return std::make_unique(create_output_format_file(context, file_uri, preferred_column_names, format_hint), context); + return std::make_unique(createOutputFormatFile(context, file_uri, preferred_schema, format_hint), context); } } diff --git a/cpp-ch/local-engine/Storages/Output/FileWriterWrappers.h b/cpp-ch/local-engine/Storages/Output/FileWriterWrappers.h index 57cb47e41a559..736f5a95f6bd1 100644 --- a/cpp-ch/local-engine/Storages/Output/FileWriterWrappers.h +++ b/cpp-ch/local-engine/Storages/Output/FileWriterWrappers.h @@ -41,6 +41,7 @@ class FileWriterWrapper public: explicit FileWriterWrapper(const OutputFormatFilePtr & file_) : file(file_) { } virtual ~FileWriterWrapper() = default; + virtual void consume(DB::Block & block) = 0; virtual void close() = 0; @@ -53,10 +54,9 @@ using FileWriterWrapperPtr = std::shared_ptr; class NormalFileWriter : public FileWriterWrapper { public: - //TODO: EmptyFileReader and ConstColumnsFileReader ? - //TODO: to support complex types NormalFileWriter(const OutputFormatFilePtr & file_, const DB::ContextPtr & context_); ~NormalFileWriter() override = default; + void consume(DB::Block & block) override; void close() override; @@ -71,13 +71,13 @@ class NormalFileWriter : public FileWriterWrapper std::unique_ptr createFileWriterWrapper( const DB::ContextPtr & context, const std::string & file_uri, - const DB::Names & preferred_column_names, + const DB::Block & preferred_schema, const std::string & format_hint); -OutputFormatFilePtr create_output_format_file( +OutputFormatFilePtr createOutputFormatFile( const DB::ContextPtr & context, const std::string & file_uri, - const DB::Names & preferred_column_names, + const DB::Block & preferred_schema, const std::string & format_hint); class WriteStats : public DB::ISimpleTransform @@ -191,7 +191,7 @@ class SubstraitFileSink final : public SinkToStorage : SinkToStorage(header) , partition_id_(partition_id.empty() ? NO_PARTITION_ID : partition_id) , relative_path_(relative) - , output_format_(create_output_format_file(context, makeFilename(base_path, partition_id, relative), header.getNames(), format_hint) + , output_format_(createOutputFormatFile(context, makeFilename(base_path, partition_id, relative), header, format_hint) ->createOutputFormat(header)) { } diff --git a/cpp-ch/local-engine/Storages/Output/ORCOutputFormatFile.cpp b/cpp-ch/local-engine/Storages/Output/ORCOutputFormatFile.cpp index 007325a515ae7..c54f2e7b33bf2 100644 --- a/cpp-ch/local-engine/Storages/Output/ORCOutputFormatFile.cpp +++ b/cpp-ch/local-engine/Storages/Output/ORCOutputFormatFile.cpp @@ -27,8 +27,8 @@ ORCOutputFormatFile::ORCOutputFormatFile( DB::ContextPtr context_, const std::string & file_uri_, WriteBufferBuilderPtr write_buffer_builder_, - const std::vector & preferred_column_names_) - : OutputFormatFile(context_, file_uri_, write_buffer_builder_, preferred_column_names_) + const DB::Block & preferred_schema_) + : OutputFormatFile(context_, file_uri_, write_buffer_builder_, preferred_schema_) { } @@ -37,7 +37,7 @@ OutputFormatFile::OutputFormatPtr ORCOutputFormatFile::createOutputFormat(const auto res = std::make_shared(); res->write_buffer = write_buffer_builder->build(file_uri); - auto new_header = creatHeaderWithPreferredColumnNames(header); + auto new_header = creatHeaderWithPreferredSchema(header); // TODO: align all spark orc config with ch orc config auto format_settings = DB::getFormatSettings(context); auto output_format = std::make_shared(*(res->write_buffer), new_header, format_settings); diff --git a/cpp-ch/local-engine/Storages/Output/ORCOutputFormatFile.h b/cpp-ch/local-engine/Storages/Output/ORCOutputFormatFile.h index 0654f4ebcfdbf..2ea197cddaa0d 100644 --- a/cpp-ch/local-engine/Storages/Output/ORCOutputFormatFile.h +++ b/cpp-ch/local-engine/Storages/Output/ORCOutputFormatFile.h @@ -20,8 +20,6 @@ #include "config.h" #if USE_ORC - -# include # include # include @@ -34,7 +32,7 @@ class ORCOutputFormatFile : public OutputFormatFile DB::ContextPtr context_, const std::string & file_uri_, WriteBufferBuilderPtr write_buffer_builder_, - const std::vector & preferred_column_names_); + const DB::Block & preferred_schema_); ~ORCOutputFormatFile() override = default; OutputFormatFile::OutputFormatPtr createOutputFormat(const DB::Block & header) override; diff --git a/cpp-ch/local-engine/Storages/Output/OutputFormatFile.cpp b/cpp-ch/local-engine/Storages/Output/OutputFormatFile.cpp index e785f78aae67b..1e8364c6dac2d 100644 --- a/cpp-ch/local-engine/Storages/Output/OutputFormatFile.cpp +++ b/cpp-ch/local-engine/Storages/Output/OutputFormatFile.cpp @@ -38,49 +38,48 @@ OutputFormatFile::OutputFormatFile( DB::ContextPtr context_, const std::string & file_uri_, WriteBufferBuilderPtr write_buffer_builder_, - const std::vector & preferred_column_names_) - : context(context_), file_uri(file_uri_), write_buffer_builder(write_buffer_builder_), preferred_column_names(preferred_column_names_) + const DB::Block & preferred_schema_) + : context(context_), file_uri(file_uri_), write_buffer_builder(write_buffer_builder_), preferred_schema(preferred_schema_) { } -Block OutputFormatFile::creatHeaderWithPreferredColumnNames(const Block & header) +Block OutputFormatFile::creatHeaderWithPreferredSchema(const Block & header) { - if (!preferred_column_names.empty()) + if (!preferred_schema) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "preferred_schema is empty"); + + /// Create a new header with the preferred column name and type + DB::ColumnsWithTypeAndName columns; + columns.reserve(preferred_schema.columns()); + size_t index = 0; + for (const auto & name_type : header.getNamesAndTypesList()) { - /// Create a new header with the preferred column name - DB::NamesAndTypesList names_types_list = header.getNamesAndTypesList(); - DB::ColumnsWithTypeAndName cols; - size_t index = 0; - for (const auto & name_type : header.getNamesAndTypesList()) - { - if (name_type.name.starts_with("__bucket_value__")) - continue; + if (name_type.name.starts_with("__bucket_value__")) + continue; - DB::ColumnWithTypeAndName col(name_type.type->createColumn(), name_type.type, preferred_column_names.at(index++)); - cols.emplace_back(std::move(col)); - } - assert(preferred_column_names.size() == index); - return {std::move(cols)}; + const auto & preferred_column = preferred_schema.getByPosition(index++); + ColumnWithTypeAndName column(preferred_column.type->createColumn(), preferred_column.type, preferred_column.name); + columns.emplace_back(std::move(column)); } - else - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "preferred_column_names is empty"); + assert(preferred_column_names.size() == index); + return {std::move(columns)}; } OutputFormatFilePtr OutputFormatFileUtil::createFile( DB::ContextPtr context, local_engine::WriteBufferBuilderPtr write_buffer_builder, const std::string & file_uri, - const std::vector & preferred_column_names, + const DB::Block & preferred_schema, const std::string & format_hint) { #if USE_PARQUET if (boost::to_lower_copy(file_uri).ends_with(".parquet") || "parquet" == boost::to_lower_copy(format_hint)) - return std::make_shared(context, file_uri, write_buffer_builder, preferred_column_names); + return std::make_shared(context, file_uri, write_buffer_builder, preferred_schema); #endif #if USE_ORC if (boost::to_lower_copy(file_uri).ends_with(".orc") || "orc" == boost::to_lower_copy(format_hint)) - return std::make_shared(context, file_uri, write_buffer_builder, preferred_column_names); + return std::make_shared(context, file_uri, write_buffer_builder, preferred_schema); #endif diff --git a/cpp-ch/local-engine/Storages/Output/OutputFormatFile.h b/cpp-ch/local-engine/Storages/Output/OutputFormatFile.h index 54c6ba4cdc04d..93c26d7d188bc 100644 --- a/cpp-ch/local-engine/Storages/Output/OutputFormatFile.h +++ b/cpp-ch/local-engine/Storages/Output/OutputFormatFile.h @@ -43,19 +43,21 @@ class OutputFormatFile DB::ContextPtr context_, const std::string & file_uri_, WriteBufferBuilderPtr write_buffer_builder_, - const std::vector & preferred_column_names_); + const DB::Block & prefered_schema_); virtual ~OutputFormatFile() = default; virtual OutputFormatPtr createOutputFormat(const DB::Block & header_) = 0; + virtual const DB::Block getPreferredSchema() const { return preferred_schema; } + protected: - DB::Block creatHeaderWithPreferredColumnNames(const DB::Block & header); + DB::Block creatHeaderWithPreferredSchema(const DB::Block & header); DB::ContextPtr context; std::string file_uri; WriteBufferBuilderPtr write_buffer_builder; - std::vector preferred_column_names; + DB::Block preferred_schema; }; using OutputFormatFilePtr = std::shared_ptr; @@ -66,7 +68,7 @@ class OutputFormatFileUtil DB::ContextPtr context, WriteBufferBuilderPtr write_buffer_builder_, const std::string & file_uri_, - const std::vector & preferred_column_names, + const DB::Block & prefered_schema_, const std::string & format_hint = ""); }; } diff --git a/cpp-ch/local-engine/Storages/Output/ParquetOutputFormatFile.cpp b/cpp-ch/local-engine/Storages/Output/ParquetOutputFormatFile.cpp index 6ef8b45246755..ea173b03cba57 100644 --- a/cpp-ch/local-engine/Storages/Output/ParquetOutputFormatFile.cpp +++ b/cpp-ch/local-engine/Storages/Output/ParquetOutputFormatFile.cpp @@ -17,10 +17,8 @@ #include "ParquetOutputFormatFile.h" #if USE_PARQUET - # include # include -# include # include # include @@ -35,8 +33,8 @@ ParquetOutputFormatFile::ParquetOutputFormatFile( DB::ContextPtr context_, const std::string & file_uri_, const WriteBufferBuilderPtr & write_buffer_builder_, - const std::vector & preferred_column_names_) - : OutputFormatFile(context_, file_uri_, write_buffer_builder_, preferred_column_names_) + const DB::Block & preferred_schema_) + : OutputFormatFile(context_, file_uri_, write_buffer_builder_, preferred_schema_) { } @@ -45,7 +43,7 @@ OutputFormatFile::OutputFormatPtr ParquetOutputFormatFile::createOutputFormat(co auto res = std::make_shared(); res->write_buffer = write_buffer_builder->build(file_uri); - auto new_header = creatHeaderWithPreferredColumnNames(header); + auto new_header = creatHeaderWithPreferredSchema(header); // TODO: align all spark parquet config with ch parquet config auto format_settings = DB::getFormatSettings(context); auto output_format = std::make_shared(*(res->write_buffer), new_header, format_settings); diff --git a/cpp-ch/local-engine/Storages/Output/ParquetOutputFormatFile.h b/cpp-ch/local-engine/Storages/Output/ParquetOutputFormatFile.h index 13b731600938c..cc87da7da8542 100644 --- a/cpp-ch/local-engine/Storages/Output/ParquetOutputFormatFile.h +++ b/cpp-ch/local-engine/Storages/Output/ParquetOutputFormatFile.h @@ -20,7 +20,6 @@ #if USE_PARQUET -#include #include #include @@ -33,7 +32,7 @@ class ParquetOutputFormatFile : public OutputFormatFile DB::ContextPtr context_, const std::string & file_uri_, const WriteBufferBuilderPtr & write_buffer_builder_, - const std::vector & preferred_column_names_); + const DB::Block & preferred_schema_); ~ParquetOutputFormatFile() override = default; OutputFormatFile::OutputFormatPtr createOutputFormat(const DB::Block & header) override; diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.h b/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.h index e67259ce59cc4..8ab82f312f287 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.h +++ b/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.h @@ -19,21 +19,14 @@ #include #include #include - +#include #include #include - #include - #include - #include - #include -#include - - namespace DB { diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/ORCFormatFile.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/ORCFormatFile.cpp index d213342f6d76b..1c57010751c0f 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/ORCFormatFile.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/ORCFormatFile.cpp @@ -67,6 +67,7 @@ FormatFile::InputFormatPtr ORCFormatFile::createInputFormat(const DB::Block & he std::back_inserter(skip_stripe_indices)); format_settings.orc.skip_stripes = std::unordered_set(skip_stripe_indices.begin(), skip_stripe_indices.end()); + auto input_format = std::make_shared(*file_format->read_buffer, header, format_settings); file_format->input = input_format; return file_format; diff --git a/cpp-ch/local-engine/local_engine_jni.cpp b/cpp-ch/local-engine/local_engine_jni.cpp index 8807a0f639911..c4e8ec67b106a 100644 --- a/cpp-ch/local-engine/local_engine_jni.cpp +++ b/cpp-ch/local-engine/local_engine_jni.cpp @@ -14,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#include #include #include #include @@ -55,6 +54,7 @@ #include #include #include +#include #include @@ -826,23 +826,33 @@ JNIEXPORT void Java_org_apache_gluten_vectorized_CHBlockWriterJniWrapper_nativeC } JNIEXPORT jlong Java_org_apache_spark_sql_execution_datasources_CHDatasourceJniWrapper_nativeInitFileWriterWrapper( - JNIEnv * env, jobject, jstring file_uri_, jobjectArray names_, jstring format_hint_) + JNIEnv * env, jobject, jstring file_uri_, jbyteArray preferred_schema_, jstring format_hint_) { LOCAL_ENGINE_JNI_METHOD_START - const int num_columns = env->GetArrayLength(names_); - DB::Names names; - names.reserve(num_columns); - for (int i = 0; i < num_columns; i++) + + const auto preferred_schema_ref = local_engine::getByteArrayElementsSafe(env, preferred_schema_); + auto parse_named_struct = [&]() -> std::optional { - auto * name = static_cast(env->GetObjectArrayElement(names_, i)); - names.emplace_back(jstring2string(env, name)); - env->DeleteLocalRef(name); - } + std::string_view view{ + reinterpret_cast(preferred_schema_ref.elems()), static_cast(preferred_schema_ref.length())}; + + substrait::NamedStruct res; + bool ok = res.ParseFromString(view); + if (!ok) + return {}; + return std::move(res); + }; + + auto named_struct = parse_named_struct(); + if (!named_struct.has_value()) + throw DB::Exception(DB::ErrorCodes::CANNOT_PARSE_PROTOBUF_SCHEMA, "Parse schema from substrait protobuf failed"); + + DB::Block preferred_schema = local_engine::TypeParser::buildBlockFromNamedStructWithoutDFS(*named_struct); const auto file_uri = jstring2string(env, file_uri_); // for HiveFileFormat, the file url may not end with .parquet, so we pass in the format as a hint const auto format_hint = jstring2string(env, format_hint_); const auto context = local_engine::QueryContextManager::instance().currentQueryContext(); - auto * writer = local_engine::createFileWriterWrapper(context, file_uri, names, format_hint).release(); + auto * writer = local_engine::createFileWriterWrapper(context, file_uri, preferred_schema, format_hint).release(); return reinterpret_cast(writer); LOCAL_ENGINE_JNI_METHOD_END(env, 0) } From 01a44d9023dc9954739adab52e2ce0c54e38d00b Mon Sep 17 00:00:00 2001 From: Jin Chengcheng Date: Mon, 5 Aug 2024 09:02:13 +0800 Subject: [PATCH 60/61] [VL] Remove redundant hash function in substrait function validation (#6690) --- .../substrait/SubstraitToVeloxPlanValidator.cc | 14 ++------------ 1 file changed, 2 insertions(+), 12 deletions(-) diff --git a/cpp/velox/substrait/SubstraitToVeloxPlanValidator.cc b/cpp/velox/substrait/SubstraitToVeloxPlanValidator.cc index c18d265986f8c..7bb0eab77758d 100644 --- a/cpp/velox/substrait/SubstraitToVeloxPlanValidator.cc +++ b/cpp/velox/substrait/SubstraitToVeloxPlanValidator.cc @@ -68,7 +68,8 @@ static const std::unordered_set kBlackList = { "trunc", "sequence", "approx_percentile", - "get_array_struct_fields"}; + "get_array_struct_fields", + "map_from_arrays"}; } // namespace @@ -189,9 +190,6 @@ bool SubstraitToVeloxPlanValidator::validateScalarFunction( return validateRound(scalarFunction, inputType); } else if (name == "extract") { return validateExtractExpr(params); - } else if (name == "map_from_arrays") { - LOG_VALIDATION_MSG("map_from_arrays is not supported."); - return false; } else if (name == "concat") { for (const auto& type : types) { if (type.find("struct") != std::string::npos || type.find("map") != std::string::npos || @@ -200,14 +198,6 @@ bool SubstraitToVeloxPlanValidator::validateScalarFunction( return false; } } - } else if (name == "murmur3hash") { - for (const auto& type : types) { - if (type.find("struct") != std::string::npos || type.find("map") != std::string::npos || - type.find("list") != std::string::npos) { - LOG_VALIDATION_MSG(type + " is not supported in murmur3hash."); - return false; - } - } } // Validate regex functions. From 809fef6aae21139d0ed690fed72662383ef52235 Mon Sep 17 00:00:00 2001 From: Nicholas Jiang Date: Mon, 5 Aug 2024 09:26:24 +0800 Subject: [PATCH 61/61] [GLUTEN-6656][UNIFFLE] VeloxUniffleColumnarShuffleWriter should send commit for all ColumnBatch with empty rows (#6698) --- .../VeloxUniffleColumnarShuffleWriter.java | 21 ++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/gluten-uniffle/velox/src/main/java/org/apache/spark/shuffle/writer/VeloxUniffleColumnarShuffleWriter.java b/gluten-uniffle/velox/src/main/java/org/apache/spark/shuffle/writer/VeloxUniffleColumnarShuffleWriter.java index a80e34fb1d995..d2032fa48564a 100644 --- a/gluten-uniffle/velox/src/main/java/org/apache/spark/shuffle/writer/VeloxUniffleColumnarShuffleWriter.java +++ b/gluten-uniffle/velox/src/main/java/org/apache/spark/shuffle/writer/VeloxUniffleColumnarShuffleWriter.java @@ -126,8 +126,8 @@ public VeloxUniffleColumnarShuffleWriter( @Override protected void writeImpl(Iterator> records) { - if (!records.hasNext() && !isMemoryShuffleEnabled) { - super.sendCommit(); + if (!records.hasNext()) { + sendCommit(); return; } // writer already init @@ -189,11 +189,13 @@ public long spill(MemoryTarget self, Spiller.Phase phase, long size) { } } - long startTime = System.nanoTime(); LOG.info("nativeShuffleWriter value {}", nativeShuffleWriter); + // If all of the ColumnarBatch have empty rows, the nativeShuffleWriter still equals -1 if (nativeShuffleWriter == -1L) { - throw new IllegalStateException("nativeShuffleWriter should not be -1L"); + sendCommit(); + return; } + long startTime = System.nanoTime(); SplitResult splitResult; try { splitResult = jniWrapper.stop(nativeShuffleWriter); @@ -219,9 +221,7 @@ public long spill(MemoryTarget self, Spiller.Phase phase, long size) { long pushMergedDataTime = System.nanoTime(); // clear all sendRestBlockAndWait(); - if (!isMemoryShuffleEnabled) { - super.sendCommit(); - } + sendCommit(); long writeDurationMs = System.nanoTime() - pushMergedDataTime; shuffleWriteMetrics.incWriteTime(writeDurationMs); LOG.info( @@ -229,6 +229,13 @@ public long spill(MemoryTarget self, Spiller.Phase phase, long size) { TimeUnit.MILLISECONDS.toNanos(writeDurationMs)); } + @Override + protected void sendCommit() { + if (!isMemoryShuffleEnabled) { + super.sendCommit(); + } + } + @Override public Option stop(boolean success) { if (!stopping) {