diff --git a/.github/workflows/velox_be.yml b/.github/workflows/velox_be.yml index bd16547f51636..85abed9018083 100644 --- a/.github/workflows/velox_be.yml +++ b/.github/workflows/velox_be.yml @@ -17,25 +17,6 @@ name: Velox backend on: pull_request: - paths: - - '.github/**' - - 'pom.xml' - - 'backends-velox/**' - - 'gluten-celeborn/**' - - 'gluten-core/**' - - 'gluten-data/**' - - 'gluten-ut/**' - - 'shims/**' - - 'tools/gluten-it/**' - - 'tools/gluten-te/**' - - 'ep/build-arrow/**' - - 'ep/build-velox/**' - - 'cpp/*' - - 'cpp/CMake/**' - - 'cpp/velox/**' - - 'cpp/core/**' - - 'dev**' -# - 'substrait/substrait-spark/**' concurrency: diff --git a/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHTransformerApi.scala b/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHTransformerApi.scala index 204a6729bc7c2..b4ee3ff62cf79 100644 --- a/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHTransformerApi.scala +++ b/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHTransformerApi.scala @@ -84,18 +84,30 @@ class CHTransformerApi extends TransformerApi with Logging { relation: HadoopFsRelation, selectedPartitions: Array[PartitionDirectory], output: Seq[Attribute], + bucketedScan: Boolean, optionalBucketSet: Option[BitSet], optionalNumCoalescedBuckets: Option[Int], disableBucketedScan: Boolean): Seq[InputPartition] = { if (relation.location.isInstanceOf[ClickHouseFileIndex]) { // Generate NativeMergeTreePartition for MergeTree - relation.location.asInstanceOf[ClickHouseFileIndex].partsPartitions + relation.location + .asInstanceOf[ClickHouseFileIndex] + .partsPartitions( + relation, + selectedPartitions, + output, + bucketedScan, + optionalBucketSet, + optionalNumCoalescedBuckets, + disableBucketedScan + ) } else { // Generate FilePartition for Parquet CHInputPartitionsUtil( relation, selectedPartitions, output, + bucketedScan, optionalBucketSet, optionalNumCoalescedBuckets, disableBucketedScan).genInputPartitionSeq() diff --git a/backends-clickhouse/src/main/scala/io/glutenproject/metrics/BatchScanMetricsUpdater.scala b/backends-clickhouse/src/main/scala/io/glutenproject/metrics/BatchScanMetricsUpdater.scala index f3d1bfd53bdd4..5cd44a508a6c6 100644 --- a/backends-clickhouse/src/main/scala/io/glutenproject/metrics/BatchScanMetricsUpdater.scala +++ b/backends-clickhouse/src/main/scala/io/glutenproject/metrics/BatchScanMetricsUpdater.scala @@ -63,6 +63,7 @@ class BatchScanMetricsUpdater(@transient val metrics: Map[String, SQLMetric]) } object BatchScanMetricsUpdater { - val INCLUDING_PROCESSORS = Array("MergeTreeInOrder", "SubstraitFileSource") - val CH_PLAN_NODE_NAME = Array("MergeTreeInOrder", "SubstraitFileSource") + // in mergetree format, the processor name is `MergeTreeSelect(pool: XXX, algorithm: XXX)` + val INCLUDING_PROCESSORS = Array("MergeTreeSelect(pool", "SubstraitFileSource") + val CH_PLAN_NODE_NAME = Array("MergeTreeSelect(pool", "SubstraitFileSource") } diff --git a/backends-clickhouse/src/main/scala/io/glutenproject/metrics/FileSourceScanMetricsUpdater.scala b/backends-clickhouse/src/main/scala/io/glutenproject/metrics/FileSourceScanMetricsUpdater.scala index ad32526272a20..7985efbf0ee6a 100644 --- a/backends-clickhouse/src/main/scala/io/glutenproject/metrics/FileSourceScanMetricsUpdater.scala +++ b/backends-clickhouse/src/main/scala/io/glutenproject/metrics/FileSourceScanMetricsUpdater.scala @@ -67,6 +67,7 @@ class FileSourceScanMetricsUpdater(@transient val metrics: Map[String, SQLMetric } object FileSourceScanMetricsUpdater { - val INCLUDING_PROCESSORS = Array("MergeTreeInOrder", "SubstraitFileSource") - val CH_PLAN_NODE_NAME = Array("MergeTreeInOrder", "SubstraitFileSource") + // in mergetree format, the processor name is `MergeTreeSelect(pool: XXX, algorithm: XXX)` + val INCLUDING_PROCESSORS = Array("MergeTreeSelect(pool", "SubstraitFileSource") + val CH_PLAN_NODE_NAME = Array("MergeTreeSelect(pool", "SubstraitFileSource") } diff --git a/backends-clickhouse/src/main/scala/io/glutenproject/metrics/MetricsUtil.scala b/backends-clickhouse/src/main/scala/io/glutenproject/metrics/MetricsUtil.scala index 6c2af3eeaa5c4..425fd126bfd3e 100644 --- a/backends-clickhouse/src/main/scala/io/glutenproject/metrics/MetricsUtil.scala +++ b/backends-clickhouse/src/main/scala/io/glutenproject/metrics/MetricsUtil.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.metric.SQLMetric import java.lang.{Long => JLong} -import java.util.{ArrayList => JArrayList, List => JList, Map => JMap} +import java.util.{ArrayList => JArrayList, Collections => JCollections, List => JList, Map => JMap} import scala.collection.JavaConverters._ @@ -143,11 +143,12 @@ object MetricsUtil extends Logging { relMap .get(operatorIdx) .forEach( - _ => { - nodeMetricsList.add(metrics.metricsDataList.get(curMetricsIdx)) + idx => { + nodeMetricsList.add(metrics.metricsDataList.get(idx.toInt)) curMetricsIdx -= 1 }) + JCollections.reverse(nodeMetricsList) val operatorMetrics = new OperatorMetrics( nodeMetricsList, joinParamsMap.getOrDefault(operatorIdx, null), @@ -195,10 +196,10 @@ object MetricsUtil extends Logging { val processors = MetricsUtil.getAllProcessorList(metricData) processors.foreach( processor => { - if (!includingMetrics.contains(processor.name)) { + if (!includingMetrics.exists(processor.name.startsWith(_))) { extraTime += (processor.time / 1000L).toLong } - if (planNodeNames.contains(processor.name)) { + if (planNodeNames.exists(processor.name.startsWith(_))) { outputRows += processor.outputRows outputBytes += processor.outputBytes inputRows += processor.inputRows diff --git a/backends-clickhouse/src/main/scala/io/glutenproject/utils/CHInputPartitionsUtil.scala b/backends-clickhouse/src/main/scala/io/glutenproject/utils/CHInputPartitionsUtil.scala index 96682fa95721a..b4f73d82f5608 100644 --- a/backends-clickhouse/src/main/scala/io/glutenproject/utils/CHInputPartitionsUtil.scala +++ b/backends-clickhouse/src/main/scala/io/glutenproject/utils/CHInputPartitionsUtil.scala @@ -33,24 +33,12 @@ case class CHInputPartitionsUtil( relation: HadoopFsRelation, selectedPartitions: Array[PartitionDirectory], output: Seq[Attribute], + bucketedScan: Boolean, optionalBucketSet: Option[BitSet], optionalNumCoalescedBuckets: Option[Int], disableBucketedScan: Boolean) extends Logging { - private val bucketedScan: Boolean = { - if ( - relation.sparkSession.sessionState.conf.bucketingEnabled && relation.bucketSpec.isDefined - && !disableBucketedScan - ) { - val spec = relation.bucketSpec.get - val bucketColumns = spec.bucketColumnNames.flatMap(n => toAttribute(n)) - bucketColumns.size == spec.bucketColumnNames.size - } else { - false - } - } - def genInputPartitionSeq(): Seq[InputPartition] = { if (bucketedScan) { genBucketedInputPartitionSeq() 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 49ad39b7fd975..a70ea9bbd4219 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 @@ -21,9 +21,14 @@ import io.glutenproject.execution.GlutenMergeTreePartition import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.read.InputPartition +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, PartitionDirectory} +import org.apache.spark.sql.execution.datasources.v2.clickhouse.metadata.AddMergeTreeParts import org.apache.spark.sql.execution.datasources.v2.clickhouse.table.ClickHouseTableV2 import org.apache.spark.util.SparkResourceUtil +import org.apache.spark.util.collection.BitSet import scala.collection.mutable.ArrayBuffer @@ -116,4 +121,275 @@ object MergeTreePartsPartitionsUtil extends Logging { closePartition() partitions } + + def getMergeTreePartsPartitions( + relation: HadoopFsRelation, + selectedPartitions: Array[PartitionDirectory], + output: Seq[Attribute], + bucketedScan: Boolean, + sparkSession: SparkSession, + table: ClickHouseTableV2, + optionalBucketSet: Option[BitSet], + optionalNumCoalescedBuckets: Option[Int], + disableBucketedScan: Boolean): Seq[InputPartition] = { + val partsFiles = table.listFiles() + + val partitions = new ArrayBuffer[InputPartition] + val (database, tableName) = if (table.catalogTable.isDefined) { + (table.catalogTable.get.identifier.database.get, table.catalogTable.get.identifier.table) + } else { + // for file_format.`file_path` + ("default", "file_format") + } + val engine = table.snapshot.metadata.configuration.get("engine").get + // TODO: remove `substring` + val tablePath = table.deltaLog.dataPath.toString.substring(6) + + // bucket table + if (table.bucketOption.isDefined) { + if (bucketedScan) { + genBucketedInputPartitionSeq( + engine, + database, + tableName, + tablePath, + table.bucketOption.get, + partsFiles, + partitions, + optionalBucketSet, + optionalNumCoalescedBuckets, + sparkSession + ) + } else { + genInputPartitionSeqWithBucketTable( + engine, + database, + tableName, + tablePath, + table.bucketOption.get, + partsFiles, + partitions, + optionalBucketSet, + sparkSession) + } + } else { + genNonBuckedInputPartitionSeq( + engine, + database, + tableName, + tablePath, + partsFiles, + partitions, + sparkSession + ) + } + partitions + } + + /** Generate bucket partition */ + def genBucketedInputPartitionSeq( + engine: String, + database: String, + tableName: String, + tablePath: String, + bucketSpec: BucketSpec, + partsFiles: Seq[AddMergeTreeParts], + partitions: ArrayBuffer[InputPartition], + optionalBucketSet: Option[BitSet], + optionalNumCoalescedBuckets: Option[Int], + sparkSession: SparkSession): Unit = { + val bucketGroupParts = partsFiles.groupBy(p => Integer.parseInt(p.bucketNum)) + + val prunedFilesGroupedToBuckets = if (optionalBucketSet.isDefined) { + val bucketSet = optionalBucketSet.get + bucketGroupParts.filter(f => bucketSet.get(f._1)) + } else { + bucketGroupParts + } + + if (optionalNumCoalescedBuckets.isDefined) { + throw new UnsupportedOperationException( + "Currently CH backend can't support coalesced buckets.") + } + Seq.tabulate(bucketSpec.numBuckets) { + bucketId => + val currBucketParts = prunedFilesGroupedToBuckets.getOrElse(bucketId, Seq.empty) + if (!currBucketParts.isEmpty) { + var currentMinPartsNum = Long.MaxValue + var currentMaxPartsNum = -1L + var currTableName = tableName + "_" + currBucketParts(0).bucketNum + var currTablePath = tablePath + "/" + currBucketParts(0).bucketNum + currBucketParts.foreach( + p => { + if (currentMinPartsNum >= p.minBlockNumber) currentMinPartsNum = p.minBlockNumber + if (currentMaxPartsNum <= p.maxBlockNumber) currentMaxPartsNum = p.maxBlockNumber + }) + if (currentMaxPartsNum >= currentMinPartsNum) { + val newPartition = GlutenMergeTreePartition( + bucketId, + engine, + database, + currTableName, + currTablePath, + currentMinPartsNum, + currentMaxPartsNum + 1) + partitions += newPartition + } + } + } + } + + /** Generate partition from the bucket table */ + def genInputPartitionSeqWithBucketTable( + engine: String, + database: String, + tableName: String, + tablePath: String, + bucketSpec: BucketSpec, + partsFiles: Seq[AddMergeTreeParts], + partitions: ArrayBuffer[InputPartition], + optionalBucketSet: Option[BitSet], + sparkSession: SparkSession): Unit = { + val bucketGroupParts = partsFiles.groupBy(p => Integer.parseInt(p.bucketNum)) + + val prunedFilesGroupedToBuckets = if (optionalBucketSet.isDefined) { + val bucketSet = optionalBucketSet.get + bucketGroupParts.filter(f => bucketSet.get(f._1)) + } else { + bucketGroupParts + } + + val openCostInBytes = sparkSession.sessionState.conf.filesOpenCostInBytes + val maxSplitBytes = sparkSession.sessionState.conf.filesMaxPartitionBytes + + def closePartition( + currTableName: String, + currTablePath: String, + currentMinPartsNum: Long, + currentMaxPartsNum: Long): Unit = { + if (currentMaxPartsNum >= currentMinPartsNum) { + val newPartition = GlutenMergeTreePartition( + partitions.size, + engine, + database, + currTableName, + currTablePath, + currentMinPartsNum, + currentMaxPartsNum + 1) + partitions += newPartition + } + } + + Seq.tabulate(bucketSpec.numBuckets) { + bucketId => + val currBucketParts = prunedFilesGroupedToBuckets.getOrElse(bucketId, Seq.empty) + if (!currBucketParts.isEmpty) { + var currentMinPartsNum = Long.MaxValue + var currentMaxPartsNum = -1L + var currentSize = 0L + var currTableName = tableName + "_" + currBucketParts(0).bucketNum + var currTablePath = tablePath + "/" + currBucketParts(0).bucketNum + + currBucketParts.foreach { + parts => + if (currentSize + parts.bytesOnDisk > maxSplitBytes) { + closePartition(currTableName, currTablePath, currentMinPartsNum, currentMaxPartsNum) + currentMinPartsNum = Long.MaxValue + currentMaxPartsNum = -1L + currentSize = 0L + } + // Add the given file to the current partition. + currentSize += parts.bytesOnDisk + openCostInBytes + if (currentMinPartsNum >= parts.minBlockNumber) { + currentMinPartsNum = parts.minBlockNumber + } + if (currentMaxPartsNum <= parts.maxBlockNumber) { + currentMaxPartsNum = parts.maxBlockNumber + } + } + closePartition(currTableName, currTablePath, currentMinPartsNum, currentMaxPartsNum) + } + } + } + + /** Generate partition from the non-bucket table */ + def genNonBuckedInputPartitionSeq( + engine: String, + database: String, + tableName: String, + tablePath: String, + partsFiles: Seq[AddMergeTreeParts], + partitions: ArrayBuffer[InputPartition], + sparkSession: SparkSession): Unit = { + var currentMinPartsNum = -1L + var currentMaxPartsNum = -1L + var currentSize = 0L + var currentFileCnt = 0L + + /** Close the current partition and move to the next. */ + def closePartition(): Unit = { + if (currentMinPartsNum > 0L && currentMaxPartsNum >= currentMinPartsNum) { + val newPartition = GlutenMergeTreePartition( + partitions.size, + engine, + database, + tableName, + tablePath, + currentMinPartsNum, + currentMaxPartsNum + 1) + partitions += newPartition + } + currentMinPartsNum = -1L + currentMaxPartsNum = -1L + currentSize = 0 + currentFileCnt = 0L + } + + val totalCores = SparkResourceUtil.getTotalCores(sparkSession.sessionState.conf) + val fileCntPerPartition = math.ceil((partsFiles.size * 1.0) / totalCores).toInt + val fileCntThreshold = sparkSession.sessionState.conf + .getConfString( + CHBackendSettings.GLUTEN_CLICKHOUSE_FILES_PER_PARTITION_THRESHOLD, + CHBackendSettings.GLUTEN_CLICKHOUSE_FILES_PER_PARTITION_THRESHOLD_DEFAULT + ) + .toInt + + if (fileCntThreshold > 0 && fileCntPerPartition > fileCntThreshold) { + // generate `Seq[InputPartition]` by file count + // Assign files to partitions using "Next Fit Decreasing" + partsFiles.foreach { + parts => + if (currentFileCnt >= fileCntPerPartition) { + closePartition() + } + // Add the given file to the current partition. + currentFileCnt += 1 + if (currentMinPartsNum == -1L) { + currentMinPartsNum = parts.minBlockNumber + } + currentMaxPartsNum = parts.maxBlockNumber + } + } else { + // generate `Seq[InputPartition]` by file size + val openCostInBytes = sparkSession.sessionState.conf.filesOpenCostInBytes + val maxSplitBytes = sparkSession.sessionState.conf.filesMaxPartitionBytes + logInfo( + s"Planning scan with bin packing, max size: $maxSplitBytes bytes, " + + s"open cost is considered as scanning $openCostInBytes bytes.") + // Assign files to partitions using "Next Fit Decreasing" + partsFiles.foreach { + parts => + if (currentSize + parts.bytesOnDisk > maxSplitBytes) { + closePartition() + } + // Add the given file to the current partition. + currentSize += parts.bytesOnDisk + openCostInBytes + if (currentMinPartsNum == -1L) { + currentMinPartsNum = parts.minBlockNumber + } + currentMaxPartsNum = parts.maxBlockNumber + } + } + closePartition() + } } diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/ClickHouseFileIndexBase.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/ClickHouseFileIndexBase.scala index 458a22fe1a2c2..047ec5e0490a3 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/ClickHouseFileIndexBase.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/ClickHouseFileIndexBase.scala @@ -17,15 +17,16 @@ package org.apache.spark.sql.execution.datasources.v1 import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.expressions.{Expression, GenericInternalRow} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, GenericInternalRow} import org.apache.spark.sql.connector.read.InputPartition import org.apache.spark.sql.delta.{DeltaLog, Snapshot} import org.apache.spark.sql.delta.actions.AddFile import org.apache.spark.sql.delta.files.TahoeFileIndex -import org.apache.spark.sql.execution.datasources.PartitionDirectory +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, PartitionDirectory} import org.apache.spark.sql.execution.datasources.utils.MergeTreePartsPartitionsUtil import org.apache.spark.sql.execution.datasources.v2.clickhouse.table.ClickHouseTableV2 import org.apache.spark.sql.types.StructType +import org.apache.spark.util.collection.BitSet import org.apache.hadoop.fs.{FileStatus, Path} @@ -80,8 +81,24 @@ abstract class ClickHouseFileIndexBase( }) } - def partsPartitions: Seq[InputPartition] = - MergeTreePartsPartitionsUtil.getPartsPartitions(spark, table) + def partsPartitions( + relation: HadoopFsRelation, + selectedPartitions: Array[PartitionDirectory], + output: Seq[Attribute], + bucketedScan: Boolean, + optionalBucketSet: Option[BitSet], + optionalNumCoalescedBuckets: Option[Int], + disableBucketedScan: Boolean): Seq[InputPartition] = + MergeTreePartsPartitionsUtil.getMergeTreePartsPartitions( + relation, + selectedPartitions, + output, + bucketedScan, + spark, + table, + optionalBucketSet, + optionalNumCoalescedBuckets, + disableBucketedScan) override def refresh(): Unit = {} diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseConfig.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseConfig.scala index bdbb89ea47e13..d47d2bd2159f9 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseConfig.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseConfig.scala @@ -16,6 +16,8 @@ */ package org.apache.spark.sql.execution.datasources.v2.clickhouse +import org.apache.spark.sql.catalyst.catalog.BucketSpec + import java.util import scala.collection.JavaConverters.mapAsScalaMapConverter @@ -36,8 +38,10 @@ object ClickHouseConfig { val CLICKHOUSE_WAREHOUSE_DIR = "spark.gluten.sql.columnar.backend.ch.warehouse.dir" - /** Validates specified configurations and returns the normalized key -> value map. */ - def validateConfigurations(allProperties: util.Map[String, String]): Map[String, String] = { + /** Create a mergetree configurations and returns the normalized key -> value map. */ + def createMergeTreeConfigurations( + allProperties: util.Map[String, String], + buckets: Option[BucketSpec]): Map[String, String] = { val configurations = scala.collection.mutable.Map[String, String]() allProperties.asScala.foreach(configurations += _) if (!configurations.contains("metadata_path")) { @@ -63,6 +67,13 @@ object ClickHouseConfig { if (!configurations.contains("is_distribute")) { configurations += ("is_distribute" -> "true") } + + if (buckets.isDefined) { + val bucketSpec = buckets.get + configurations += ("numBuckets" -> bucketSpec.numBuckets.toString) + configurations += ("bucketColumnNames" -> bucketSpec.bucketColumnNames.mkString(",")) + configurations += ("sortColumnNames" -> bucketSpec.sortColumnNames.mkString(",")) + } configurations.toMap } } diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseSparkCatalog.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseSparkCatalog.scala index de51ccda6fb57..0fc1fcd16cb68 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseSparkCatalog.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseSparkCatalog.scala @@ -92,13 +92,16 @@ class ClickHouseSparkCatalog writeOptions: Map[String, String], sourceQuery: Option[DataFrame], operation: TableCreationModes.CreationMode): Table = { - val tableProperties = ClickHouseConfig.validateConfigurations(allTableProperties) val (partitionColumns, maybeBucketSpec) = SparkShimLoader.getSparkShims.convertPartitionTransforms(partitions) var newSchema = schema var newPartitionColumns = partitionColumns var newBucketSpec = maybeBucketSpec + // Delta does not support bucket feature, so save the bucket infos into properties if exists. + val tableProperties = + ClickHouseConfig.createMergeTreeConfigurations(allTableProperties, newBucketSpec) + val isByPath = isPathIdentifier(ident) val location = if (isByPath) { Option(ident.name()) @@ -146,22 +149,31 @@ class ClickHouseSparkCatalog loadedNewTable match { case v: ClickHouseTableV2 => // TODO: remove this operation after implementing write mergetree into table - ScanMergeTreePartsUtils.scanMergeTreePartsToAddFile(spark.sessionState.newHadoopConf(), v) - v.refresh() + scanMergeTreePartsToAddFile(v) case _ => } loadedNewTable } + def scanMergeTreePartsToAddFile(clickHouseTableV2: ClickHouseTableV2): Unit = { + val (pathFilter, isBucketTable) = if (clickHouseTableV2.bucketOption.isDefined) { + ("/[0-9]*/*_[0-9]*_[0-9]*_[0-9]*", true) + } else { + ("/*_[0-9]*_[0-9]*_[0-9]*", false) + } + ScanMergeTreePartsUtils.scanMergeTreePartsToAddFile( + spark.sessionState.newHadoopConf(), + clickHouseTableV2, + pathFilter, + isBucketTable) + clickHouseTableV2.refresh() + } + /** Performs checks on the parameters provided for table creation for a ClickHouse table. */ private def verifyTableAndSolidify( tableDesc: CatalogTable, query: Option[LogicalPlan]): CatalogTable = { - if (tableDesc.bucketSpec.isDefined) { - throw new UnsupportedOperationException("Do not support Bucketing") - } - val schema = query .map { plan => @@ -215,8 +227,7 @@ class ClickHouseSparkCatalog try { loadTable(ident) match { case v: ClickHouseTableV2 => - ScanMergeTreePartsUtils.scanMergeTreePartsToAddFile(spark.sessionState.newHadoopConf(), v) - v.refresh() + scanMergeTreePartsToAddFile(v) } super.invalidateTable(ident) } catch { 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 f89fb30e97181..1c22aea88faf6 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 @@ -36,6 +36,8 @@ case class AddMergeTreeParts( maxBlockNumber: Long, level: Int, dataVersion: Long, + bucketNum: String, + dirName: String, dataChange: Boolean, partition: String = "", defaultCompressionCodec: String = "LZ4", @@ -93,6 +95,8 @@ object AddFileTags { maxBlockNumber: Long, level: Int, dataVersion: Long, + bucketNum: String, + dirName: String, dataChange: Boolean, partition: String = "", defaultCompressionCodec: String = "LZ4", @@ -117,7 +121,9 @@ object AddFileTags { "maxBlockNumber" -> maxBlockNumber.toString, "level" -> level.toString, "dataVersion" -> dataVersion.toString, - "defaultCompressionCodec" -> defaultCompressionCodec + "defaultCompressionCodec" -> defaultCompressionCodec, + "bucketNum" -> bucketNum, + "dirName" -> dirName ) AddFile(name, partitionValues, bytesOnDisk, modificationTime, dataChange, stats, tags) } @@ -142,6 +148,8 @@ object AddFileTags { addFile.tags.get("maxBlockNumber").get.toLong, addFile.tags.get("level").get.toInt, addFile.tags.get("dataVersion").get.toLong, + addFile.tags.get("bucketNum").get, + addFile.tags.get("dirName").get, addFile.dataChange, addFile.tags.get("partition").get, addFile.tags.get("defaultCompressionCodec").get, diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/table/ClickHouseTableV2.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/table/ClickHouseTableV2.scala index d153206e857d9..4873796b86c7e 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/table/ClickHouseTableV2.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/table/ClickHouseTableV2.scala @@ -162,6 +162,20 @@ case class ClickHouseTableV2( new ClickHouseScanBuilder(spark, this, tableSchema, options) } + lazy val bucketOption: Option[BucketSpec] = { + val tableProperties = properties() + if (tableProperties.containsKey("numBuckets")) { + val numBuckets = tableProperties.get("numBuckets").toInt + val bucketColumnNames: Seq[String] = + tableProperties.get("bucketColumnNames").split(",").toSeq + val sortColumnNames: Seq[String] = + tableProperties.get("sortColumnNames").split(",").toSeq + Some(BucketSpec(numBuckets, bucketColumnNames, sortColumnNames)) + } else { + None + } + } + /** Return V1Table. */ override def v1Table: CatalogTable = { if (catalogTable.isEmpty) { @@ -206,7 +220,6 @@ case class ClickHouseTableV2( } val fileIndex = ClickHouseFileIndex(spark, deltaLog, deltaLog.dataPath, this, snapshotToUse, partitionFilters) - var bucketSpec: Option[BucketSpec] = None new HadoopFsRelation( fileIndex, partitionSchema = @@ -217,7 +230,7 @@ case class ClickHouseTableV2( dataSchema = DeltaColumnMapping.dropColumnMappingMetadata( ColumnWithDefaultExprUtils.removeDefaultExpressions( SchemaUtils.dropNullTypeColumns(snapshotToUse.metadata.schema))), - bucketSpec = bucketSpec, + bucketSpec = bucketOption, fileFormat(snapshotToUse.metadata), // `metadata.format.options` is not set today. Even if we support it in future, we shouldn't // store any file system options since they may contain credentials. Hence, it will never @@ -307,7 +320,15 @@ object ClickHouseTableV2 extends Logging { .as[AddFile] .collect() .map(AddFileTags.partsMapToParts) - .sortWith(_.minBlockNumber < _.minBlockNumber) + .sortWith( + (a, b) => { + if (a.bucketNum.nonEmpty) { + (Integer.parseInt(a.bucketNum) < Integer.parseInt(b.bucketNum)) || + (a.minBlockNumber < b.minBlockNumber) + } else { + a.minBlockNumber < b.minBlockNumber + } + }) .toSeq logInfo( s"Get ${allParts.size} parts from path ${tablePath.toString} " + diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/utils/ScanMergeTreePartsUtils.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/utils/ScanMergeTreePartsUtils.scala index 48f1011fb6fd3..94ecb3525ede1 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/utils/ScanMergeTreePartsUtils.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/utils/ScanMergeTreePartsUtils.scala @@ -31,9 +31,11 @@ object ScanMergeTreePartsUtils extends Logging { def scanMergeTreePartsToAddFile( configuration: Configuration, - clickHouseTableV2: ClickHouseTableV2): Seq[AddFile] = { + clickHouseTableV2: ClickHouseTableV2, + pathFilter: String, + isBucketTable: Boolean): Seq[AddFile] = { // scan parts dir - val scanPath = new Path(clickHouseTableV2.path + "/*_[0-9]*_[0-9]*_[0-9]*") + val scanPath = new Path(clickHouseTableV2.path + pathFilter) val fs = scanPath.getFileSystem(configuration) val fileGlobStatuses = fs.globStatus(scanPath) val allDirSummary = fileGlobStatuses @@ -44,11 +46,26 @@ object ScanMergeTreePartsUtils extends Logging { val sum = fs.getContentSummary(p.getPath) val pathName = p.getPath.getName val pathNameArr = pathName.split("_") - val (partitionId, minBlockNum, maxBlockNum, level) = if (pathNameArr.length == 4) { - (pathNameArr(0), pathNameArr(1).toLong, pathNameArr(2).toLong, pathNameArr(3).toInt) - } else { - ("", 0L, 0L, 0) - } + val (partitionId, bucketNum, minBlockNum, maxBlockNum, level) = + if (pathNameArr.length == 4) { + if (isBucketTable) { + ( + pathNameArr(0), + p.getPath.getParent.getName, + pathNameArr(1).toLong, + pathNameArr(2).toLong, + pathNameArr(3).toInt) + } else { + ( + pathNameArr(0), + "", + pathNameArr(1).toLong, + pathNameArr(2).toLong, + pathNameArr(3).toInt) + } + } else { + ("", "", 0L, 0L, 0) + } ( pathName, partitionId, @@ -56,7 +73,8 @@ object ScanMergeTreePartsUtils extends Logging { maxBlockNum, level, sum.getLength, - p.getModificationTime) + p.getModificationTime, + bucketNum) }) .filter(!_._2.equals("")) @@ -67,13 +85,20 @@ object ScanMergeTreePartsUtils extends Logging { } val finalActions = allDirSummary.map( dir => { + val (filePath, name) = if (isBucketTable) { + ( + clickHouseTableV2.deltaLog.dataPath.toString + "/" + dir._8 + "/" + dir._1, + dir._8 + "/" + dir._1) + } else { + (clickHouseTableV2.deltaLog.dataPath.toString + "/" + dir._1, dir._1) + } AddFileTags.partsInfoToAddFile( clickHouseTableV2.catalogTable.get.identifier.database.get, clickHouseTableV2.catalogTable.get.identifier.table, clickHouseTableV2.snapshot.metadata.configuration("engine"), - clickHouseTableV2.deltaLog.dataPath.toString + "/" + dir._1, + filePath, "", - dir._1, + name, "", 0L, dir._6, @@ -85,6 +110,8 @@ object ScanMergeTreePartsUtils extends Logging { dir._4, dir._5, dir._3, + dir._8, + dir._1, dataChange = true ) }) diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q01.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q01.out new file mode 100644 index 0000000000000..5e74c1cb3cf0b --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q01.out @@ -0,0 +1,5 @@ +4 +A|-|F|-|3774200.0|-|5.320753880690033E9|-|5.054096266682795E9|-|5.256751331449255E9|-|25.537587116854997|-|36002.12382901437|-|0.05014459706341336|-|147790 +N|-|F|-|95257.0|-|1.3373779583999987E8|-|1.271323726511998E8|-|1.3228629122944495E8|-|25.30066401062417|-|35521.32691633463|-|0.049394422310757|-|3765 +N|-|O|-|7454519.0|-|1.0505523577969997E10|-|9.979830780624588E9|-|1.0378897125948418E10|-|25.54518413937502|-|36000.3823559628|-|0.05009701285399946|-|291817 +R|-|F|-|3785523.0|-|5.337950526469997E9|-|5.071818532941964E9|-|5.274405503049355E9|-|25.5259438574251|-|35994.0292140309|-|0.049989278561856136|-|148301 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q02.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q02.out new file mode 100644 index 0000000000000..053518a5a1086 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q02.out @@ -0,0 +1,45 @@ +44 +9828.21|-|Supplier#000000647|-|UNITED KINGDOM|-|13120|-|Manufacturer#5|-|x5U7MBZmwfG9|-|33-258-202-4782|-|s the slyly even ideas poach fluffily +9508.37|-|Supplier#000000070|-|FRANCE|-|3563|-|Manufacturer#1|-|INWNH2w,OOWgNDq0BRCcBwOMQc6PdFDc4|-|16-821-608-1166|-|ests sleep quickly express ideas. ironic ideas haggle about the final T +9508.37|-|Supplier#000000070|-|FRANCE|-|17268|-|Manufacturer#4|-|INWNH2w,OOWgNDq0BRCcBwOMQc6PdFDc4|-|16-821-608-1166|-|ests sleep quickly express ideas. ironic ideas haggle about the final T +9453.01|-|Supplier#000000802|-|ROMANIA|-|10021|-|Manufacturer#5|-|,6HYXb4uaHITmtMBj4Ak57Pd|-|29-342-882-6463|-|gular frets. permanently special multipliers believe blithely alongs +9453.01|-|Supplier#000000802|-|ROMANIA|-|13275|-|Manufacturer#4|-|,6HYXb4uaHITmtMBj4Ak57Pd|-|29-342-882-6463|-|gular frets. permanently special multipliers believe blithely alongs +9192.1|-|Supplier#000000115|-|UNITED KINGDOM|-|13325|-|Manufacturer#1|-|nJ 2t0f7Ve,wL1,6WzGBJLNBUCKlsV|-|33-597-248-1220|-|es across the carefully express accounts boost caref +9032.15|-|Supplier#000000959|-|GERMANY|-|4958|-|Manufacturer#4|-|8grA EHBnwOZhO|-|17-108-642-3106|-|nding dependencies nag furiou +8702.02|-|Supplier#000000333|-|RUSSIA|-|11810|-|Manufacturer#3|-|MaVf XgwPdkiX4nfJGOis8Uu2zKiIZH|-|32-508-202-6136|-|oss the deposits cajole carefully even pinto beans. regular foxes detect alo +8615.5|-|Supplier#000000812|-|FRANCE|-|10551|-|Manufacturer#2|-|8qh4tezyScl5bidLAysvutB,,ZI2dn6xP|-|16-585-724-6633|-|y quickly regular deposits? quickly pending packages after the caref +8615.5|-|Supplier#000000812|-|FRANCE|-|13811|-|Manufacturer#4|-|8qh4tezyScl5bidLAysvutB,,ZI2dn6xP|-|16-585-724-6633|-|y quickly regular deposits? quickly pending packages after the caref +8488.53|-|Supplier#000000367|-|RUSSIA|-|6854|-|Manufacturer#4|-|E Sv9brQVf43Mzz|-|32-458-198-9557|-|ages. carefully final excuses nag finally. carefully ironic deposits abov +8430.52|-|Supplier#000000646|-|FRANCE|-|11384|-|Manufacturer#3|-|IUzsmT,2oBgjhWP2TlXTL6IkJH,4h,1SJRt|-|16-601-220-5489|-|ites among the always final ideas kindle according to the theodolites. notornis in +8271.39|-|Supplier#000000146|-|RUSSIA|-|4637|-|Manufacturer#5|-|rBDNgCr04x0sfdzD5,gFOutCiG2|-|32-792-619-3155|-|s cajole quickly special requests. quickly enticing theodolites h +8096.98|-|Supplier#000000574|-|RUSSIA|-|323|-|Manufacturer#4|-|2O8 sy9g2mlBOuEjzj0pA2pevk,|-|32-866-246-8752|-|ully after the regular requests. slyly final dependencies wake slyly along the busy deposit +7392.78|-|Supplier#000000170|-|UNITED KINGDOM|-|7655|-|Manufacturer#2|-|RtsXQ,SunkA XHy9|-|33-803-340-5398|-|ake carefully across the quickly +7205.2|-|Supplier#000000477|-|GERMANY|-|10956|-|Manufacturer#5|-|VtaNKN5Mqui5yh7j2ldd5waf|-|17-180-144-7991|-|excuses wake express deposits. furiously careful asymptotes according to the carefull +6820.35|-|Supplier#000000007|-|UNITED KINGDOM|-|13217|-|Manufacturer#5|-|s,4TicNGB4uO6PaSqNBUq|-|33-990-965-2201|-|s unwind silently furiously regular courts. final requests are deposits. requests wake quietly blit +6721.7|-|Supplier#000000954|-|FRANCE|-|4191|-|Manufacturer#3|-|P3O5p UFz1QsLmZX|-|16-537-341-8517|-|ect blithely blithely final acco +6329.9|-|Supplier#000000996|-|GERMANY|-|10735|-|Manufacturer#2|-|Wx4dQwOAwWjfSCGupfrM|-|17-447-811-3282|-|ironic forges cajole blithely agai +6173.87|-|Supplier#000000408|-|RUSSIA|-|18139|-|Manufacturer#1|-|qcor1u,vJXAokjnL5,dilyYNmh|-|32-858-724-2950|-|blithely pending packages cajole furiously slyly pending notornis. slyly final +5364.99|-|Supplier#000000785|-|RUSSIA|-|13784|-|Manufacturer#4|-|W VkHBpQyD3qjQjWGpWicOpmILFehmEdWy67kUGY|-|32-297-653-2203|-|packages boost carefully. express ideas along +5069.27|-|Supplier#000000328|-|GERMANY|-|16327|-|Manufacturer#1|-|SMm24d WG62|-|17-231-513-5721|-|he unusual ideas. slyly final packages a +4941.88|-|Supplier#000000321|-|ROMANIA|-|7320|-|Manufacturer#5|-|pLngFl5yeMcHyov|-|29-573-279-1406|-|y final requests impress s +4672.25|-|Supplier#000000239|-|RUSSIA|-|12238|-|Manufacturer#1|-|XO101kgHrJagK2FL1U6QCaTE ncCsMbeuTgK6o8|-|32-396-654-6826|-|arls wake furiously deposits. even, regular depen +4586.49|-|Supplier#000000680|-|RUSSIA|-|5679|-|Manufacturer#3|-|UhvDfdEfJh,Qbe7VZb8uSGO2TU 0jEa6nXZXE|-|32-522-382-1620|-|the regularly regular dependencies. carefully bold excuses under th +4518.31|-|Supplier#000000149|-|FRANCE|-|18344|-|Manufacturer#5|-|pVyWsjOidpHKp4NfKU4yLeym|-|16-660-553-2456|-|ts detect along the foxes. final Tiresias are. idly pending deposits haggle; even, blithe pin +4315.15|-|Supplier#000000509|-|FRANCE|-|18972|-|Manufacturer#2|-|SF7dR8V5pK|-|16-298-154-3365|-|ronic orbits are furiously across the requests. quickly express ideas across the special, bold +3526.53|-|Supplier#000000553|-|FRANCE|-|8036|-|Manufacturer#4|-|a,liVofXbCJ|-|16-599-552-3755|-|lar dinos nag slyly brave +3526.53|-|Supplier#000000553|-|FRANCE|-|17018|-|Manufacturer#3|-|a,liVofXbCJ|-|16-599-552-3755|-|lar dinos nag slyly brave +3294.68|-|Supplier#000000350|-|GERMANY|-|4841|-|Manufacturer#4|-|KIFxV73eovmwhh|-|17-113-181-4017|-|e slyly special foxes. furiously unusual deposits detect carefully carefully ruthless foxes. quick +2972.26|-|Supplier#000000016|-|RUSSIA|-|1015|-|Manufacturer#4|-|YjP5C55zHDXL7LalK27zfQnwejdpin4AMpvh|-|32-822-502-4215|-|ously express ideas haggle quickly dugouts? fu +2963.09|-|Supplier#000000840|-|ROMANIA|-|3080|-|Manufacturer#2|-|iYzUIypKhC0Y|-|29-781-337-5584|-|eep blithely regular dependencies. blithely regular platelets sublate alongside o +2221.25|-|Supplier#000000771|-|ROMANIA|-|13981|-|Manufacturer#2|-|lwZ I15rq9kmZXUNhl|-|29-986-304-9006|-|nal foxes eat slyly about the fluffily permanent id +1381.97|-|Supplier#000000104|-|FRANCE|-|18103|-|Manufacturer#3|-|Dcl4yGrzqv3OPeRO49bKh78XmQEDR7PBXIs0m|-|16-434-972-6922|-|gular ideas. bravely bold deposits haggle through the carefully final deposits. slyly unusual idea +906.07|-|Supplier#000000138|-|ROMANIA|-|8363|-|Manufacturer#4|-|utbplAm g7RmxVfYoNdhcrQGWuzRqPe0qHSwbKw|-|29-533-434-6776|-|ickly unusual requests cajole. accounts above the furiously special excuses +765.69|-|Supplier#000000799|-|RUSSIA|-|11276|-|Manufacturer#2|-|jwFN7ZB3T9sMF|-|32-579-339-1495|-|nusual requests. furiously unusual epitaphs integrate. slyly +727.89|-|Supplier#000000470|-|ROMANIA|-|6213|-|Manufacturer#3|-|XckbzsAgBLbUkdfjgJEPjmUMTM8ebSMEvI|-|29-165-289-1523|-|gular excuses. furiously regular excuses sleep slyly caref +683.07|-|Supplier#000000651|-|RUSSIA|-|4888|-|Manufacturer#4|-|oWekiBV6s,1g|-|32-181-426-4490|-|ly regular requests cajole abou +167.56|-|Supplier#000000290|-|FRANCE|-|2037|-|Manufacturer#1|-|6Bk06GVtwZaKqg01|-|16-675-286-5102|-|the theodolites. ironic, ironic deposits above +91.39|-|Supplier#000000949|-|UNITED KINGDOM|-|9430|-|Manufacturer#2|-|a,UE,6nRVl2fCphkOoetR1ajIzAEJ1Aa1G1HV|-|33-332-697-2768|-|pinto beans. carefully express requests hagg +-314.06|-|Supplier#000000510|-|ROMANIA|-|17242|-|Manufacturer#4|-|VmXQl ,vY8JiEseo8Mv4zscvNCfsY|-|29-207-852-3454|-|bold deposits. carefully even d +-820.89|-|Supplier#000000409|-|GERMANY|-|2156|-|Manufacturer#5|-|LyXUYFz7aXrvy65kKAbTatGzGS,NDBcdtD|-|17-719-517-9836|-|y final, slow theodolites. furiously regular req +-845.44|-|Supplier#000000704|-|ROMANIA|-|9926|-|Manufacturer#5|-|hQvlBqbqqnA5Dgo1BffRBX78tkkRu|-|29-300-896-5991|-|ctions. carefully sly requ +-942.73|-|Supplier#000000563|-|GERMANY|-|5797|-|Manufacturer#1|-|Rc7U1cRUhYs03JD|-|17-108-537-2691|-|slyly furiously final decoys; silent, special realms poach f diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q03.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q03.out new file mode 100644 index 0000000000000..fb8357ef659a5 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q03.out @@ -0,0 +1,11 @@ +10 +223140|-|355369.0698|-|1995-03-14|-|0 +584291|-|354494.7318|-|1995-02-21|-|0 +405063|-|353125.4577|-|1995-03-03|-|0 +573861|-|351238.27699999994|-|1995-03-09|-|0 +554757|-|349181.7426|-|1995-03-14|-|0 +506021|-|321075.58099999995|-|1995-03-10|-|0 +121604|-|318576.41540000006|-|1995-03-07|-|0 +108514|-|314967.0754|-|1995-02-20|-|0 +462502|-|312604.542|-|1995-03-08|-|0 +178727|-|309728.93059999996|-|1995-02-25|-|0 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q04.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q04.out new file mode 100644 index 0000000000000..766fc7ea4f426 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q04.out @@ -0,0 +1,6 @@ +5 +1-URGENT|-|999 +2-HIGH|-|997 +3-MEDIUM|-|1031 +4-NOT SPECIFIED|-|989 +5-LOW|-|1077 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q05.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q05.out new file mode 100644 index 0000000000000..b0936e013ebc3 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q05.out @@ -0,0 +1,6 @@ +5 +CHINA|-|7822102.999999997 +INDIA|-|6376121.5084999995 +JAPAN|-|6000077.218400001 +INDONESIA|-|5580475.4026999995 +VIETNAM|-|4497840.546600002 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q06.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q06.out new file mode 100644 index 0000000000000..444c5dc7290b1 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q06.out @@ -0,0 +1,2 @@ +1 +1.1803420253400011E7 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q07.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q07.out new file mode 100644 index 0000000000000..4ef62d4e98481 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q07.out @@ -0,0 +1,5 @@ +4 +FRANCE|-|GERMANY|-|1995|-|4637235.1501 +FRANCE|-|GERMANY|-|1996|-|5224779.573599998 +GERMANY|-|FRANCE|-|1995|-|6232818.703699999 +GERMANY|-|FRANCE|-|1996|-|5557312.1121000005 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q08.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q08.out new file mode 100644 index 0000000000000..4a6c1930fd557 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q08.out @@ -0,0 +1,3 @@ +2 +1995|-|0.02864874130561755 +1996|-|0.018250279107962144 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q09.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q09.out new file mode 100644 index 0000000000000..0934cc443ecdc --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q09.out @@ -0,0 +1,176 @@ +175 +ALGERIA|-|1998|-|2321785.366697696 +ALGERIA|-|1997|-|3685016.8671991806 +ALGERIA|-|1996|-|4276597.428312926 +ALGERIA|-|1995|-|4418370.424887989 +ALGERIA|-|1994|-|3864849.9593177494 +ALGERIA|-|1993|-|3541051.3923231354 +ALGERIA|-|1992|-|4310013.353297314 +ARGENTINA|-|1998|-|2685983.8006414105 +ARGENTINA|-|1997|-|4242147.80452833 +ARGENTINA|-|1996|-|3907866.997697765 +ARGENTINA|-|1995|-|4605921.504230648 +ARGENTINA|-|1994|-|3542096.1580567244 +ARGENTINA|-|1993|-|3949965.945871686 +ARGENTINA|-|1992|-|4521180.466483507 +BRAZIL|-|1998|-|2778730.394547526 +BRAZIL|-|1997|-|4642037.467789658 +BRAZIL|-|1996|-|4530304.605538634 +BRAZIL|-|1995|-|4502344.872796597 +BRAZIL|-|1994|-|4875806.508928281 +BRAZIL|-|1993|-|4687478.655158408 +BRAZIL|-|1992|-|5035200.053016095 +CANADA|-|1998|-|2194509.047656807 +CANADA|-|1997|-|3482197.946842927 +CANADA|-|1996|-|3712231.268635334 +CANADA|-|1995|-|4014814.8447086867 +CANADA|-|1994|-|4145304.4862727826 +CANADA|-|1993|-|3787069.596131775 +CANADA|-|1992|-|4168009.4262125185 +CHINA|-|1998|-|3398577.9873981243 +CHINA|-|1997|-|6358959.337492856 +CHINA|-|1996|-|6435158.325252927 +CHINA|-|1995|-|6174776.210928064 +CHINA|-|1994|-|6385751.069562026 +CHINA|-|1993|-|5765034.130704248 +CHINA|-|1992|-|6324034.235200413 +EGYPT|-|1998|-|2333148.3330481704 +EGYPT|-|1997|-|3661244.2682996984 +EGYPT|-|1996|-|3765371.225550458 +EGYPT|-|1995|-|4094744.291482351 +EGYPT|-|1994|-|3566508.073745191 +EGYPT|-|1993|-|3725283.76369315 +EGYPT|-|1992|-|3373762.3252903135 +ETHIOPIA|-|1998|-|1953927.2704539902 +ETHIOPIA|-|1997|-|3285786.32113064 +ETHIOPIA|-|1996|-|3525028.7966111703 +ETHIOPIA|-|1995|-|3781674.8841145285 +ETHIOPIA|-|1994|-|3037409.4286103216 +ETHIOPIA|-|1993|-|3008978.2727072844 +ETHIOPIA|-|1992|-|2721203.240953414 +FRANCE|-|1998|-|2604373.8898065495 +FRANCE|-|1997|-|3982872.0631428845 +FRANCE|-|1996|-|3622479.246487785 +FRANCE|-|1995|-|4479939.720681944 +FRANCE|-|1994|-|3531013.206400797 +FRANCE|-|1993|-|4086437.326516616 +FRANCE|-|1992|-|3637792.1427063704 +GERMANY|-|1998|-|3291023.2809489267 +GERMANY|-|1997|-|5139337.326118546 +GERMANY|-|1996|-|4799810.43073009 +GERMANY|-|1995|-|5405785.777712038 +GERMANY|-|1994|-|4555556.43499315 +GERMANY|-|1993|-|4428195.092631204 +GERMANY|-|1992|-|4656148.404258718 +INDIA|-|1998|-|2591288.188878577 +INDIA|-|1997|-|5159562.700538857 +INDIA|-|1996|-|5307258.300371652 +INDIA|-|1995|-|5148208.801210843 +INDIA|-|1994|-|5164001.959532555 +INDIA|-|1993|-|4321398.435788524 +INDIA|-|1992|-|5297703.703198987 +INDONESIA|-|1998|-|3094900.15318282 +INDONESIA|-|1997|-|5719773.037064193 +INDONESIA|-|1996|-|6037238.598229062 +INDONESIA|-|1995|-|5266783.485130789 +INDONESIA|-|1994|-|5470762.866589345 +INDONESIA|-|1993|-|6189826.642484006 +INDONESIA|-|1992|-|4414623.150751896 +IRAN|-|1998|-|3214864.1193034053 +IRAN|-|1997|-|3688049.0721192565 +IRAN|-|1996|-|3621649.2234387854 +IRAN|-|1995|-|4420783.412560043 +IRAN|-|1994|-|4373984.6530368095 +IRAN|-|1993|-|3731301.7799542593 +IRAN|-|1992|-|4417133.370105695 +IRAQ|-|1998|-|2338859.411355956 +IRAQ|-|1997|-|3622681.560582499 +IRAQ|-|1996|-|4762291.873562039 +IRAQ|-|1995|-|4558092.741335291 +IRAQ|-|1994|-|4951604.173189375 +IRAQ|-|1993|-|3830077.986645673 +IRAQ|-|1992|-|3938636.4991397304 +JAPAN|-|1998|-|1849535.076985583 +JAPAN|-|1997|-|4068688.8535843 +JAPAN|-|1996|-|4044774.761344057 +JAPAN|-|1995|-|4793005.800355982 +JAPAN|-|1994|-|4114717.051689451 +JAPAN|-|1993|-|3614468.7382097025 +JAPAN|-|1992|-|4266694.464107132 +JORDAN|-|1998|-|1811488.0784755081 +JORDAN|-|1997|-|2951297.8744189194 +JORDAN|-|1996|-|3302528.3108127 +JORDAN|-|1995|-|3221814.0085915625 +JORDAN|-|1994|-|2417892.0989085087 +JORDAN|-|1993|-|3107641.7733764583 +JORDAN|-|1992|-|3316379.068783395 +KENYA|-|1998|-|2579075.415152621 +KENYA|-|1997|-|2929194.2393937483 +KENYA|-|1996|-|3569129.559210632 +KENYA|-|1995|-|3542889.1139644715 +KENYA|-|1994|-|3983095.3918699673 +KENYA|-|1993|-|3713988.976578933 +KENYA|-|1992|-|3304641.8360172305 +MOROCCO|-|1998|-|1815334.8205032034 +MOROCCO|-|1997|-|3693214.8475239435 +MOROCCO|-|1996|-|4116175.928718384 +MOROCCO|-|1995|-|3515127.1417130576 +MOROCCO|-|1994|-|4003072.102959623 +MOROCCO|-|1993|-|3599199.6739639966 +MOROCCO|-|1992|-|3958335.4259240194 +MOZAMBIQUE|-|1998|-|1620428.7378170486 +MOZAMBIQUE|-|1997|-|2802166.655346226 +MOZAMBIQUE|-|1996|-|2409955.1787294447 +MOZAMBIQUE|-|1995|-|2771602.6354734246 +MOZAMBIQUE|-|1994|-|2548226.2146838196 +MOZAMBIQUE|-|1993|-|2843748.9081963077 +MOZAMBIQUE|-|1992|-|2556501.101767652 +PERU|-|1998|-|2036430.3628828002 +PERU|-|1997|-|4064142.401114619 +PERU|-|1996|-|4068678.575663385 +PERU|-|1995|-|4657694.850584611 +PERU|-|1994|-|4731959.460547456 +PERU|-|1993|-|4144006.6611095574 +PERU|-|1992|-|3754635.006121686 +ROMANIA|-|1998|-|1992773.684887232 +ROMANIA|-|1997|-|2854639.8804400717 +ROMANIA|-|1996|-|3139337.303646003 +ROMANIA|-|1995|-|3222153.3740034285 +ROMANIA|-|1994|-|3222844.3231433732 +ROMANIA|-|1993|-|3488994.033106488 +ROMANIA|-|1992|-|3029274.44639354 +RUSSIA|-|1998|-|2339865.669088989 +RUSSIA|-|1997|-|4153619.5480268304 +RUSSIA|-|1996|-|3772067.3965853318 +RUSSIA|-|1995|-|4704988.859066772 +RUSSIA|-|1994|-|4479082.878186393 +RUSSIA|-|1993|-|4767719.968000547 +RUSSIA|-|1992|-|4533465.554854187 +SAUDI ARABIA|-|1998|-|3386948.9601899004 +SAUDI ARABIA|-|1997|-|5425980.341708716 +SAUDI ARABIA|-|1996|-|5227607.1639614105 +SAUDI ARABIA|-|1995|-|4506731.627395909 +SAUDI ARABIA|-|1994|-|4698658.734425808 +SAUDI ARABIA|-|1993|-|5493626.53336555 +SAUDI ARABIA|-|1992|-|4573560.016888149 +UNITED KINGDOM|-|1998|-|2252021.513909146 +UNITED KINGDOM|-|1997|-|4343926.797442301 +UNITED KINGDOM|-|1996|-|4189476.308943189 +UNITED KINGDOM|-|1995|-|4469569.883577569 +UNITED KINGDOM|-|1994|-|4410094.631914679 +UNITED KINGDOM|-|1993|-|4054677.1017434313 +UNITED KINGDOM|-|1992|-|3978688.875390002 +UNITED STATES|-|1998|-|2238771.5603515483 +UNITED STATES|-|1997|-|4135581.563030852 +UNITED STATES|-|1996|-|3624013.266055848 +UNITED STATES|-|1995|-|3892244.5118940985 +UNITED STATES|-|1994|-|3289224.1137186307 +UNITED STATES|-|1993|-|3626170.192997565 +UNITED STATES|-|1992|-|3993973.494530933 +VIETNAM|-|1998|-|1924313.4846883027 +VIETNAM|-|1997|-|3436195.3714692867 +VIETNAM|-|1996|-|4017288.8905669632 +VIETNAM|-|1995|-|3644054.1472892915 +VIETNAM|-|1994|-|4141277.6721435566 +VIETNAM|-|1993|-|2556114.1622103085 +VIETNAM|-|1992|-|4090524.4854196985 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q10.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q10.out new file mode 100644 index 0000000000000..661ba845a0e6d --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q10.out @@ -0,0 +1,21 @@ +20 +8242|-|Customer#000008242|-|622786.7297|-|6322.09|-|ETHIOPIA|-|P2n4nJhy,UqSo2s43YfSvYJDZ6lk|-|15-792-676-1184|-|slyly regular packages haggle carefully ironic ideas. courts are furiously. furiously unusual theodolites cajole. i +7714|-|Customer#000007714|-|557400.3053|-|9799.98|-|IRAN|-|SnnIGB,SkmnWpX3|-|20-922-418-6024|-|arhorses according to the blithely express re +11032|-|Customer#000011032|-|512500.9640999999|-|8496.93|-|UNITED KINGDOM|-|WIKHC7K3Cn7156iNOyfVG3cZ7YqkgsR,Ly|-|33-102-772-3533|-|posits-- furiously ironic accounts are again +2455|-|Customer#000002455|-|481592.40530000004|-|2070.99|-|GERMANY|-|RVn1ZSRtLqPlJLIZxvpmsbgC02|-|17-946-225-9977|-|al asymptotes. finally ironic accounts cajole furiously. permanently unusual theodolites aro +12106|-|Customer#000012106|-|479414.2133|-|5342.11|-|UNITED STATES|-|wth3twOmu6vy|-|34-905-346-4472|-|ly after the blithely regular foxes. accounts haggle carefully alongside of the blithely even ideas. +8530|-|Customer#000008530|-|457855.94670000003|-|9734.95|-|MOROCCO|-|GMQyte94oDM7eD7exnkj 4hH9yq3|-|25-736-932-5850|-|slyly asymptotes. quickly final deposits in +13984|-|Customer#000013984|-|446316.5104000001|-|3482.28|-|IRAN|-|qZXwuapCHvxbX|-|20-981-264-2952|-|y unusual courts could wake furiously +1966|-|Customer#000001966|-|444059.03819999995|-|1937.72|-|ALGERIA|-|jPv1 UHra5JLALR5Isci5u0636RoAu7t vH|-|10-973-269-8886|-|the blithely even accounts. final deposits cajole around the blithely final packages. +11026|-|Customer#000011026|-|417913.4142|-|7738.76|-|ALGERIA|-|XorIktoJOAEJkpNNMx|-|10-184-163-4632|-|ly even dolphins eat along the blithely even instructions. express attainments cajole slyly. busy dolphins in +8501|-|Customer#000008501|-|412797.51000000007|-|6906.7|-|ARGENTINA|-|776af4rOa mZ66hczs|-|11-317-552-5840|-|y final deposits after the fluffily even accounts are slyly final, regular +1565|-|Customer#000001565|-|412506.00619999995|-|1820.03|-|BRAZIL|-|EWQO5Ck,nMuHVQimqL8dLrixRP6QKveXcz9QgorW|-|12-402-178-2007|-|ously regular accounts wake slyly ironic idea +14398|-|Customer#000014398|-|408575.36000000004|-|-602.24|-|UNITED STATES|-|GWRCgIPHajtU21vICVvbJJerFu2cUk|-|34-814-111-5424|-|s. blithely even accounts cajole blithely. even foxes doubt-- +1465|-|Customer#000001465|-|405055.34569999995|-|9365.93|-|INDIA|-|tDRaTC7UgFbBX7VF6cVXYQA0|-|18-807-487-1074|-|s lose blithely ironic, regular packages. regular, final foxes haggle c +12595|-|Customer#000012595|-|401402.2391|-|-6.92|-|INDIA|-|LmeaX5cR,w9NqKugl yRm98|-|18-186-132-3352|-|o the busy accounts. blithely special gifts maintain a +961|-|Customer#000000961|-|401198.17370000004|-|6963.68|-|JAPAN|-|5,81YDLFuRR47KKzv8GXdmi3zyP37PlPn|-|22-989-463-6089|-|e final requests: busily final accounts believe a +14299|-|Customer#000014299|-|400968.3751|-|6595.97|-|RUSSIA|-|7lFczTya0iM1bhEWT|-|32-156-618-1224|-|carefully regular requests. quickly ironic accounts against the ru +623|-|Customer#000000623|-|399883.4257|-|7887.6|-|INDONESIA|-|HXiFb9oWlgqZXrJPUCEJ6zZIPxAM4m6|-|19-113-202-7085|-|requests. dolphins above the busily regular dependencies cajole after +9151|-|Customer#000009151|-|396562.0295|-|5691.95|-|IRAQ|-|7gIdRdaxB91EVdyx8DyPjShpMD|-|21-834-147-4906|-|ajole fluffily. furiously regular accounts are special, silent account +14819|-|Customer#000014819|-|396271.10360000003|-|7308.39|-|FRANCE|-|w8StIbymUXmLCcUag6sx6LUIp8E3pA,Ux|-|16-769-398-7926|-|ss, final asymptotes use furiously slyly ironic dependencies. special, express dugouts according to the dep +13478|-|Customer#000013478|-|395513.13580000005|-|-778.11|-|KENYA|-|9VIsvIeZrJpC6OOdYheMC2vdtq8Ai0Rt|-|24-983-202-8240|-|r theodolites. slyly unusual pinto beans sleep fluffily against the asymptotes. quickly r diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q11.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q11.out new file mode 100644 index 0000000000000..2a0acdd41cd41 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q11.out @@ -0,0 +1,2542 @@ +2541 +12098|-|1.6227681302001953E7 +5134|-|1.5709338600280762E7 +13334|-|1.5023662028076172E7 +17052|-|1.4351644134521484E7 +3452|-|1.4070870250854492E7 +12552|-|1.3332468853271484E7 +1084|-|1.3170427993041992E7 +5797|-|1.3038622729980469E7 +12633|-|1.2892561625549316E7 +403|-|1.2856217222351074E7 +1833|-|1.202458152166748E7 +2084|-|1.1502875112976074E7 +17349|-|1.1354212805175781E7 +18427|-|1.1282385292816162E7 +2860|-|1.1262530243408203E7 +17852|-|1.0934711921264648E7 +9871|-|1.0889253548828125E7 +12231|-|1.0841131487487793E7 +6366|-|1.0759787055236816E7 +12146|-|1.0257362796264648E7 +5043|-|1.0226396083862305E7 +12969|-|1.012577785180664E7 +1504|-|1.000439711505127E7 +14327|-|9981697.163085938 +134|-|9965150.782409668 +6860|-|9805871.418884277 +10624|-|9776138.49609375 +15819|-|9775705.187072754 +3293|-|9674928.096130371 +19865|-|9653766.708618164 +8870|-|9648981.921569824 +15778|-|9636332.747680664 +12360|-|9635023.893310547 +14389|-|9475588.448608398 +3257|-|9451029.524414062 +9476|-|9435207.380859375 +19629|-|9391236.689453125 +7179|-|9386222.462280273 +15723|-|9383900.68359375 +4054|-|9313809.727294922 +2380|-|9307751.31225586 +19084|-|9302916.649169922 +4703|-|9280804.705810547 +18791|-|9267017.754821777 +19994|-|9235972.75805664 +9149|-|9121803.65637207 +15118|-|9120819.403503418 +6116|-|9079369.183227539 +7052|-|9077468.71899414 +14147|-|9069193.912963867 +7305|-|9035228.5390625 +9130|-|9024379.35357666 +16698|-|8991337.720825195 +1553|-|8977225.985046387 +16777|-|8961355.341064453 +1402|-|8953778.921081543 +18963|-|8934063.592407227 +8358|-|8930611.456237793 +17547|-|8860117.219238281 +5128|-|8844222.816467285 +17063|-|8840649.624023438 +15490|-|8833581.16040039 +14761|-|8817240.756347656 +19601|-|8791341.086547852 +16160|-|8740262.78338623 +13597|-|8702669.693637848 +13653|-|8693170.205322266 +16383|-|8691506.236450195 +325|-|8667741.473754883 +8879|-|8667584.401550293 +10564|-|8667098.267211914 +17429|-|8661827.972106934 +17403|-|8643350.085571289 +18294|-|8616583.627380371 +4181|-|8592684.797241211 +13008|-|8567480.8203125 +13211|-|8537000.156066895 +1884|-|8532644.174194336 +11101|-|8530945.055419922 +11562|-|8528028.735031128 +15878|-|8523591.907104492 +834|-|8522135.550048828 +2423|-|8517903.10180664 +15383|-|8513433.094543457 +18119|-|8507612.065429688 +7389|-|8506098.984375 +5016|-|8489783.899841309 +17473|-|8444766.384887695 +6669|-|8428618.723022461 +384|-|8418472.420593262 +12052|-|8411519.051513672 +17562|-|8409022.674499512 +8128|-|8379149.592407227 +13813|-|8374830.797607422 +12800|-|8318626.645629883 +10887|-|8315019.5205078125 +1644|-|8285452.998046875 +16638|-|8274567.789916992 +1394|-|8255140.706726074 +7219|-|8254985.187194824 +13358|-|8253829.867919922 +5562|-|8252365.134216309 +14861|-|8242295.892150879 +15416|-|8196621.351074219 +1963|-|8192206.631652832 +2841|-|8148678.428649902 +6635|-|8122918.347412109 +3609|-|8099812.335205078 +6372|-|8093694.963500977 +5231|-|8091704.0107421875 +8755|-|8085017.173873901 +4071|-|8083755.3486328125 +4977|-|8058501.938415527 +11873|-|8057540.239868164 +12127|-|8051143.096252441 +2401|-|8049525.825378418 +15964|-|8037547.541809082 +10129|-|8030854.880859375 +7605|-|8028622.395629883 +9327|-|8022291.034057617 +11814|-|7983589.842041016 +4646|-|7981660.829956055 +6032|-|7981511.544433594 +1076|-|7977074.816467285 +4043|-|7971930.119384766 +8932|-|7967222.031799316 +13705|-|7953344.6142578125 +16855|-|7923819.0 +3234|-|7920022.0 +17752|-|7901362.887817383 +2097|-|7892993.344177246 +18599|-|7890774.278808594 +19596|-|7874564.710388184 +11392|-|7861172.29296875 +18121|-|7857581.8212890625 +17452|-|7838465.422790527 +6076|-|7821317.0341796875 +15134|-|7804760.25 +8641|-|7802917.367553711 +2134|-|7800185.262374878 +16352|-|7797112.152099609 +19312|-|7775952.218261719 +2895|-|7759327.196136475 +12860|-|7758135.167358398 +153|-|7755681.4609375 +15089|-|7735438.240905762 +14797|-|7725353.121582031 +15946|-|7722773.891845703 +10919|-|7722425.451660156 +9867|-|7721597.614929199 +11881|-|7713136.2685546875 +16552|-|7708517.961303711 +6925|-|7703999.798461914 +12147|-|7703826.796264648 +8923|-|7702690.561279297 +16116|-|7697970.658813477 +10661|-|7673830.010009766 +17094|-|7665368.267822266 +18648|-|7650862.03894043 +12172|-|7641326.5126953125 +15123|-|7633032.4783325195 +4993|-|7600570.908935547 +17162|-|7592062.707092285 +13506|-|7580810.058837891 +3436|-|7575616.171081543 +6271|-|7559794.049926758 +13314|-|7555156.726318359 +17242|-|7550949.617614746 +2753|-|7549573.943603516 +7391|-|7543159.912353516 +7418|-|7541449.544494629 +116|-|7520874.218811035 +12436|-|7520234.376159668 +1181|-|7494798.041259766 +12963|-|7491248.032836914 +213|-|7479470.417358398 +7114|-|7477681.092712402 +18521|-|7476478.258056641 +8973|-|7458603.649963379 +4202|-|7454095.783569336 +12009|-|7442105.203491211 +10609|-|7429346.288696289 +5622|-|7424142.858764648 +5143|-|7422760.191894531 +898|-|7414133.955078125 +12257|-|7408190.745422363 +6740|-|7400350.103759766 +1146|-|7394394.743774414 +5485|-|7378181.939208984 +8437|-|7376353.476928711 +6531|-|7362366.996826172 +16463|-|7362106.593017578 +10412|-|7359552.441833496 +12857|-|7340801.43347168 +12354|-|7332343.0849609375 +7616|-|7320032.390075684 +3426|-|7312340.632568359 +8622|-|7307266.390197754 +6818|-|7304782.037963867 +3705|-|7299379.945068359 +12733|-|7298398.104553223 +1574|-|7293446.1943359375 +10554|-|7289933.552734375 +9363|-|7284647.821655273 +4611|-|7282114.910888672 +7857|-|7266949.0673828125 +9616|-|7265005.353790283 +15860|-|7254466.680541992 +15554|-|7247592.661499023 +3422|-|7247586.549133301 +9134|-|7236404.478027344 +17408|-|7220083.485656738 +15853|-|7219985.026794434 +9266|-|7218517.036376953 +1881|-|7208346.306152344 +10148|-|7205335.997497559 +8860|-|7202401.369445801 +8373|-|7189039.683227539 +10859|-|7188990.444030762 +12670|-|7188177.08215332 +2553|-|7180006.185241699 +19346|-|7176797.25 +1681|-|7160169.945556641 +15225|-|7158861.182128906 +1052|-|7158586.096191406 +77|-|7155531.239624023 +7231|-|7155250.423706055 +12622|-|7137408.324859619 +9814|-|7105363.344665527 +8695|-|7103187.0 +13174|-|7099182.366638184 +6179|-|7095133.949645996 +6451|-|7082495.4892578125 +19860|-|7073206.925537109 +9307|-|7059973.657470703 +7819|-|7055963.275634766 +10556|-|7053491.228027344 +9366|-|7048690.602905273 +12124|-|7040021.477142334 +11476|-|7037906.622802734 +19245|-|7034045.488037109 +7562|-|7030275.7380981445 +12290|-|7020371.895080566 +18118|-|7003396.701599121 +1253|-|7001569.64251709 +8662|-|6999834.2888793945 +1779|-|6997385.874389648 +15386|-|6996871.75869751 +147|-|6989079.989013672 +9562|-|6983076.9216918945 +3958|-|6969833.342041016 +7211|-|6966606.608703613 +12842|-|6923277.385986328 +4368|-|6918784.0388793945 +11131|-|6918656.511962891 +4628|-|6894893.838684082 +5879|-|6881367.098999023 +16586|-|6865931.921081543 +32|-|6852925.70413208 +12119|-|6842773.790405273 +1371|-|6831137.637084961 +6136|-|6827917.031799316 +13857|-|6824240.778198242 +1074|-|6821748.004943848 +1863|-|6821522.124938965 +14597|-|6817385.703491211 +9271|-|6783068.896728516 +6389|-|6781075.729034424 +11703|-|6776538.446533203 +13701|-|6768880.780517578 +14880|-|6763788.366943359 +18428|-|6763670.798339844 +1006|-|6762065.771118164 +4927|-|6756764.979553223 +11659|-|6755246.622680664 +6815|-|6738928.461853027 +14367|-|6733856.956298828 +10703|-|6730936.196655273 +3150|-|6727920.229980469 +9963|-|6725919.502197266 +10438|-|6710153.687133789 +4745|-|6682153.768188477 +15297|-|6681711.262207031 +10848|-|6678666.102096558 +11749|-|6677895.779327393 +16739|-|6675548.987792969 +14915|-|6672248.951416016 +19841|-|6669191.291503906 +628|-|6666893.0322265625 +1650|-|6657178.3419799805 +7069|-|6648672.3673706055 +7108|-|6646445.7658081055 +8887|-|6641656.088623047 +18735|-|6636457.606201172 +3711|-|6632665.1982421875 +2325|-|6630475.801025391 +6484|-|6622964.988525391 +2643|-|6617304.879394531 +7895|-|6615061.113830566 +12840|-|6604848.704223633 +4314|-|6600905.513122559 +19077|-|6591092.154846191 +17476|-|6576028.791809082 +7039|-|6559838.718261719 +8106|-|6558223.037231445 +2391|-|6557120.103363037 +7717|-|6547706.90234375 +12476|-|6546988.352416992 +9155|-|6540793.7138671875 +360|-|6530297.499755859 +6383|-|6529336.0283203125 +4830|-|6518998.776855469 +12600|-|6511549.268859863 +13740|-|6508057.797363281 +4678|-|6507848.167053223 +17815|-|6502284.585021973 +7329|-|6490811.854248047 +13884|-|6490063.071685791 +5147|-|6487069.189453125 +16548|-|6482024.5 +14144|-|6476413.493041992 +10181|-|6474985.073974609 +11031|-|6463307.950134277 +19958|-|6461506.50402832 +6043|-|6458177.505554199 +10060|-|6455476.871154785 +9144|-|6454041.959655762 +7043|-|6448019.869750977 +17346|-|6444307.572265625 +13963|-|6442014.463745117 +7111|-|6441946.990966797 +14140|-|6439955.426147461 +2327|-|6438977.224365234 +14812|-|6437152.762390137 +16755|-|6430895.072143555 +14840|-|6430548.970031738 +14134|-|6422079.210327148 +12655|-|6405496.899597168 +1518|-|6390148.062011719 +9888|-|6385032.905639648 +7387|-|6384005.2341918945 +5393|-|6381083.167236328 +11057|-|6375974.42565918 +16818|-|6368828.832763672 +4576|-|6364925.991455078 +19644|-|6350000.580749512 +139|-|6336065.414428711 +11735|-|6334305.803833008 +10108|-|6332055.9845581055 +15628|-|6329801.422363281 +4349|-|6314949.513183594 +7048|-|6313868.6720581055 +17119|-|6298935.465454102 +18597|-|6291416.160644531 +2488|-|6286371.93951416 +2296|-|6275519.485473633 +4343|-|6272834.280883789 +9958|-|6267714.385986328 +2147|-|6267475.0986328125 +9368|-|6254578.868408203 +13134|-|6252432.635620117 +10119|-|6251456.25 +2093|-|6249342.516357422 +2392|-|6237476.778442383 +17231|-|6233509.230987549 +3773|-|6226654.890014648 +9839|-|6214044.281005859 +19349|-|6213969.458251953 +7869|-|6212751.0 +8158|-|6210279.509033203 +13179|-|6205773.454467773 +2349|-|6205589.162841797 +9052|-|6200729.778686523 +1744|-|6189967.235534668 +597|-|6183103.373657227 +16721|-|6176606.507568359 +7498|-|6176277.166748047 +15157|-|6176222.5 +17524|-|6171107.169433594 +7922|-|6170906.161987305 +6615|-|6170738.276489258 +2706|-|6165991.482055664 +16432|-|6162740.631591797 +16473|-|6162428.176025391 +12825|-|6161595.536499023 +18813|-|6154678.713745117 +1030|-|6146500.111328125 +2571|-|6145772.685852051 +7707|-|6144754.875549316 +10327|-|6137611.892089844 +4710|-|6132346.763671875 +3649|-|6130602.564208984 +4893|-|6128461.310058594 +12844|-|6128191.5185546875 +8794|-|6122690.351074219 +1157|-|6117749.034912109 +10895|-|6112017.556884766 +16166|-|6108251.213562012 +11920|-|6107122.505859375 +5621|-|6102123.578979492 +6141|-|6093826.577270508 +14076|-|6086671.184692383 +15884|-|6080485.499755859 +4814|-|6080337.751464844 +5814|-|6079842.868408203 +1134|-|6078685.3818359375 +19048|-|6071813.099853516 +303|-|6070601.556518555 +15211|-|6065678.1463012695 +1020|-|6054168.1033325195 +11292|-|6052522.887268066 +7134|-|6049865.085952759 +14119|-|6049619.34475708 +2389|-|6042428.9326171875 +5926|-|6034269.631225586 +8553|-|6030922.882080078 +18814|-|6023255.409912109 +12702|-|6023191.234680176 +2644|-|6020931.110595703 +19628|-|6010704.0 +18112|-|6008707.6396484375 +13860|-|6008170.15851593 +1456|-|6005092.087463379 +1633|-|6002068.720092773 +2301|-|6000302.08404541 +10740|-|5999808.927612305 +2630|-|5997008.380023956 +8818|-|5992297.044067383 +10043|-|5990594.109863281 +653|-|5987942.619018555 +6829|-|5985990.5419921875 +15179|-|5977727.351074219 +9663|-|5973524.063964844 +5863|-|5973328.962524414 +3628|-|5966339.861572266 +7618|-|5960156.128417969 +2588|-|5952648.4501953125 +4865|-|5949383.547363281 +4233|-|5944699.3798828125 +13390|-|5944104.823295593 +16321|-|5942714.792358398 +9653|-|5941308.318481445 +18884|-|5925548.072265625 +4394|-|5920927.341247559 +19774|-|5916723.156738281 +1257|-|5914052.5021362305 +2963|-|5911917.549072266 +17157|-|5899572.872558594 +383|-|5884693.19140625 +11709|-|5884134.3427734375 +18135|-|5871431.557067871 +13169|-|5869736.765258789 +2932|-|5868995.733038902 +2888|-|5863229.089355469 +6820|-|5853106.897949219 +18622|-|5850951.483764648 +9066|-|5846052.5634765625 +19840|-|5832251.279296875 +6325|-|5827298.359130859 +14092|-|5823711.827819824 +11532|-|5823277.076904297 +18043|-|5815248.878662109 +3932|-|5809134.717407227 +10364|-|5808371.517333984 +1179|-|5808303.366882324 +11441|-|5799378.315124512 +15767|-|5798263.213623047 +14063|-|5797203.8330078125 +11383|-|5793893.076416016 +10065|-|5781764.231262207 +17785|-|5766838.923736572 +18349|-|5761765.970947266 +14824|-|5760339.707763672 +14699|-|5759490.3134765625 +11628|-|5755137.299621582 +4403|-|5752376.759765625 +13090|-|5751949.335632324 +15332|-|5744606.4169921875 +17573|-|5744383.947875977 +12895|-|5741007.202209473 +13577|-|5739210.0849609375 +16594|-|5732523.799926758 +8497|-|5727509.883544922 +2872|-|5724068.866699219 +16181|-|5721787.9775390625 +6639|-|5712041.176513672 +13095|-|5708798.25 +4535|-|5693341.342163086 +10956|-|5692585.5 +19755|-|5686913.407043457 +12995|-|5682200.377685547 +13157|-|5681454.881713867 +1887|-|5681341.26373291 +18472|-|5680356.0 +19814|-|5679360.15625 +18583|-|5669360.568237305 +3894|-|5664385.625732422 +1740|-|5659552.071777344 +62|-|5659470.315979004 +16532|-|5653779.597351074 +995|-|5648973.3865356445 +7486|-|5646851.936279297 +19007|-|5642710.911071777 +13060|-|5642024.591308594 +12371|-|5635710.356678009 +2280|-|5634077.596435547 +3115|-|5631352.335388184 +11107|-|5631252.5623168945 +5873|-|5629125.75 +14743|-|5628732.584472656 +2224|-|5624746.758911133 +2653|-|5623860.028869629 +17622|-|5623057.4638671875 +14700|-|5615343.8232421875 +14740|-|5613929.609283447 +6403|-|5611924.4900512695 +6896|-|5609169.838256836 +10384|-|5607337.7080078125 +16433|-|5605706.863952637 +5898|-|5604937.6337890625 +4789|-|5600488.560180664 +8132|-|5593107.5 +3838|-|5592313.270019531 +13631|-|5586424.658203125 +11233|-|5585247.120605469 +849|-|5583516.667358398 +14653|-|5581550.6408081055 +14788|-|5580433.0 +18181|-|5578562.896972656 +19815|-|5577102.77746582 +5584|-|5576692.219848633 +5385|-|5576420.109619141 +13780|-|5569028.450683594 +9342|-|5566783.749694824 +19056|-|5566523.983154297 +8189|-|5565694.551269531 +13808|-|5560721.831542969 +10635|-|5560058.317565918 +8304|-|5550784.243652344 +14257|-|5549164.2529296875 +8999|-|5542100.162658691 +19134|-|5539312.38583374 +8360|-|5538031.209716797 +6397|-|5536651.866760254 +2597|-|5525317.7001953125 +8631|-|5515909.300048828 +16729|-|5512663.717041016 +11861|-|5511785.877807617 +16853|-|5511690.148681641 +6341|-|5502790.0634765625 +1312|-|5496649.234619141 +5566|-|5495885.978881836 +12519|-|5490649.905700684 +19032|-|5488104.884399414 +8231|-|5479312.5 +3026|-|5466732.1748046875 +6388|-|5466168.759765625 +15349|-|5464571.4921875 +18985|-|5463897.258300781 +19848|-|5454266.482177734 +17378|-|5453284.826416016 +4000|-|5448690.299926758 +3710|-|5445822.699645996 +13181|-|5439774.144287109 +6420|-|5438325.193359375 +3644|-|5437772.141098022 +1117|-|5437024.868469238 +13027|-|5436968.577270508 +12884|-|5432632.476566315 +12781|-|5429160.972290039 +10084|-|5428231.571533203 +5640|-|5423318.725097656 +16208|-|5422901.193969727 +753|-|5416899.832763672 +4120|-|5413822.516418457 +12348|-|5412061.74206543 +1586|-|5411198.56842041 +2503|-|5411125.597412109 +1381|-|5397789.987182617 +19673|-|5397746.727233887 +19389|-|5394426.345214844 +15532|-|5386361.202636719 +3562|-|5380335.203491211 +19114|-|5375132.916870117 +3204|-|5372215.437072754 +6653|-|5365178.0 +3553|-|5363690.341247559 +12687|-|5361682.239746094 +3279|-|5357505.546936035 +9840|-|5350112.194030762 +8258|-|5347064.642700195 +11387|-|5345284.0 +15210|-|5341117.935058594 +15092|-|5340896.915283203 +6052|-|5339762.264892578 +14043|-|5339587.5546875 +6284|-|5336153.102111816 +6297|-|5332357.802307129 +16254|-|5326607.86340332 +18436|-|5326198.32824707 +14436|-|5325517.143447876 +10368|-|5319982.969528198 +6349|-|5317943.289794922 +19984|-|5317619.8974609375 +19080|-|5310678.432128906 +1403|-|5306771.419189453 +5995|-|5305056.170654297 +13214|-|5299420.238037109 +19293|-|5297054.296508789 +7130|-|5289761.362792969 +9809|-|5286784.634033203 +9273|-|5277924.529724121 +16786|-|5260035.0 +11032|-|5256354.57409668 +17719|-|5246322.274169922 +3400|-|5244203.6376953125 +8278|-|5243718.6767578125 +7873|-|5241168.8916015625 +9357|-|5239695.61920166 +2239|-|5235767.694152832 +18695|-|5223960.379364014 +19271|-|5216498.215576172 +4563|-|5214673.551635742 +1896|-|5210699.938537598 +15755|-|5210481.369506836 +17628|-|5209882.317260742 +5747|-|5206892.341796875 +18057|-|5204907.1826171875 +5075|-|5204413.011657715 +5325|-|5190162.620544434 +17675|-|5183135.307250977 +18818|-|5181295.010986328 +11064|-|5180667.3986206055 +12578|-|5177222.121520996 +3725|-|5169183.75 +11112|-|5161545.787353516 +4749|-|5161499.3388671875 +7632|-|5159180.598449707 +13119|-|5151665.577392578 +5653|-|5140324.559326172 +16134|-|5137482.491760254 +5671|-|5136815.460144043 +18348|-|5132980.987426758 +562|-|5132487.530517578 +2562|-|5126763.857269287 +10304|-|5125180.0 +7622|-|5120719.940734863 +12755|-|5118213.738769531 +1814|-|5100705.401733398 +11269|-|5096671.481262207 +2964|-|5095253.5595703125 +1616|-|5091834.1748046875 +9294|-|5090753.442626953 +16793|-|5085330.739562988 +4999|-|5081651.661071777 +17209|-|5079029.167480469 +7151|-|5078937.689697266 +15522|-|5072469.46105957 +3056|-|5072329.481201172 +7612|-|5068322.744506836 +18453|-|5063893.064819336 +18324|-|5058901.131103516 +12266|-|5058186.75 +19394|-|5056235.616027832 +1713|-|5054967.864990234 +15681|-|5051569.63961792 +8274|-|5043328.125 +18160|-|5043074.653747559 +18253|-|5041571.786499023 +11840|-|5040589.7724609375 +1532|-|5033171.0 +584|-|5031602.589111328 +12382|-|5028900.929260254 +14814|-|5022200.2208862305 +19058|-|5019001.76953125 +4487|-|5016640.704345703 +8482|-|5015444.25 +18476|-|5011136.218017578 +12335|-|5003581.307373047 +4455|-|4997933.337524414 +14355|-|4992823.065795898 +15253|-|4992641.995605469 +14069|-|4983244.096069336 +17843|-|4977294.352844238 +9389|-|4975885.794708252 +14435|-|4971442.158325195 +13254|-|4959481.276245117 +9773|-|4955887.912719727 +7615|-|4952421.346801758 +6476|-|4947250.132507324 +9253|-|4945159.7673950195 +14932|-|4934395.533935547 +13253|-|4932867.584838867 +19322|-|4931525.743225098 +16945|-|4931440.647094727 +731|-|4930191.910827637 +6540|-|4925114.492492676 +5148|-|4923047.790527344 +1934|-|4921197.092590332 +15402|-|4920840.825439453 +17914|-|4919607.1953125 +5416|-|4916041.8515625 +16734|-|4914205.101623535 +14967|-|4900262.231689453 +8706|-|4894595.676269531 +136|-|4891960.934814453 +19494|-|4886028.212036133 +8737|-|4880640.568405151 +7653|-|4879423.8212890625 +4149|-|4875782.34375 +7890|-|4872424.212402344 +11142|-|4871415.393676758 +10386|-|4863623.5 +8603|-|4861814.816345215 +2680|-|4861223.819928408 +4891|-|4858103.90246582 +19236|-|4855097.594238281 +14251|-|4854739.705566406 +18895|-|4853365.020751953 +17134|-|4852040.909912109 +4932|-|4843701.620727539 +10033|-|4841647.833496094 +1383|-|4839144.297821045 +18721|-|4837983.256896973 +8618|-|4833125.3302612305 +17386|-|4831545.862121582 +3790|-|4830282.442749023 +1043|-|4825921.49810791 +12434|-|4822597.65222168 +18385|-|4819643.560058594 +6046|-|4817460.103637695 +5821|-|4814423.387390137 +10836|-|4814303.368774414 +6848|-|4813954.126464844 +6880|-|4804600.252258301 +11249|-|4800116.750732422 +11970|-|4799739.897216797 +14253|-|4796521.322387695 +7782|-|4793227.303466797 +75|-|4790042.787597656 +7076|-|4789347.4599609375 +9566|-|4782531.823425293 +2137|-|4767931.627807617 +2336|-|4763870.767028809 +15362|-|4759043.4904174805 +3284|-|4755048.849243164 +12964|-|4753627.362182617 +1781|-|4752835.270996094 +5454|-|4748342.825683594 +12597|-|4742077.73449707 +19120|-|4733459.977172852 +14884|-|4731499.487548828 +825|-|4730720.185546875 +14683|-|4730482.173583984 +5361|-|4726113.098144531 +12179|-|4725018.062133789 +1461|-|4710954.750732422 +9890|-|4709658.477355957 +13369|-|4705085.355682373 +11612|-|4701627.890991211 +3096|-|4699414.604003906 +10383|-|4697866.486999512 +11293|-|4697335.990661621 +3383|-|4695825.221252441 +6231|-|4694381.618530273 +7396|-|4691319.1111450195 +17827|-|4688797.546325684 +15856|-|4683368.358215332 +8253|-|4678560.997009277 +12327|-|4677984.817504883 +4395|-|4676829.9169921875 +4232|-|4676646.528808594 +14260|-|4670522.9169921875 +15288|-|4669274.073791504 +17526|-|4668545.5458984375 +9884|-|4662693.873291016 +2118|-|4660352.82800293 +4524|-|4653956.5236206055 +19090|-|4650873.041564941 +3928|-|4649359.626403809 +14325|-|4647762.154724121 +15476|-|4643469.148864746 +4179|-|4639931.806640625 +14408|-|4639631.101318359 +19424|-|4634817.4169921875 +3334|-|4633102.5 +9477|-|4628073.531555176 +11803|-|4625396.892623901 +14805|-|4618452.125610352 +463|-|4616307.090332031 +16628|-|4607490.821777344 +3116|-|4604462.978942871 +19962|-|4602949.520690918 +12859|-|4602870.621398926 +12063|-|4600708.30090332 +5648|-|4592273.286437988 +8556|-|4590726.698730469 +15281|-|4589425.466552734 +9414|-|4587426.94152832 +13951|-|4586281.25 +19328|-|4582624.795043945 +15963|-|4579705.430297852 +10773|-|4573276.243286133 +14179|-|4568815.91796875 +1895|-|4563988.246765137 +6408|-|4561496.422485352 +5958|-|4554000.0 +3653|-|4548134.5458984375 +11218|-|4546237.9326171875 +19327|-|4543987.849060059 +9572|-|4535941.016967773 +14556|-|4531464.75 +2475|-|4529761.576538086 +9631|-|4529261.529052734 +1901|-|4528592.450134277 +86|-|4528475.542480469 +9586|-|4527146.243469238 +17361|-|4519098.917480469 +8112|-|4514949.5333862305 +13468|-|4499728.176269531 +18239|-|4497633.723449707 +10215|-|4494553.505859375 +6211|-|4492265.11328125 +836|-|4490945.196228027 +895|-|4489141.567993164 +19542|-|4488393.684387207 +4322|-|4487884.096252441 +2116|-|4486944.625854492 +553|-|4486075.287902832 +2515|-|4485188.394104004 +16286|-|4481470.502471924 +12271|-|4478224.889465332 +16570|-|4465818.176879883 +7995|-|4457574.602233887 +18396|-|4457229.7998046875 +16331|-|4455735.341308594 +18157|-|4452196.66998291 +5271|-|4452040.137268066 +11622|-|4451244.733520508 +4052|-|4446397.302978516 +2864|-|4446008.511779785 +490|-|4442892.223022461 +19837|-|4434172.495391846 +4114|-|4433657.825469971 +11436|-|4433070.259918213 +6085|-|4431306.438903809 +9735|-|4430445.496154785 +17834|-|4416286.344665527 +8157|-|4416116.683654785 +18840|-|4414925.303222656 +13553|-|4412261.64642334 +12562|-|4411183.010620117 +14025|-|4403441.9482421875 +17964|-|4400360.095458984 +636|-|4399863.7763671875 +8390|-|4389024.43762207 +231|-|4387397.207641602 +9699|-|4385890.995589256 +10622|-|4384005.409423828 +14364|-|4383236.970153809 +10580|-|4381533.211486816 +10124|-|4369801.018798828 +10451|-|4368867.477416992 +4673|-|4367113.59765625 +11351|-|4362616.5 +4770|-|4362397.307067871 +12932|-|4362042.668457031 +10603|-|4357216.558532715 +19733|-|4348931.767272949 +4222|-|4348871.946166992 +17319|-|4347687.793029785 +3375|-|4346529.495361328 +14995|-|4338295.762023926 +7675|-|4337499.50592041 +15043|-|4333921.116943359 +4835|-|4332648.0 +4408|-|4332588.947753906 +5559|-|4330577.172637939 +7376|-|4328936.4228515625 +18061|-|4328793.885620117 +2749|-|4328671.324951172 +6628|-|4328501.69720459 +5888|-|4323049.6622924805 +18872|-|4322595.4638671875 +5476|-|4319642.427246094 +1755|-|4318935.650268555 +10623|-|4315822.5732421875 +18775|-|4314677.421386719 +3570|-|4312697.793701172 +11147|-|4310740.499450684 +6071|-|4307612.346496582 +10807|-|4306006.0 +9550|-|4299478.715332031 +657|-|4296794.158325195 +19669|-|4294640.777954102 +8532|-|4290651.657989502 +13469|-|4281715.703979492 +8809|-|4280778.874511719 +11301|-|4276847.8777318 +6147|-|4266879.889434814 +2612|-|4265962.322387695 +15699|-|4256118.787719727 +12300|-|4254409.009460449 +3494|-|4250810.780517578 +11040|-|4250030.305297852 +6190|-|4244046.669616699 +17616|-|4239937.591552734 +7271|-|4234407.0 +14048|-|4226977.31640625 +4456|-|4224684.941772461 +10012|-|4223841.397155762 +11175|-|4223704.178924561 +18675|-|4215406.742797852 +10792|-|4214898.596313477 +10806|-|4209678.513305664 +18749|-|4204787.157592773 +17410|-|4198025.459472656 +8032|-|4195429.946899414 +11094|-|4192304.7974243164 +17582|-|4187341.4150390625 +12246|-|4183231.0021362305 +6640|-|4182968.7302246094 +7346|-|4174707.446411133 +12747|-|4169865.9834594727 +3869|-|4164957.48828125 +13106|-|4161901.9420776367 +10547|-|4159541.3818359375 +15289|-|4156205.864135742 +1679|-|4156156.8388671875 +1126|-|4155593.256286621 +19106|-|4147439.337890625 +9705|-|4144024.211730957 +15324|-|4142518.4815063477 +16544|-|4140375.5946655273 +8812|-|4139322.6481933594 +10772|-|4134101.6876220703 +2800|-|4127149.953125 +15549|-|4124704.596069336 +3607|-|4118697.649291992 +1980|-|4117633.7825927734 +214|-|4113117.2513427734 +19217|-|4104217.701171875 +2460|-|4098577.6850585938 +19156|-|4093864.5733032227 +18359|-|4092727.219848633 +12865|-|4092526.7197875977 +14616|-|4092434.464111328 +908|-|4088856.1779785156 +11791|-|4083804.8350219727 +4157|-|4078345.5336914062 +3857|-|4070873.078857422 +15114|-|4056112.449645996 +395|-|4052997.6689453125 +17456|-|4051457.26171875 +10562|-|4050894.0726013184 +10884|-|4050330.8107910156 +12177|-|4049842.648864746 +15595|-|4040577.5778808594 +15916|-|4036044.5 +7084|-|4035102.84375 +4424|-|4034761.604736328 +10874|-|4031015.730895996 +4740|-|4030403.712890625 +16585|-|4030010.2072143555 +18824|-|4028984.1595458984 +14875|-|4028452.0900878906 +13855|-|4024828.3287963867 +10932|-|4024002.3693847656 +9084|-|4021362.614135742 +14352|-|4018089.609741211 +18086|-|4015180.668823242 +9514|-|4013666.597351074 +15787|-|4013154.5744628906 +714|-|4010249.4217529297 +8811|-|4009588.8416137695 +14386|-|4007210.970336914 +616|-|4004057.293701172 +7460|-|4003412.515625 +866|-|4003182.570739746 +782|-|4001300.0307998657 +8562|-|3999441.6299438477 +1366|-|3994061.0306396484 +2879|-|3993056.5740966797 +16679|-|3992434.9600219727 +17306|-|3990723.359069824 +13140|-|3982817.3999328613 +17942|-|3980857.1481933594 +6572|-|3977676.2143554688 +3578|-|3977523.9614868164 +15802|-|3969947.017211914 +336|-|3967938.2634277344 +9807|-|3964469.5294189453 +12104|-|3964273.4399414062 +4271|-|3962359.2583007812 +6702|-|3961657.408721924 +19763|-|3955582.75 +369|-|3953702.748779297 +4089|-|3953455.662963867 +2593|-|3946153.8842773438 +590|-|3943841.3459472656 +8325|-|3942118.637084961 +158|-|3941881.6287231445 +12054|-|3938362.7642211914 +18330|-|3938303.769104004 +5354|-|3936239.644958496 +8150|-|3925793.3950195312 +8344|-|3921293.4858398438 +6069|-|3921130.6686401367 +4032|-|3920008.510986328 +17939|-|3917750.340209961 +7014|-|3914471.029296875 +2840|-|3913131.778076172 +1868|-|3912987.7229003906 +10975|-|3911920.4897460938 +5374|-|3910802.7037963867 +11128|-|3908156.506958008 +18449|-|3907589.2485351562 +11740|-|3907459.8506774902 +2356|-|3907189.059753418 +5721|-|3901586.026916504 +4231|-|3900779.1184692383 +4352|-|3899933.2465820312 +432|-|3899836.372741699 +15321|-|3899516.5906677246 +10296|-|3897015.3104248047 +5647|-|3895088.204345703 +7386|-|3891916.6166381836 +507|-|3891487.7563476562 +3995|-|3887387.045715332 +4278|-|3882293.9220581055 +18407|-|3880267.991455078 +6127|-|3879166.7860412598 +145|-|3875277.2717285156 +19269|-|3874685.8342285156 +18257|-|3874454.983581543 +9068|-|3869767.7082366943 +576|-|3860007.8736572266 +4860|-|3852861.904525757 +18793|-|3849838.076171875 +15988|-|3847257.015991211 +6891|-|3846386.748565674 +3231|-|3846344.2056274414 +15237|-|3845420.9197998047 +9035|-|3844166.9897460938 +7597|-|3838643.271270752 +16349|-|3837121.6875 +2497|-|3827850.2641296387 +3616|-|3827390.8569335938 +11566|-|3826122.459777832 +18403|-|3822033.0950927734 +2972|-|3821903.605041504 +812|-|3821523.6330566406 +2043|-|3820561.3869628906 +505|-|3818922.0415649414 +8257|-|3815071.929748535 +6084|-|3814195.057220459 +11253|-|3813917.1293945312 +366|-|3812257.971801758 +13632|-|3811601.4802246094 +14298|-|3801412.4647216797 +7092|-|3798729.458129883 +2058|-|3796109.107772827 +14820|-|3791195.745300293 +7157|-|3788690.8326416016 +17211|-|3786030.133605957 +16644|-|3786019.143676758 +15693|-|3783662.0514526367 +2627|-|3782394.4555664062 +11231|-|3782077.5537109375 +12696|-|3781761.7524414062 +8705|-|3778076.8936157227 +16052|-|3771577.163696289 +99|-|3760269.2545776367 +2082|-|3757517.6696777344 +872|-|3750005.232727051 +7126|-|3749138.9309692383 +10302|-|3744475.25 +17122|-|3741012.9901123047 +10080|-|3740106.9815063477 +16021|-|3739611.263671875 +3074|-|3739225.025390625 +3142|-|3738811.042541504 +13213|-|3735116.25 +13442|-|3733132.125854492 +11542|-|3731000.0503845215 +13732|-|3730444.8372802734 +2608|-|3729372.3095703125 +5|-|3725511.3647460938 +19157|-|3723844.697265625 +18231|-|3721707.90524292 +8179|-|3714155.082885742 +12740|-|3708646.8178710938 +11597|-|3706528.71774292 +13968|-|3702376.100830078 +6436|-|3687346.4084472656 +9181|-|3687134.093383789 +564|-|3680200.9166259766 +13464|-|3678406.3126220703 +14084|-|3673790.3356933594 +2755|-|3670593.7201538086 +14284|-|3668640.6528320312 +12178|-|3653392.494445801 +15730|-|3650258.239379883 +5560|-|3649569.574645996 +8594|-|3647140.6193847656 +7032|-|3646439.4858398438 +16846|-|3644843.203063965 +1530|-|3642837.938659668 +3978|-|3639712.0220947266 +2897|-|3639442.5087890625 +16625|-|3636527.7016601562 +12029|-|3636339.9001464844 +16830|-|3633448.560974121 +9597|-|3632662.0404052734 +5533|-|3630338.7088623047 +5181|-|3625965.9406280518 +8131|-|3625738.5552978516 +8560|-|3620761.3450012207 +11860|-|3618746.1837768555 +12008|-|3614604.3872070312 +10737|-|3611990.6669311523 +18208|-|3611596.2127685547 +5119|-|3611038.0553474426 +11958|-|3601654.538269043 +15124|-|3598278.3068847656 +14058|-|3597490.1450195312 +12270|-|3593912.0416259766 +17793|-|3593318.9062805176 +9385|-|3587327.7302246094 +12814|-|3587083.7614746094 +5304|-|3586230.7133483887 +3631|-|3582841.7126464844 +610|-|3581917.384033203 +19317|-|3580412.2911987305 +128|-|3567004.4588623047 +11616|-|3566154.6940307617 +10176|-|3565392.118835449 +7349|-|3564110.6279907227 +1712|-|3560408.479248047 +18860|-|3559340.655517578 +17617|-|3557516.0 +6443|-|3556296.9904174805 +15408|-|3554814.438720703 +16350|-|3554388.6864624023 +17436|-|3554105.249267578 +5740|-|3551324.6645202637 +12181|-|3550218.471008301 +16895|-|3550119.3099975586 +19995|-|3548839.637451172 +4968|-|3548306.836517334 +2257|-|3546692.233581543 +1825|-|3543198.6877441406 +18989|-|3539038.189453125 +18727|-|3536081.463623047 +16165|-|3533789.7639160156 +3249|-|3533709.896118164 +11731|-|3532874.9084472656 +13032|-|3532415.8079223633 +9377|-|3531582.0412597656 +5883|-|3531479.0 +1211|-|3528833.3890686035 +12065|-|3526948.2104492188 +10866|-|3526146.7030029297 +2073|-|3520131.346557617 +2378|-|3512186.1069335938 +16860|-|3509693.15826416 +389|-|3507814.5942382812 +15604|-|3505653.3858947754 +11257|-|3502831.875854492 +1327|-|3502022.69329834 +16602|-|3501074.7788085938 +1493|-|3498809.083404541 +8224|-|3498179.6228027344 +622|-|3497158.3435668945 +3072|-|3495958.7485351562 +1478|-|3494880.3910217285 +3125|-|3494169.7932128906 +2052|-|3488438.0466308594 +8476|-|3487191.348876953 +10735|-|3477740.830078125 +14860|-|3476235.7895507812 +6586|-|3475745.057647705 +5130|-|3472024.5 +7181|-|3471306.362182617 +618|-|3467906.4907226562 +15698|-|3464859.514404297 +17585|-|3462450.507598877 +2548|-|3456856.8356933594 +2632|-|3456230.7512817383 +2882|-|3453986.901489258 +12216|-|3452907.1087646484 +4925|-|3452904.7536621094 +9012|-|3442581.2966308594 +6667|-|3430076.3749694824 +17958|-|3424962.4669189453 +6093|-|3424241.86328125 +10648|-|3417414.0 +1462|-|3413248.7124938965 +2569|-|3412388.917602539 +18616|-|3409880.9376831055 +7368|-|3408036.5661621094 +3110|-|3407374.7145996094 +10824|-|3406819.280822754 +11510|-|3404702.100830078 +4840|-|3397236.5057373047 +4449|-|3396993.7866210938 +1358|-|3396616.348388672 +3885|-|3395817.509765625 +13381|-|3391953.4838867188 +1655|-|3383051.5503845215 +282|-|3381785.3560180664 +4928|-|3374270.3647460938 +3199|-|3372488.6806030273 +16086|-|3370710.6073303223 +8612|-|3362922.5 +19597|-|3360763.9086914062 +8867|-|3354400.0017700195 +4098|-|3353574.403076172 +12617|-|3351499.0365600586 +14365|-|3347296.0 +10443|-|3345493.17388916 +76|-|3342081.729309082 +11585|-|3341941.1455078125 +4383|-|3338960.174133301 +13910|-|3335964.0290527344 +8076|-|3332449.9939575195 +16005|-|3332190.3076171875 +2622|-|3329364.5013427734 +12822|-|3321183.612060547 +17076|-|3320398.0513305664 +5392|-|3320357.1685791016 +18628|-|3319615.9897460938 +13695|-|3318525.9436035156 +10326|-|3318274.172973633 +9109|-|3317833.9587402344 +1489|-|3317620.6997070312 +3378|-|3315948.0 +7738|-|3312979.1337890625 +1844|-|3312277.341064453 +19963|-|3307500.0 +2436|-|3306419.005218506 +886|-|3302180.7923583984 +15475|-|3301693.5424804688 +6327|-|3300680.8302001953 +6050|-|3299460.2420043945 +9876|-|3298409.9957580566 +19586|-|3291131.25 +14349|-|3289862.5565185547 +10993|-|3287980.591003418 +18784|-|3286752.163330078 +1800|-|3285466.403442383 +990|-|3284595.432281494 +3823|-|3281992.824279785 +15737|-|3279305.9416503906 +19518|-|3276759.656616211 +9032|-|3272440.3560791016 +7786|-|3271217.2989501953 +8648|-|3271162.5646362305 +5532|-|3270188.0427856445 +15914|-|3268521.0692749023 +16065|-|3265068.950805664 +11212|-|3264657.0001831055 +13229|-|3262022.2408447266 +15827|-|3260862.626953125 +1582|-|3260340.0 +3827|-|3260093.8667907715 +3546|-|3259243.9357910156 +15849|-|3258918.0603027344 +14856|-|3258379.358276367 +2028|-|3255014.0568847656 +6618|-|3254581.9054260254 +17461|-|3252926.8703918457 +13551|-|3241602.299560547 +19561|-|3239795.201904297 +2276|-|3236172.3681640625 +14203|-|3234649.4604492188 +7757|-|3231351.8129882812 +122|-|3226214.0131225586 +12954|-|3225943.0 +647|-|3224783.795288086 +12383|-|3223989.3447265625 +3831|-|3223126.5568847656 +16836|-|3222260.69430542 +4565|-|3221597.5703125 +19426|-|3218106.4317626953 +17855|-|3217813.072998047 +5624|-|3207777.2919921875 +8368|-|3203376.54964447 +9480|-|3200904.0467834473 +11181|-|3199500.6275177 +8981|-|3197864.0 +16426|-|3195995.9939575195 +1648|-|3195558.815185547 +14404|-|3192729.71875 +17867|-|3188571.1111450195 +18117|-|3183229.0297851562 +14289|-|3182261.478149414 +53|-|3182255.889892578 +15546|-|3180180.031311035 +16245|-|3178277.436401367 +1597|-|3176247.4902648926 +1653|-|3173456.7607421875 +2845|-|3171619.5626220703 +15906|-|3171187.61819458 +18304|-|3168571.456604004 +14068|-|3167367.6510620117 +6837|-|3165012.5779418945 +9446|-|3164446.4759521484 +18889|-|3156140.987121582 +16587|-|3154210.2822875977 +7705|-|3152977.2998657227 +1120|-|3151591.1345214844 +17665|-|3148848.0 +5311|-|3146721.798095703 +14157|-|3144707.3118286133 +7996|-|3131350.9814453125 +8663|-|3130526.3837890625 +18271|-|3127800.8067626953 +6446|-|3125685.914794922 +6972|-|3125007.0802001953 +2572|-|3123186.7198791504 +13536|-|3122527.621398926 +6196|-|3122172.51171875 +9338|-|3121262.5478515625 +11992|-|3118647.639770508 +2580|-|3118284.4561157227 +9098|-|3117494.163696289 +5118|-|3112661.9350585938 +10184|-|3109293.5043945312 +9932|-|3105818.1396484375 +18545|-|3102273.3803100586 +10963|-|3099314.6447753906 +8405|-|3097121.0288085938 +9037|-|3095194.9768066406 +179|-|3091107.3139648438 +1930|-|3090915.8635253906 +17723|-|3090624.7774658203 +4308|-|3089472.851257324 +8702|-|3080129.96484375 +18621|-|3079984.7451171875 +4501|-|3079781.0495910645 +3590|-|3079049.5182495117 +18264|-|3078858.335357666 +15648|-|3078564.2014160156 +5998|-|3073264.0 +16904|-|3072610.725402832 +3794|-|3071333.0126342773 +3147|-|3068485.215576172 +17221|-|3068337.2501220703 +4709|-|3067523.235107422 +18017|-|3066743.440551758 +15613|-|3063987.932067871 +16271|-|3057051.243041992 +13621|-|3054774.5126342773 +12919|-|3054518.4603881836 +12493|-|3050836.1889648438 +15838|-|3050645.8373413086 +3273|-|3048955.118408203 +8324|-|3046011.25 +13628|-|3045324.4122314453 +5522|-|3044408.5 +2202|-|3043131.962585449 +19052|-|3042566.4797973633 +5767|-|3041871.7602539062 +17895|-|3036452.231323242 +12586|-|3036386.393737793 +12425|-|3035041.5849609375 +13517|-|3034351.600341797 +2363|-|3033336.546936035 +15060|-|3032598.603790283 +6764|-|3032591.1453552246 +340|-|3030522.1069335938 +4723|-|3028910.2111816406 +3566|-|3027858.6860351562 +17796|-|3026838.8897094727 +15384|-|3023792.578125 +16336|-|3010813.478668213 +679|-|3010713.344604492 +7554|-|3010667.6899414062 +14553|-|3009756.8935546875 +8379|-|3009745.2450256348 +15436|-|3007499.8440856934 +12471|-|3003991.863304138 +18059|-|3003037.4895629883 +8536|-|3000745.9350585938 +19033|-|2999373.2282714844 +18179|-|2996151.2915039062 +10711|-|2996143.1240844727 +17271|-|2994264.7814331055 +13932|-|2989023.5170898438 +3101|-|2987788.302307129 +14550|-|2977853.6235046387 +3080|-|2977232.5885620117 +14533|-|2976490.5729370117 +14439|-|2975313.2834472656 +9237|-|2973124.6833496094 +1205|-|2971470.2993164062 +12361|-|2963419.493774414 +429|-|2962631.9146728516 +3970|-|2960418.432006836 +8403|-|2957698.399383545 +1098|-|2957514.0899658203 +7932|-|2955046.082672119 +16266|-|2952298.344482422 +19386|-|2948854.533721924 +13147|-|2947037.881072998 +2720|-|2947011.0479125977 +3840|-|2944219.253540039 +13482|-|2942474.947265625 +9436|-|2940396.258544922 +19779|-|2937105.8419189453 +18032|-|2933224.391052246 +7743|-|2932733.780944824 +14620|-|2930766.9780273438 +4606|-|2927832.6963806152 +18076|-|2924134.807067871 +19276|-|2918176.1877441406 +7483|-|2915919.015197754 +8575|-|2915132.6059570312 +11052|-|2913140.9509277344 +17251|-|2908345.850830078 +8788|-|2907935.816467285 +10960|-|2906511.1927490234 +18782|-|2903643.813232422 +19988|-|2897461.6274414062 +726|-|2896009.2184143066 +19942|-|2894251.367553711 +10864|-|2892252.5493164062 +17840|-|2891563.2795410156 +18717|-|2888939.8825683594 +12391|-|2886051.3537597656 +18219|-|2885920.995788574 +15100|-|2883342.3768310547 +2491|-|2880385.682739258 +12389|-|2879696.8638305664 +3880|-|2877770.2478637695 +18579|-|2874542.435546875 +13647|-|2873838.3503112793 +15758|-|2873804.818359375 +12917|-|2873659.5251464844 +18866|-|2873616.267883301 +13894|-|2872986.000732422 +15200|-|2872572.021789551 +9628|-|2872404.590698242 +8568|-|2871598.1217041016 +8389|-|2870238.0225219727 +5788|-|2867210.204788208 +19450|-|2863310.568725586 +9440|-|2863162.8435058594 +16795|-|2860135.2837524414 +19643|-|2858987.7473449707 +1974|-|2856825.715484619 +14622|-|2852089.0698242188 +6885|-|2851437.7064208984 +12532|-|2848992.5150146484 +1087|-|2847858.878173828 +5777|-|2846407.4865112305 +5629|-|2846076.0428466797 +6316|-|2840544.710998535 +12767|-|2840514.233642578 +12134|-|2840036.862548828 +14476|-|2839852.970489502 +803|-|2838388.204772949 +18964|-|2836942.512084961 +6020|-|2833459.096069336 +10401|-|2832688.8435058594 +1323|-|2829964.402770996 +1151|-|2829662.5451660156 +1458|-|2824034.4858398438 +2271|-|2820756.481201172 +18740|-|2814140.8579101562 +7348|-|2811731.043823242 +4281|-|2807190.532775879 +8043|-|2804706.1772460938 +3843|-|2804217.932373047 +7813|-|2802350.8267822266 +347|-|2802245.3862304688 +745|-|2801725.149230957 +10388|-|2799170.4808044434 +18100|-|2793358.5 +19043|-|2789013.861694336 +10644|-|2787796.8673706055 +16170|-|2787402.87109375 +398|-|2782729.127807617 +9370|-|2780078.1205444336 +14504|-|2780035.946166992 +1210|-|2778485.6623535156 +13385|-|2777445.5392456055 +3799|-|2775223.6352539062 +11325|-|2769765.9909057617 +3489|-|2769554.4653320312 +17181|-|2769028.3795166016 +6964|-|2766653.6810913086 +7381|-|2764898.7114257812 +6253|-|2764394.5572509766 +5975|-|2760819.7994384766 +11996|-|2760687.8385620117 +7570|-|2758977.1689453125 +4387|-|2757671.8505859375 +9014|-|2755367.42779541 +9403|-|2748021.6427001953 +11653|-|2739730.983444214 +17697|-|2739312.204559326 +958|-|2738032.0 +18816|-|2737139.9536132812 +14104|-|2735008.6606445312 +15966|-|2732250.2307128906 +17912|-|2724160.959777832 +7089|-|2720170.031982422 +16032|-|2718976.0388183594 +16891|-|2717293.2658081055 +19579|-|2716909.836791992 +17470|-|2715048.8865356445 +12408|-|2712556.5137023926 +4763|-|2711800.934448242 +1138|-|2709709.7604370117 +7363|-|2708414.33203125 +7877|-|2705439.5303649902 +17532|-|2703698.752319336 +10512|-|2701235.9489746094 +11957|-|2700133.234741211 +2455|-|2699593.858276367 +15119|-|2696860.833984375 +9868|-|2696801.4624023438 +14172|-|2695307.4061584473 +16120|-|2689337.8975524902 +13958|-|2679025.2431640625 +15169|-|2676686.086669922 +2648|-|2672232.0336914062 +6164|-|2671317.244873047 +12701|-|2669216.4587402344 +16382|-|2669034.5114746094 +15588|-|2667212.0443725586 +14830|-|2666758.130645752 +9119|-|2665812.2794189453 +1622|-|2665206.492614746 +878|-|2664045.7276306152 +13269|-|2662784.1020507812 +619|-|2655417.6220092773 +18386|-|2653795.01361084 +2501|-|2652260.4766845703 +2310|-|2651631.1859664917 +19420|-|2649395.637451172 +4895|-|2645152.2278060913 +7553|-|2643681.9628601074 +17814|-|2642781.32421875 +16097|-|2642500.0 +10995|-|2640811.183898926 +14895|-|2637733.7329101562 +18546|-|2637026.842102051 +9875|-|2631358.7497558594 +9591|-|2626899.469116211 +6001|-|2625893.671875 +7739|-|2624573.204223633 +10431|-|2624379.52935791 +4544|-|2615313.6489868164 +16387|-|2603195.666015625 +18375|-|2601407.794555664 +8395|-|2598728.4901123047 +18853|-|2593356.426513672 +4900|-|2592813.1634521484 +1302|-|2592197.8223876953 +17032|-|2589806.318359375 +14292|-|2589749.644165039 +43|-|2587359.6740722656 +5221|-|2587024.048187256 +397|-|2579751.5158691406 +17890|-|2579674.2705078125 +12157|-|2575510.537689209 +7340|-|2574645.822418213 +19368|-|2572619.0034484863 +8848|-|2570819.2907714844 +13789|-|2570243.276916504 +14596|-|2568234.1953125 +8408|-|2567434.4887390137 +19726|-|2565750.3982543945 +13964|-|2565579.001647949 +7740|-|2563027.5268554688 +14768|-|2560392.5514526367 +11734|-|2559062.138305664 +10294|-|2558258.104370117 +15032|-|2557926.2736968994 +9127|-|2556379.7252197266 +2181|-|2553175.0 +16653|-|2552229.62109375 +3866|-|2549994.8923339844 +16814|-|2548710.6931152344 +1866|-|2545838.3203125 +3512|-|2532626.9079589844 +4145|-|2529786.1860046387 +12120|-|2528298.5899658203 +644|-|2528123.0612182617 +15379|-|2525181.0533447266 +6392|-|2524063.2072143555 +2652|-|2521456.7431640625 +3363|-|2519202.307800293 +19167|-|2517993.239501953 +16042|-|2516600.0161743164 +2892|-|2511854.4744873047 +5711|-|2509401.8071289062 +14591|-|2506344.669647217 +6564|-|2506277.262084961 +1231|-|2505421.140930176 +5049|-|2502603.0395507812 +14576|-|2501606.7992248535 +10211|-|2500852.2442626953 +293|-|2493168.4423828125 +7371|-|2491134.5950317383 +18154|-|2491047.1584472656 +9494|-|2489825.4729003906 +14836|-|2480432.452850342 +19471|-|2480403.7979125977 +802|-|2478998.2611083984 +12541|-|2477242.539916992 +15065|-|2473563.866333008 +15995|-|2472803.2690429688 +9408|-|2471953.5029296875 +9776|-|2470448.005554199 +17325|-|2468989.0130615234 +3391|-|2468317.634460449 +16123|-|2467022.3071289062 +18758|-|2463797.954498291 +407|-|2460304.3577575684 +6840|-|2456170.6811523438 +9995|-|2455155.4196777344 +3877|-|2453696.583251953 +5817|-|2452493.1763916016 +14122|-|2452226.2705078125 +16699|-|2450274.0151367188 +8921|-|2450116.490966797 +15103|-|2449861.266723633 +7637|-|2449628.7778320312 +3076|-|2443927.3540649414 +6648|-|2443248.844116211 +17116|-|2442263.6349487305 +1645|-|2440838.4868164062 +3181|-|2440017.5247192383 +5966|-|2431558.1162109375 +15882|-|2428947.3501586914 +7529|-|2428381.2517089844 +12836|-|2427897.337097168 +18052|-|2427637.70703125 +13616|-|2426638.5749816895 +16615|-|2424775.1391601562 +18147|-|2424412.7150878906 +4586|-|2424123.8583374023 +14403|-|2423141.969116211 +11606|-|2422794.3608398438 +13526|-|2422212.7685546875 +3677|-|2421404.386291504 +5553|-|2418506.1848449707 +12109|-|2416514.1503601074 +13118|-|2415931.7456054688 +1563|-|2408855.393676758 +16591|-|2408045.473510742 +6411|-|2404918.6111450195 +10272|-|2402834.504119873 +10597|-|2400247.6235961914 +13700|-|2398035.8814697266 +9548|-|2397147.929626465 +14963|-|2395781.1557617188 +13325|-|2390637.514770508 +13864|-|2388067.935913086 +7450|-|2383447.680786133 +9275|-|2382868.42376709 +5829|-|2378038.0029296875 +13437|-|2377806.434326172 +13594|-|2375046.375732422 +11442|-|2374591.08782959 +15619|-|2374052.2861938477 +9063|-|2374035.85546875 +5990|-|2368686.5 +7811|-|2363829.23248291 +9525|-|2362974.501647949 +5597|-|2361031.846282959 +8963|-|2360774.0 +1709|-|2359839.3669433594 +15814|-|2358656.58984375 +17613|-|2357519.0229492188 +5022|-|2354550.4083251953 +17740|-|2354242.7854003906 +3388|-|2351042.2163085938 +13773|-|2348739.107788086 +14467|-|2348665.0895996094 +11544|-|2345324.373779297 +349|-|2344664.1772460938 +10356|-|2340862.6950683594 +18272|-|2338754.6755371094 +4627|-|2337430.850128174 +327|-|2335298.536376953 +19846|-|2332224.6345825195 +10814|-|2330319.610595703 +13102|-|2326122.75 +18867|-|2323972.035675049 +2824|-|2323315.0638427734 +19117|-|2319911.143798828 +1906|-|2319757.5622558594 +245|-|2319450.934753418 +17318|-|2317860.3787231445 +3862|-|2316453.7719726562 +8100|-|2313874.0541992188 +2958|-|2312239.4603881836 +10263|-|2308514.1148986816 +13814|-|2304940.337585449 +9394|-|2303161.745941162 +18080|-|2299416.790157318 +1271|-|2289526.907775879 +3327|-|2278474.467529297 +8740|-|2278405.9985961914 +8119|-|2276428.2103271484 +3368|-|2274373.545776367 +7963|-|2272300.7830810547 +2151|-|2270932.7026367188 +16995|-|2270264.6533203125 +9918|-|2269733.0579223633 +503|-|2268535.1638793945 +16692|-|2256484.5108032227 +793|-|2254198.8298339844 +16455|-|2252361.8503112793 +6644|-|2249521.8873901367 +17280|-|2249437.5 +6813|-|2248982.0513916016 +4674|-|2246915.378051758 +16325|-|2244369.7688598633 +182|-|2243290.0 +4626|-|2242474.3936157227 +10860|-|2241291.5270996094 +14034|-|2241220.8599853516 +2476|-|2240855.1263427734 +4253|-|2239985.7114257812 +3211|-|2239871.0924072266 +1290|-|2233312.97454834 +8479|-|2232189.021713257 +11895|-|2231607.0 +3487|-|2230171.5392456055 +14870|-|2229915.3003845215 +16328|-|2229484.001220703 +18585|-|2228215.4418945312 +7638|-|2228207.9917907715 +5436|-|2225672.3259887695 +14594|-|2223005.0853881836 +4532|-|2215710.971923828 +7586|-|2210562.5312805176 +11870|-|2205182.7026367188 +18487|-|2203653.653564453 +9179|-|2202720.5006713867 +16500|-|2201185.3262939453 +3679|-|2200592.5982666016 +12803|-|2198295.0134277344 +18056|-|2196741.8298339844 +11396|-|2195645.5908203125 +5087|-|2194120.762817383 +8067|-|2192048.619140625 +15357|-|2191646.555786133 +4491|-|2189713.4143066406 +208|-|2189046.713745117 +10958|-|2188766.8798675537 +9126|-|2188410.5 +15084|-|2184327.005859375 +18850|-|2183309.5993652344 +3398|-|2180250.0 +16137|-|2177318.7674560547 +211|-|2174808.926513672 +18422|-|2174381.057739258 +15840|-|2173510.3826904297 +19553|-|2173079.730697632 +8221|-|2169992.1791992188 +17000|-|2169611.2247314453 +6755|-|2168505.068206787 +10817|-|2167710.663925171 +8327|-|2167650.643310547 +543|-|2167367.964477539 +4553|-|2163371.47265625 +15019|-|2162287.9272460938 +334|-|2162178.43460083 +8516|-|2161478.9765625 +11349|-|2158941.968811035 +3902|-|2157027.8850097656 +14731|-|2155302.2080078125 +326|-|2153380.1381835938 +11403|-|2151242.3597717285 +11657|-|2150446.0576171875 +9496|-|2149218.9655151367 +8110|-|2149120.1103515625 +5153|-|2148527.3208618164 +884|-|2148324.8961868286 +8637|-|2146185.1928710938 +2364|-|2145790.7607421875 +12386|-|2145001.3764572144 +10133|-|2144903.972137451 +9895|-|2143324.86328125 +13755|-|2142539.4653320312 +4327|-|2138501.4697265625 +3369|-|2137408.7163085938 +5815|-|2136985.0 +19357|-|2132657.337890625 +2675|-|2124158.6796875 +17869|-|2123991.639678955 +11702|-|2122132.93850708 +17257|-|2117850.6005859375 +9952|-|2116686.3525390625 +3881|-|2111457.2221069336 +10951|-|2111185.603942871 +2128|-|2109702.3321533203 +6699|-|2106578.418273926 +3155|-|2103636.646118164 +16649|-|2101956.143798828 +15257|-|2100297.686126709 +9978|-|2099566.5798339844 +16810|-|2098301.3759765625 +10653|-|2093388.7595214844 +10476|-|2092766.402709961 +10883|-|2087495.2159423828 +9704|-|2086967.6427612305 +1119|-|2085182.7796020508 +19139|-|2079788.3895263672 +2144|-|2078391.1307373047 +9135|-|2076377.7429199219 +18548|-|2075584.3802490234 +10545|-|2075230.320892334 +6220|-|2074341.7611694336 +8616|-|2072887.6767425537 +5230|-|2072161.7690200806 +13916|-|2070504.7272949219 +4299|-|2069922.978149414 +894|-|2069688.126586914 +17847|-|2063367.09375 +18879|-|2061902.25 +13036|-|2061600.186340332 +10606|-|2060492.3840332031 +9454|-|2060016.404296875 +118|-|2059808.7673339844 +9601|-|2059715.8142089844 +13769|-|2057667.9653320312 +1987|-|2057289.2335205078 +13863|-|2055367.972869873 +13562|-|2054754.173828125 +1840|-|2054183.9311523438 +17995|-|2053221.9276733398 +17389|-|2051128.1726074219 +15168|-|2045987.4976501465 +2139|-|2045365.2990722656 +4024|-|2044243.0604248047 +8964|-|2041648.8778686523 +181|-|2040167.0458984375 +7628|-|2039548.974243164 +3|-|2038846.1099853516 +15553|-|2036959.0168457031 +11355|-|2035405.6726074219 +13006|-|2034991.224761963 +3091|-|2031393.5194702148 +1281|-|2030628.4670410156 +1408|-|2028621.6350097656 +18211|-|2024538.7075805664 +2287|-|2020754.2392578125 +6228|-|2019198.8252868652 +4362|-|2018495.297241211 +10873|-|2013280.2802734375 +7383|-|2009581.8969726562 +1386|-|2006544.2039794922 +9820|-|2005815.8198242188 +18134|-|2003409.7178649902 +15727|-|2000654.5 +157|-|2000148.1732177734 +19571|-|1999891.1595153809 +17728|-|1997944.4525146484 +5278|-|1996644.1983642578 +17737|-|1994653.74609375 +10220|-|1989890.9174804688 +1397|-|1984509.1595458984 +6195|-|1983928.1798095703 +4270|-|1983726.9793701172 +16965|-|1983286.1907958984 +1683|-|1980638.5913085938 +13086|-|1978609.3798828125 +7124|-|1974039.398071289 +5211|-|1973843.7998046875 +6794|-|1973149.444885254 +257|-|1973035.476928711 +6995|-|1968281.5246582031 +8447|-|1967292.732269287 +15873|-|1967257.8836975098 +12862|-|1964014.1873779297 +8295|-|1961467.1154785156 +931|-|1958825.215057373 +6876|-|1957359.5209960938 +1932|-|1954592.3408203125 +1061|-|1952688.0393676758 +18108|-|1951143.7312927246 +5138|-|1950861.0397338867 +12598|-|1950211.6165161133 +10829|-|1943924.6043701172 +11950|-|1941210.9121704102 +12076|-|1939324.0358276367 +2176|-|1938691.3228149414 +6616|-|1937401.9128417969 +5893|-|1934358.6018066406 +976|-|1933066.8347167969 +13173|-|1932557.5862121582 +14947|-|1929229.9713745117 +16857|-|1928814.7243652344 +13403|-|1928702.8103027344 +4819|-|1926969.7286376953 +13127|-|1926929.848815918 +6871|-|1926787.6062011719 +15465|-|1925145.1381530762 +1131|-|1920005.5 +11845|-|1913576.3525390625 +8364|-|1909122.1602172852 +16588|-|1904272.3926391602 +6759|-|1903906.2498779297 +11586|-|1901895.5828857422 +8145|-|1901787.6020507812 +17333|-|1897297.1284179688 +13290|-|1890633.75 +6499|-|1887620.9167480469 +4881|-|1887535.8764038086 +7147|-|1886710.140991211 +3883|-|1886567.746307373 +18911|-|1885597.1535644531 +11336|-|1883573.6238708496 +8653|-|1883275.7177581787 +19476|-|1881492.5407104492 +14799|-|1880543.4484863281 +14491|-|1879219.9541625977 +11815|-|1877434.3993530273 +3173|-|1874302.1049499512 +7161|-|1873023.4817504883 +14631|-|1873015.319519043 +4247|-|1869912.9061279297 +3568|-|1865824.3737792969 +1500|-|1865450.9802246094 +11833|-|1863665.2496948242 +495|-|1860771.3789367676 +6776|-|1855589.0936889648 +11374|-|1855221.1085205078 +5637|-|1853782.114654541 +3597|-|1852826.880432129 +981|-|1852083.6515808105 +16076|-|1850349.7155761719 +17597|-|1845421.0075378418 +19609|-|1843185.4454956055 +10997|-|1843072.0914916992 +3403|-|1842975.0 +897|-|1842845.052520752 +16697|-|1840630.7087402344 +17644|-|1840597.7307128906 +6485|-|1838812.0330810547 +5492|-|1836202.818359375 +12038|-|1835075.0694580078 +9325|-|1832634.8212890625 +10637|-|1832347.4095458984 +11318|-|1830158.4047088623 +4357|-|1828729.9438476562 +18553|-|1826335.2055358887 +12623|-|1825950.817199707 +961|-|1825869.6166992188 +1677|-|1821816.9250488281 +8211|-|1820432.5127563477 +19719|-|1819333.578491211 +19663|-|1819074.4201660156 +16296|-|1818353.7156066895 +16527|-|1817834.3540039062 +4964|-|1815400.0594177246 +1769|-|1812929.1693115234 +13126|-|1808799.9389648438 +7854|-|1807608.1387939453 +18380|-|1803641.2076416016 +6584|-|1802346.9595947266 +7665|-|1801765.3981933594 +16553|-|1796146.7329101562 +17761|-|1795095.779296875 +11179|-|1794890.2955627441 +15171|-|1794148.6752929688 +3018|-|1793183.8253173828 +15741|-|1788611.9757080078 +5331|-|1783901.2844848633 +9860|-|1775071.2819213867 +7984|-|1774302.801361084 +15354|-|1774270.7454833984 +17884|-|1774212.3813476562 +16257|-|1771869.7191467285 +10696|-|1768645.2502441406 +2104|-|1767902.6162109375 +14465|-|1764946.429901123 +10089|-|1764692.337890625 +6719|-|1762699.5961914062 +3648|-|1760594.3675842285 +7241|-|1759913.6359558105 +11122|-|1757429.9960289001 +17019|-|1752560.6893920898 +13877|-|1744271.0842895508 +15325|-|1743826.3114013672 +17860|-|1739870.3830566406 +2236|-|1739795.7482910156 +4436|-|1738760.310546875 +7701|-|1738670.4733886719 +8147|-|1736855.1896209717 +6676|-|1736341.4271240234 +19505|-|1735413.412902832 +9885|-|1731366.2120361328 +2112|-|1725934.1193237305 +5330|-|1722196.964630127 +3561|-|1720377.9982910156 +10104|-|1714419.1653289795 +16362|-|1712457.3900756836 +15573|-|1712365.4072265625 +15006|-|1711381.300994873 +14629|-|1709942.0542907715 +9612|-|1709528.3438720703 +19910|-|1709211.0869750977 +13145|-|1708907.3772583008 +11494|-|1707973.7124023438 +15895|-|1706999.4177246094 +8239|-|1705479.1485595703 +2403|-|1705331.1588134766 +19436|-|1702706.0302734375 +3476|-|1702335.8241577148 +6828|-|1702292.1298828125 +771|-|1701589.5 +8448|-|1700312.4758300781 +3755|-|1699047.086517334 +13895|-|1698679.0136871338 +9785|-|1698056.3375549316 +6180|-|1695571.4729003906 +532|-|1694356.161058545 +6741|-|1692552.4584350586 +19964|-|1692367.5817871094 +3747|-|1691244.5666503906 +3253|-|1690719.3782958984 +16119|-|1688339.222946167 +7113|-|1681910.9939575195 +12368|-|1681219.8216552734 +16378|-|1679705.6228027344 +1393|-|1675545.301147461 +11119|-|1675453.435546875 +4469|-|1674023.5486450195 +6955|-|1672618.8428955078 +11579|-|1672345.3000488281 +19898|-|1671781.7362670898 +15351|-|1659204.3444824219 +6133|-|1658215.4739990234 +9110|-|1658054.6356811523 +2979|-|1656016.7265930176 +18764|-|1653708.4138183594 +8995|-|1653627.604309082 +13096|-|1651408.7465515137 +15062|-|1650548.0698242188 +7924|-|1650202.4209136963 +10076|-|1647970.1915283203 +15859|-|1646036.3041992188 +17932|-|1642640.6559143066 +19694|-|1642089.563293457 +13827|-|1642001.3620605469 +17963|-|1639689.0 +10698|-|1635848.2767028809 +18003|-|1633530.8166503906 +8416|-|1633366.7739562988 +476|-|1631154.0655975342 +2806|-|1630782.7474975586 +12129|-|1628615.5079345703 +11215|-|1626624.6862792969 +14061|-|1624933.5075683594 +5956|-|1623586.1042785645 +9043|-|1622670.3633728027 +13287|-|1621980.3876342773 +11410|-|1621420.8666992188 +13990|-|1621268.2245483398 +12952|-|1619215.2041625977 +15181|-|1619088.6878051758 +9784|-|1618120.4653015137 +10733|-|1616168.8916931152 +16054|-|1614531.2112426758 +5864|-|1614397.8735351562 +1875|-|1611927.0240783691 +17381|-|1611664.8427734375 +14562|-|1607467.8866577148 +575|-|1605941.6989746094 +2005|-|1605591.665222168 +4332|-|1605448.8093566895 +4653|-|1602596.349975586 +15403|-|1601830.3666992188 +17430|-|1599681.3586425781 +4798|-|1593630.5130004883 +12991|-|1593321.5407714844 +15653|-|1593138.7158660889 +10066|-|1593049.0897216797 +8892|-|1592100.8681030273 +6708|-|1590159.0702438354 +9825|-|1589403.9376831055 +8271|-|1588475.4641113281 +17084|-|1584280.8524169922 +4003|-|1583631.0 +869|-|1582643.133178711 +16400|-|1582313.2043457031 +19088|-|1581708.5712890625 +6581|-|1581346.8233642578 +9481|-|1581048.6053466797 +6092|-|1580846.4543457031 +3624|-|1578777.2731323242 +6503|-|1578507.8547363281 +14557|-|1578281.0028686523 +2428|-|1577543.8458251953 +15513|-|1573560.174041748 +4641|-|1573363.567993164 +10152|-|1570213.544921875 +5932|-|1566902.4616699219 +7482|-|1561323.4680175781 +13745|-|1558358.3333435059 +2251|-|1558274.6385498047 +9845|-|1558068.0759277344 +7603|-|1557388.1533813477 +1809|-|1553837.161743164 +18128|-|1547643.3653259277 +8086|-|1543199.0756835938 +14948|-|1541721.517715454 +16725|-|1540948.5 +2999|-|1540317.6873779297 +8861|-|1540008.5061340332 +1964|-|1538815.2198791504 +19374|-|1537884.8074035645 +15428|-|1535994.3779907227 +7449|-|1534782.4838256836 +16884|-|1534509.1881103516 +10271|-|1534397.2959594727 +11782|-|1529963.159790039 +8184|-|1529750.6717834473 +4560|-|1527433.2497253418 +4616|-|1525374.4865722656 +3814|-|1524076.9855957031 +17265|-|1523932.0370178223 +16520|-|1522906.3503417969 +10475|-|1518705.0485229492 +5094|-|1517317.8380126953 +8626|-|1515142.0741577148 +19895|-|1512286.6967773438 +19933|-|1506235.3649597168 +6854|-|1505626.0 +13995|-|1505562.1680908203 +7102|-|1504945.6089172363 +9079|-|1501237.1293945312 +18329|-|1500146.969116211 +3742|-|1496990.726196289 +12395|-|1496904.4160766602 +12214|-|1496489.390258789 +12298|-|1495554.3113708496 +4978|-|1495389.471130371 +2927|-|1494280.0918579102 +2119|-|1494151.127380371 +15143|-|1492039.783859253 +14548|-|1487406.5622558594 +840|-|1486128.9608764648 +5902|-|1486097.3247070312 +10614|-|1482144.7607421875 +5895|-|1481356.763305664 +15958|-|1480951.6381835938 +11408|-|1479948.9807128906 +8407|-|1474236.0 +6243|-|1471007.8878173828 +10389|-|1469004.5239562988 +13871|-|1468938.6767578125 +19811|-|1464597.0150146484 +10495|-|1464290.519104004 +4389|-|1463010.7704467773 +1311|-|1461703.4008789062 +17874|-|1459408.8374023438 +6597|-|1458761.8845214844 +19211|-|1456741.605255127 +12879|-|1456178.2114257812 +8840|-|1455731.4017944336 +14755|-|1454890.5603027344 +16957|-|1454465.9743652344 +9257|-|1454388.7916259766 +5193|-|1454011.3244018555 +6884|-|1452474.5421142578 +19948|-|1452024.0 +15076|-|1448395.0119018555 +16016|-|1447557.4293823242 +11693|-|1445839.7114868164 +6975|-|1440516.9924316406 +4290|-|1439768.5415039062 +18900|-|1438722.1258544922 +14383|-|1438477.8793945312 +15098|-|1435941.8385314941 +9322|-|1435282.8387451172 +458|-|1433040.412902832 +10042|-|1432906.392364502 +5052|-|1431900.9575500488 +6600|-|1431116.5880126953 +3630|-|1428665.0668945312 +9636|-|1428193.8202514648 +16511|-|1427308.7228393555 +4045|-|1427248.2903442383 +19562|-|1426348.7884521484 +8814|-|1425690.0772705078 +2616|-|1425178.0203857422 +4587|-|1425109.3546142578 +148|-|1424237.3502502441 +2712|-|1423780.2355957031 +10863|-|1423386.1506347656 +16096|-|1421942.0442504883 +18936|-|1421938.6294555664 +18327|-|1419872.9162597656 +11620|-|1419050.1036071777 +3740|-|1418609.87789917 +3457|-|1418603.5 +1185|-|1417637.4230957031 +8178|-|1417357.3143310547 +17791|-|1413293.0977172852 +13608|-|1411323.150390625 +17849|-|1409613.478088379 +6814|-|1406228.4159851074 +14022|-|1406138.0842285156 +14231|-|1403771.5576171875 +19546|-|1402854.5819702148 +19619|-|1402389.2103271484 +5609|-|1402302.5663909912 +5342|-|1401567.6424560547 +3084|-|1401096.0903930664 +5708|-|1400334.841003418 +17998|-|1399862.4307250977 +19850|-|1397630.3699951172 +14004|-|1395443.0749053955 +13071|-|1394653.2860870361 +2797|-|1393747.5387573242 +2866|-|1392947.2953796387 +19809|-|1389067.7252197266 +13600|-|1380865.775756836 +13614|-|1380654.2938079834 +5884|-|1380319.7587280273 +9404|-|1378623.6877441406 +10656|-|1376954.3553466797 +12324|-|1376502.3815917969 +7325|-|1375030.3835601807 +13295|-|1373987.3495178223 +11864|-|1373555.7197265625 +6987|-|1373481.4578399658 +8386|-|1371854.3801879883 +10916|-|1370374.3459472656 +12867|-|1369058.1170043945 +14668|-|1369040.3050231934 +13383|-|1367342.254119873 +18572|-|1366954.013305664 +1152|-|1366861.3978271484 +6015|-|1366452.2076416016 +3344|-|1366185.1764526367 +7889|-|1365521.9282226562 +13345|-|1364088.9291992188 +6276|-|1363421.6049804688 +8069|-|1361824.2585449219 +17509|-|1360892.4440917969 +15137|-|1358678.0104980469 +17163|-|1357391.5466308594 +4704|-|1356692.3569335938 +8609|-|1356578.2471618652 +12644|-|1356088.1536560059 +17141|-|1356022.3646240234 +11805|-|1354826.7747039795 +6386|-|1354187.1807861328 +3004|-|1352173.4365234375 +8634|-|1350211.761291504 +4399|-|1349881.159790039 +10362|-|1349411.3036499023 +1572|-|1348835.2299804688 +7359|-|1348224.1096496582 +11884|-|1346696.8058624268 +11671|-|1346424.1040039062 +5350|-|1346359.3221435547 +3119|-|1345996.4392089844 +5307|-|1345355.947265625 +16117|-|1345045.181640625 +8715|-|1342665.7254638672 +5398|-|1341179.2985229492 +7627|-|1338820.5560455322 +8457|-|1337714.7106933594 +4958|-|1334732.7673339844 +84|-|1334146.7134857178 +6932|-|1333235.3314208984 +757|-|1332921.0741577148 +4076|-|1332441.0382080078 +1751|-|1329112.306640625 +15701|-|1327052.1989135742 +4119|-|1326549.874847412 +1562|-|1325604.3093261719 +8741|-|1325517.5654296875 +1135|-|1325422.705871582 +1002|-|1323418.641418457 +5832|-|1323085.6803588867 +5368|-|1322793.980255127 +5382|-|1322628.8184814453 +5616|-|1319082.2923583984 +2832|-|1318691.9218444824 +3895|-|1317858.4164886475 +8629|-|1317756.5368652344 +5709|-|1317058.6618652344 +18383|-|1316451.0198669434 +15797|-|1314806.6162109375 +1900|-|1313660.433959961 +13882|-|1310455.8077392578 +6785|-|1309877.7795410156 +14855|-|1309280.8302001953 +7761|-|1308602.2880859375 +14268|-|1306810.3637695312 +6257|-|1306056.9331054688 +19002|-|1305509.5361328125 +5095|-|1303729.0596923828 +10320|-|1301657.6178588867 +7826|-|1299561.6943359375 +13359|-|1298717.173675537 +7436|-|1298127.3671875 +5644|-|1295055.8004760742 +11327|-|1290526.4305114746 +5277|-|1289329.6742248535 +15932|-|1286235.8208007812 +14322|-|1284809.348083496 +144|-|1284270.1287841797 +3043|-|1281162.7379608154 +16788|-|1280955.3276824951 +17136|-|1280443.0877685547 +12560|-|1279117.9238586426 +13833|-|1278834.75 +5414|-|1277893.2998199463 +12582|-|1277592.3092651367 +4644|-|1277535.0036621094 +14032|-|1277077.8298950195 +18325|-|1271719.6611328125 +7072|-|1271228.4482421875 +16868|-|1267469.430847168 +8137|-|1267425.8177490234 +5976|-|1266206.9095458984 +14125|-|1265569.0479125977 +13299|-|1265287.6037597656 +18376|-|1264249.3371582031 +6157|-|1261759.8669433594 +5002|-|1261669.6770019531 +13368|-|1260918.5923461914 +15589|-|1260059.7495117188 +2149|-|1258981.3828125 +9639|-|1256283.3509521484 +11689|-|1256027.9272460938 +9083|-|1245924.1743164062 +16231|-|1242625.6755065918 +5084|-|1242385.3291015625 +11634|-|1240760.2196655273 +15617|-|1239731.25 +9865|-|1237181.5987701416 +14212|-|1236365.53515625 +10325|-|1235223.3427734375 +19582|-|1235105.7646484375 +740|-|1234746.8372955322 +19231|-|1233623.062133789 +16840|-|1233063.890197754 +5703|-|1231744.3662261963 +5761|-|1229435.17578125 +15630|-|1226611.6397094727 +10408|-|1224698.3697509766 +9177|-|1221942.5216674805 +13389|-|1221666.7762756348 +6104|-|1221577.9013671875 +9673|-|1218826.6256866455 +2707|-|1217124.4658203125 +18672|-|1214208.7954711914 +5112|-|1209590.2445220947 +6264|-|1208318.5409545898 +18496|-|1207881.75 +10971|-|1207183.5093688965 +19059|-|1206729.8780822754 +431|-|1205938.4302368164 +3821|-|1201192.758178711 +826|-|1200454.5931091309 +3317|-|1200440.8734436035 +19689|-|1198899.5228881836 +19641|-|1198797.9747314453 +6379|-|1197195.5 +814|-|1194417.4224243164 +18643|-|1194000.783569336 +11865|-|1193965.7729492188 +12393|-|1193896.8090820312 +9218|-|1193660.5513916016 +8674|-|1191881.3533477783 +8582|-|1191803.997253418 +13084|-|1191508.0 +18844|-|1190239.9542236328 +16061|-|1189935.0 +6134|-|1185550.8386230469 +8628|-|1183245.5668945312 +8884|-|1181547.4489746094 +7697|-|1181032.5 +9044|-|1180922.6353149414 +13257|-|1180158.620300293 +8066|-|1178808.1107330322 +5876|-|1177376.77734375 +14694|-|1177059.351852417 +16062|-|1175391.0173034668 +9104|-|1175178.9472961426 +11600|-|1175091.096496582 +10337|-|1172684.9451904297 +19188|-|1172349.7580566406 +8833|-|1171372.904724121 +6895|-|1170602.089416504 +14100|-|1168878.41796875 +13538|-|1168554.3215332031 +3408|-|1166645.1654052734 +1860|-|1165673.7287902832 +13436|-|1164278.6790161133 +19325|-|1162733.705291748 +7403|-|1161982.029724121 +4882|-|1161404.8461914062 +13105|-|1161320.5768127441 +17880|-|1161256.0773925781 +19284|-|1160927.5817565918 +13476|-|1159035.1672973633 +18913|-|1158208.258392334 +18523|-|1158135.0402832031 +12508|-|1157538.5005187988 +9090|-|1156362.6936035156 +17653|-|1154338.0590820312 +3926|-|1152652.4869995117 +10183|-|1148324.5918579102 +7556|-|1146268.1630859375 +16436|-|1142656.5260009766 +4741|-|1141614.0197753906 +15651|-|1141497.958190918 +3183|-|1140081.3345947266 +9532|-|1139902.5 +16403|-|1139306.3049316406 +2368|-|1137421.1923828125 +3889|-|1136395.5505371094 +2885|-|1135838.1436157227 +7851|-|1135110.7958374023 +16234|-|1135017.2622680664 +12746|-|1134531.0544433594 +2647|-|1132941.1064453125 +5373|-|1132158.0131225586 +10340|-|1132004.2142028809 +8873|-|1131949.2911987305 +1132|-|1131338.8703613281 +15594|-|1131328.5715942383 +4376|-|1130282.1826171875 +240|-|1126682.5329589844 +2231|-|1124447.1466827393 +929|-|1121383.9423828125 +11599|-|1119307.2609481812 +3765|-|1119093.4799194336 +17635|-|1118420.1767578125 +7119|-|1118285.0696411133 +15121|-|1117715.336013794 +11858|-|1116963.5603027344 +16963|-|1116929.478881836 +16356|-|1113649.0235595703 +6924|-|1112198.37890625 +16223|-|1111257.0 +18091|-|1110043.0338134766 +12628|-|1108954.82321167 +16043|-|1108831.0647735596 +9402|-|1108290.4839019775 +708|-|1107084.0362548828 +4078|-|1105993.9275054932 +17593|-|1104713.4141845703 +12776|-|1104362.6319885254 +7583|-|1102813.4978027344 +14619|-|1102675.7803344727 +8842|-|1100110.2694702148 +4196|-|1099726.5893554688 +2019|-|1098178.6320495605 +6863|-|1097246.3224487305 +6489|-|1096503.0916137695 +2459|-|1094813.0198364258 +11964|-|1094484.9845581055 +3236|-|1093969.8435058594 +17647|-|1093809.1983642578 +17648|-|1093114.6135253906 +119|-|1092687.5137939453 +9626|-|1092079.9865722656 +9124|-|1091569.6478271484 +13175|-|1089851.7216796875 +2532|-|1088706.334411621 +16083|-|1088295.4176940918 +8874|-|1086011.3367004395 +12872|-|1082970.2655029297 +19821|-|1082520.8139038086 +4800|-|1080389.6745300293 +18696|-|1079685.401550293 +19545|-|1079184.3186950684 +13120|-|1077742.2612304688 +10588|-|1076203.8264770508 +17696|-|1075092.6732177734 +14651|-|1073222.2071533203 +903|-|1071146.7752304077 +5858|-|1070259.463256836 +8302|-|1069504.8403320312 +18728|-|1069225.5051574707 +18026|-|1068569.0145874023 +19383|-|1066907.5825958252 +18690|-|1065930.9407958984 +5924|-|1065143.144104004 +4880|-|1065011.75 +12439|-|1064381.166442871 +16529|-|1062371.728088379 +19653|-|1057683.5363769531 +3136|-|1056810.4391860962 +18932|-|1056193.6198425293 +2124|-|1054160.5241699219 +16851|-|1052646.814819336 +10123|-|1051623.9782714844 +5618|-|1048447.9822540283 +19851|-|1045187.8079223633 +16278|-|1044808.3540039062 +11479|-|1044276.2375640869 +13263|-|1042046.1585388184 +6041|-|1041123.3318481445 +7193|-|1040455.2977294922 +19408|-|1039430.0012512207 +11260|-|1036828.5228881836 +5179|-|1035633.4487304688 +1331|-|1034398.0 +7706|-|1034249.3519592285 +8436|-|1033549.3817138672 +1801|-|1031886.0131835938 +4170|-|1031642.9565429688 +11827|-|1031139.4237976074 +17114|-|1027985.8488464355 +18278|-|1026583.1158447266 +1995|-|1025165.6577148438 +7667|-|1022980.114654541 +6559|-|1021635.4828643799 +17488|-|1021612.1164245605 +16059|-|1019781.1790771484 +7633|-|1018782.5820922852 +10032|-|1016809.5 +2899|-|1016438.739440918 +14628|-|1016033.214302063 +10126|-|1015846.7950439453 +3884|-|1014413.5345458984 +16913|-|1013604.354309082 +18644|-|1010288.1134033203 +19870|-|1007919.3837890625 +18564|-|1007416.2255859375 +10179|-|1004920.0225830078 +883|-|1004650.6300048828 +3627|-|1004461.0607299805 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q12.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q12.out new file mode 100644 index 0000000000000..99541acd0f54f --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q12.out @@ -0,0 +1,3 @@ +2 +MAIL|-|647|-|945 +SHIP|-|620|-|943 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q13.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q13.out new file mode 100644 index 0000000000000..ae4362f61da31 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q13.out @@ -0,0 +1,38 @@ +37 +0|-|5000 +10|-|665 +9|-|657 +11|-|621 +12|-|567 +8|-|564 +13|-|492 +18|-|482 +7|-|480 +20|-|456 +14|-|456 +16|-|449 +19|-|447 +15|-|432 +17|-|423 +21|-|412 +22|-|371 +6|-|337 +23|-|323 +24|-|256 +25|-|204 +5|-|204 +26|-|155 +27|-|141 +28|-|97 +4|-|94 +29|-|64 +3|-|48 +30|-|27 +31|-|26 +32|-|14 +33|-|11 +2|-|11 +34|-|6 +35|-|5 +1|-|2 +36|-|1 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q14.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q14.out new file mode 100644 index 0000000000000..f8627a6705e28 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q14.out @@ -0,0 +1,2 @@ +1 +16.283855689005982 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q15.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q15.out new file mode 100644 index 0000000000000..9dac6c8998839 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q15.out @@ -0,0 +1,2 @@ +1 +677|-|Supplier#000000677|-|8mhrffG7D2WJBSQbOGstQ|-|23-290-639-3315|-|1614410.2928000002 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q16.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q16.out new file mode 100644 index 0000000000000..341afcb5718b0 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q16.out @@ -0,0 +1,2763 @@ +2762 +Brand#14|-|SMALL ANODIZED NICKEL|-|45|-|12 +Brand#22|-|SMALL BURNISHED BRASS|-|19|-|12 +Brand#25|-|PROMO POLISHED COPPER|-|14|-|12 +Brand#35|-|LARGE ANODIZED STEEL|-|45|-|12 +Brand#35|-|PROMO BRUSHED COPPER|-|9|-|12 +Brand#51|-|ECONOMY ANODIZED STEEL|-|9|-|12 +Brand#53|-|LARGE BRUSHED NICKEL|-|45|-|12 +Brand#11|-|ECONOMY POLISHED COPPER|-|14|-|8 +Brand#11|-|LARGE PLATED STEEL|-|23|-|8 +Brand#11|-|PROMO POLISHED STEEL|-|23|-|8 +Brand#11|-|STANDARD ANODIZED COPPER|-|9|-|8 +Brand#12|-|ECONOMY BURNISHED BRASS|-|9|-|8 +Brand#12|-|LARGE ANODIZED BRASS|-|14|-|8 +Brand#12|-|SMALL ANODIZED TIN|-|23|-|8 +Brand#12|-|SMALL BRUSHED NICKEL|-|23|-|8 +Brand#12|-|STANDARD ANODIZED BRASS|-|3|-|8 +Brand#12|-|STANDARD BURNISHED TIN|-|23|-|8 +Brand#13|-|ECONOMY POLISHED BRASS|-|9|-|8 +Brand#13|-|LARGE BURNISHED COPPER|-|45|-|8 +Brand#13|-|MEDIUM ANODIZED STEEL|-|23|-|8 +Brand#13|-|MEDIUM PLATED NICKEL|-|3|-|8 +Brand#13|-|PROMO BURNISHED BRASS|-|9|-|8 +Brand#13|-|PROMO POLISHED BRASS|-|3|-|8 +Brand#13|-|PROMO POLISHED TIN|-|36|-|8 +Brand#13|-|SMALL BURNISHED STEEL|-|23|-|8 +Brand#13|-|STANDARD BRUSHED STEEL|-|9|-|8 +Brand#14|-|ECONOMY BRUSHED TIN|-|3|-|8 +Brand#14|-|ECONOMY BURNISHED TIN|-|23|-|8 +Brand#14|-|PROMO BRUSHED STEEL|-|9|-|8 +Brand#14|-|PROMO PLATED TIN|-|45|-|8 +Brand#15|-|ECONOMY PLATED TIN|-|9|-|8 +Brand#15|-|STANDARD BRUSHED COPPER|-|14|-|8 +Brand#15|-|STANDARD PLATED TIN|-|3|-|8 +Brand#21|-|ECONOMY POLISHED TIN|-|3|-|8 +Brand#21|-|PROMO POLISHED COPPER|-|9|-|8 +Brand#21|-|PROMO POLISHED TIN|-|49|-|8 +Brand#21|-|SMALL POLISHED STEEL|-|3|-|8 +Brand#21|-|STANDARD PLATED BRASS|-|49|-|8 +Brand#21|-|STANDARD PLATED NICKEL|-|49|-|8 +Brand#22|-|ECONOMY ANODIZED TIN|-|49|-|8 +Brand#22|-|ECONOMY BRUSHED BRASS|-|14|-|8 +Brand#22|-|LARGE BURNISHED TIN|-|36|-|8 +Brand#22|-|MEDIUM ANODIZED STEEL|-|36|-|8 +Brand#22|-|MEDIUM PLATED STEEL|-|9|-|8 +Brand#22|-|PROMO POLISHED NICKEL|-|9|-|8 +Brand#22|-|SMALL ANODIZED STEEL|-|19|-|8 +Brand#22|-|STANDARD ANODIZED COPPER|-|23|-|8 +Brand#23|-|ECONOMY BRUSHED NICKEL|-|23|-|8 +Brand#23|-|LARGE ANODIZED BRASS|-|9|-|8 +Brand#23|-|LARGE ANODIZED STEEL|-|23|-|8 +Brand#23|-|SMALL BRUSHED COPPER|-|23|-|8 +Brand#23|-|STANDARD BRUSHED TIN|-|3|-|8 +Brand#23|-|STANDARD BURNISHED NICKEL|-|49|-|8 +Brand#23|-|STANDARD PLATED NICKEL|-|36|-|8 +Brand#24|-|ECONOMY ANODIZED BRASS|-|19|-|8 +Brand#24|-|ECONOMY POLISHED BRASS|-|36|-|8 +Brand#24|-|LARGE BURNISHED STEEL|-|14|-|8 +Brand#24|-|MEDIUM PLATED NICKEL|-|36|-|8 +Brand#25|-|ECONOMY BRUSHED STEEL|-|49|-|8 +Brand#25|-|MEDIUM BURNISHED TIN|-|3|-|8 +Brand#25|-|PROMO ANODIZED TIN|-|36|-|8 +Brand#25|-|PROMO PLATED NICKEL|-|3|-|8 +Brand#25|-|SMALL BURNISHED BRASS|-|3|-|8 +Brand#31|-|LARGE ANODIZED BRASS|-|3|-|8 +Brand#31|-|SMALL ANODIZED COPPER|-|3|-|8 +Brand#31|-|SMALL ANODIZED NICKEL|-|9|-|8 +Brand#31|-|SMALL ANODIZED STEEL|-|14|-|8 +Brand#32|-|MEDIUM ANODIZED STEEL|-|49|-|8 +Brand#32|-|MEDIUM BURNISHED COPPER|-|19|-|8 +Brand#32|-|SMALL BURNISHED STEEL|-|23|-|8 +Brand#32|-|STANDARD BURNISHED STEEL|-|45|-|8 +Brand#34|-|ECONOMY ANODIZED NICKEL|-|49|-|8 +Brand#34|-|LARGE BURNISHED TIN|-|49|-|8 +Brand#34|-|MEDIUM BURNISHED NICKEL|-|3|-|8 +Brand#34|-|PROMO ANODIZED TIN|-|3|-|8 +Brand#34|-|SMALL BRUSHED TIN|-|3|-|8 +Brand#34|-|STANDARD BURNISHED TIN|-|23|-|8 +Brand#35|-|MEDIUM BRUSHED STEEL|-|45|-|8 +Brand#35|-|PROMO BURNISHED STEEL|-|14|-|8 +Brand#35|-|SMALL BURNISHED STEEL|-|23|-|8 +Brand#35|-|SMALL POLISHED COPPER|-|14|-|8 +Brand#35|-|STANDARD PLATED COPPER|-|9|-|8 +Brand#41|-|ECONOMY BRUSHED BRASS|-|23|-|8 +Brand#41|-|LARGE BURNISHED STEEL|-|23|-|8 +Brand#41|-|PROMO BURNISHED TIN|-|14|-|8 +Brand#41|-|PROMO PLATED STEEL|-|36|-|8 +Brand#41|-|PROMO POLISHED TIN|-|19|-|8 +Brand#41|-|SMALL BURNISHED COPPER|-|23|-|8 +Brand#42|-|LARGE POLISHED TIN|-|14|-|8 +Brand#42|-|MEDIUM ANODIZED TIN|-|49|-|8 +Brand#42|-|MEDIUM BRUSHED TIN|-|14|-|8 +Brand#42|-|MEDIUM BURNISHED NICKEL|-|23|-|8 +Brand#42|-|MEDIUM PLATED COPPER|-|45|-|8 +Brand#42|-|MEDIUM PLATED TIN|-|45|-|8 +Brand#42|-|SMALL PLATED COPPER|-|36|-|8 +Brand#43|-|ECONOMY BRUSHED STEEL|-|45|-|8 +Brand#43|-|LARGE BRUSHED COPPER|-|19|-|8 +Brand#43|-|PROMO BRUSHED BRASS|-|36|-|8 +Brand#43|-|SMALL BURNISHED TIN|-|45|-|8 +Brand#43|-|SMALL PLATED COPPER|-|45|-|8 +Brand#44|-|PROMO POLISHED TIN|-|23|-|8 +Brand#44|-|SMALL POLISHED NICKEL|-|14|-|8 +Brand#44|-|SMALL POLISHED TIN|-|45|-|8 +Brand#44|-|STANDARD BURNISHED COPPER|-|3|-|8 +Brand#51|-|LARGE ANODIZED BRASS|-|19|-|8 +Brand#51|-|LARGE POLISHED COPPER|-|23|-|8 +Brand#51|-|MEDIUM ANODIZED TIN|-|9|-|8 +Brand#51|-|MEDIUM ANODIZED TIN|-|14|-|8 +Brand#51|-|MEDIUM BURNISHED NICKEL|-|23|-|8 +Brand#51|-|SMALL ANODIZED COPPER|-|45|-|8 +Brand#51|-|SMALL ANODIZED COPPER|-|49|-|8 +Brand#51|-|SMALL BRUSHED COPPER|-|45|-|8 +Brand#51|-|SMALL BRUSHED TIN|-|36|-|8 +Brand#51|-|STANDARD POLISHED TIN|-|3|-|8 +Brand#52|-|ECONOMY ANODIZED STEEL|-|3|-|8 +Brand#52|-|ECONOMY PLATED TIN|-|19|-|8 +Brand#52|-|LARGE PLATED TIN|-|3|-|8 +Brand#52|-|MEDIUM ANODIZED TIN|-|19|-|8 +Brand#52|-|MEDIUM BURNISHED COPPER|-|3|-|8 +Brand#52|-|PROMO POLISHED BRASS|-|23|-|8 +Brand#52|-|SMALL PLATED COPPER|-|36|-|8 +Brand#52|-|SMALL POLISHED NICKEL|-|9|-|8 +Brand#52|-|STANDARD POLISHED NICKEL|-|45|-|8 +Brand#53|-|ECONOMY POLISHED STEEL|-|45|-|8 +Brand#53|-|LARGE POLISHED NICKEL|-|3|-|8 +Brand#53|-|SMALL BRUSHED COPPER|-|14|-|8 +Brand#53|-|STANDARD PLATED STEEL|-|45|-|8 +Brand#54|-|ECONOMY POLISHED BRASS|-|49|-|8 +Brand#54|-|ECONOMY POLISHED TIN|-|23|-|8 +Brand#54|-|LARGE ANODIZED NICKEL|-|49|-|8 +Brand#54|-|MEDIUM BRUSHED STEEL|-|9|-|8 +Brand#54|-|SMALL BURNISHED NICKEL|-|14|-|8 +Brand#54|-|SMALL PLATED TIN|-|14|-|8 +Brand#54|-|STANDARD BURNISHED STEEL|-|14|-|8 +Brand#54|-|STANDARD PLATED BRASS|-|23|-|8 +Brand#55|-|MEDIUM BURNISHED TIN|-|36|-|8 +Brand#55|-|PROMO ANODIZED BRASS|-|14|-|8 +Brand#55|-|STANDARD BURNISHED COPPER|-|45|-|8 +Brand#15|-|STANDARD PLATED TIN|-|36|-|7 +Brand#23|-|SMALL POLISHED BRASS|-|49|-|7 +Brand#42|-|STANDARD PLATED COPPER|-|19|-|7 +Brand#51|-|LARGE POLISHED NICKEL|-|14|-|7 +Brand#11|-|ECONOMY ANODIZED BRASS|-|19|-|4 +Brand#11|-|ECONOMY ANODIZED BRASS|-|45|-|4 +Brand#11|-|ECONOMY ANODIZED NICKEL|-|36|-|4 +Brand#11|-|ECONOMY BRUSHED COPPER|-|3|-|4 +Brand#11|-|ECONOMY BRUSHED COPPER|-|9|-|4 +Brand#11|-|ECONOMY BRUSHED STEEL|-|9|-|4 +Brand#11|-|ECONOMY BRUSHED STEEL|-|36|-|4 +Brand#11|-|ECONOMY BURNISHED BRASS|-|36|-|4 +Brand#11|-|ECONOMY BURNISHED COPPER|-|9|-|4 +Brand#11|-|ECONOMY BURNISHED COPPER|-|49|-|4 +Brand#11|-|ECONOMY BURNISHED NICKEL|-|14|-|4 +Brand#11|-|ECONOMY BURNISHED NICKEL|-|49|-|4 +Brand#11|-|ECONOMY PLATED COPPER|-|19|-|4 +Brand#11|-|ECONOMY PLATED NICKEL|-|45|-|4 +Brand#11|-|ECONOMY PLATED TIN|-|9|-|4 +Brand#11|-|ECONOMY POLISHED BRASS|-|3|-|4 +Brand#11|-|ECONOMY POLISHED COPPER|-|3|-|4 +Brand#11|-|ECONOMY POLISHED COPPER|-|45|-|4 +Brand#11|-|ECONOMY POLISHED NICKEL|-|36|-|4 +Brand#11|-|ECONOMY POLISHED STEEL|-|23|-|4 +Brand#11|-|ECONOMY POLISHED TIN|-|14|-|4 +Brand#11|-|LARGE ANODIZED COPPER|-|23|-|4 +Brand#11|-|LARGE ANODIZED NICKEL|-|9|-|4 +Brand#11|-|LARGE ANODIZED STEEL|-|9|-|4 +Brand#11|-|LARGE ANODIZED TIN|-|45|-|4 +Brand#11|-|LARGE BRUSHED STEEL|-|19|-|4 +Brand#11|-|LARGE BRUSHED TIN|-|3|-|4 +Brand#11|-|LARGE BRUSHED TIN|-|14|-|4 +Brand#11|-|LARGE BURNISHED COPPER|-|9|-|4 +Brand#11|-|LARGE BURNISHED COPPER|-|19|-|4 +Brand#11|-|LARGE BURNISHED STEEL|-|23|-|4 +Brand#11|-|LARGE BURNISHED TIN|-|9|-|4 +Brand#11|-|LARGE PLATED COPPER|-|23|-|4 +Brand#11|-|LARGE PLATED TIN|-|9|-|4 +Brand#11|-|LARGE PLATED TIN|-|14|-|4 +Brand#11|-|LARGE PLATED TIN|-|23|-|4 +Brand#11|-|LARGE POLISHED NICKEL|-|49|-|4 +Brand#11|-|MEDIUM ANODIZED BRASS|-|45|-|4 +Brand#11|-|MEDIUM ANODIZED TIN|-|14|-|4 +Brand#11|-|MEDIUM BRUSHED BRASS|-|14|-|4 +Brand#11|-|MEDIUM BRUSHED BRASS|-|45|-|4 +Brand#11|-|MEDIUM BRUSHED NICKEL|-|14|-|4 +Brand#11|-|MEDIUM BRUSHED NICKEL|-|36|-|4 +Brand#11|-|MEDIUM BRUSHED STEEL|-|19|-|4 +Brand#11|-|MEDIUM BURNISHED COPPER|-|9|-|4 +Brand#11|-|MEDIUM BURNISHED TIN|-|36|-|4 +Brand#11|-|MEDIUM PLATED BRASS|-|3|-|4 +Brand#11|-|MEDIUM PLATED TIN|-|19|-|4 +Brand#11|-|PROMO ANODIZED BRASS|-|3|-|4 +Brand#11|-|PROMO ANODIZED BRASS|-|19|-|4 +Brand#11|-|PROMO ANODIZED BRASS|-|45|-|4 +Brand#11|-|PROMO ANODIZED BRASS|-|49|-|4 +Brand#11|-|PROMO ANODIZED STEEL|-|23|-|4 +Brand#11|-|PROMO ANODIZED TIN|-|45|-|4 +Brand#11|-|PROMO BRUSHED BRASS|-|23|-|4 +Brand#11|-|PROMO BRUSHED STEEL|-|3|-|4 +Brand#11|-|PROMO BURNISHED BRASS|-|23|-|4 +Brand#11|-|PROMO BURNISHED BRASS|-|36|-|4 +Brand#11|-|PROMO BURNISHED BRASS|-|49|-|4 +Brand#11|-|PROMO BURNISHED TIN|-|9|-|4 +Brand#11|-|PROMO PLATED BRASS|-|9|-|4 +Brand#11|-|PROMO PLATED BRASS|-|45|-|4 +Brand#11|-|PROMO PLATED NICKEL|-|19|-|4 +Brand#11|-|PROMO POLISHED BRASS|-|3|-|4 +Brand#11|-|PROMO POLISHED BRASS|-|9|-|4 +Brand#11|-|PROMO POLISHED BRASS|-|19|-|4 +Brand#11|-|PROMO POLISHED COPPER|-|14|-|4 +Brand#11|-|PROMO POLISHED COPPER|-|45|-|4 +Brand#11|-|PROMO POLISHED TIN|-|49|-|4 +Brand#11|-|SMALL ANODIZED COPPER|-|36|-|4 +Brand#11|-|SMALL ANODIZED NICKEL|-|3|-|4 +Brand#11|-|SMALL ANODIZED NICKEL|-|14|-|4 +Brand#11|-|SMALL ANODIZED TIN|-|14|-|4 +Brand#11|-|SMALL ANODIZED TIN|-|19|-|4 +Brand#11|-|SMALL ANODIZED TIN|-|45|-|4 +Brand#11|-|SMALL BRUSHED TIN|-|14|-|4 +Brand#11|-|SMALL BRUSHED TIN|-|23|-|4 +Brand#11|-|SMALL BRUSHED TIN|-|45|-|4 +Brand#11|-|SMALL BURNISHED BRASS|-|49|-|4 +Brand#11|-|SMALL BURNISHED COPPER|-|23|-|4 +Brand#11|-|SMALL PLATED COPPER|-|45|-|4 +Brand#11|-|SMALL PLATED NICKEL|-|3|-|4 +Brand#11|-|SMALL PLATED STEEL|-|36|-|4 +Brand#11|-|SMALL PLATED TIN|-|19|-|4 +Brand#11|-|SMALL POLISHED BRASS|-|14|-|4 +Brand#11|-|SMALL POLISHED BRASS|-|23|-|4 +Brand#11|-|SMALL POLISHED COPPER|-|14|-|4 +Brand#11|-|SMALL POLISHED COPPER|-|36|-|4 +Brand#11|-|SMALL POLISHED STEEL|-|9|-|4 +Brand#11|-|STANDARD BRUSHED COPPER|-|23|-|4 +Brand#11|-|STANDARD BRUSHED NICKEL|-|14|-|4 +Brand#11|-|STANDARD BRUSHED TIN|-|14|-|4 +Brand#11|-|STANDARD BURNISHED BRASS|-|3|-|4 +Brand#11|-|STANDARD BURNISHED STEEL|-|23|-|4 +Brand#11|-|STANDARD PLATED BRASS|-|19|-|4 +Brand#11|-|STANDARD PLATED TIN|-|19|-|4 +Brand#11|-|STANDARD POLISHED NICKEL|-|45|-|4 +Brand#11|-|STANDARD POLISHED TIN|-|14|-|4 +Brand#11|-|STANDARD POLISHED TIN|-|45|-|4 +Brand#12|-|ECONOMY ANODIZED BRASS|-|23|-|4 +Brand#12|-|ECONOMY ANODIZED COPPER|-|14|-|4 +Brand#12|-|ECONOMY ANODIZED NICKEL|-|19|-|4 +Brand#12|-|ECONOMY ANODIZED NICKEL|-|45|-|4 +Brand#12|-|ECONOMY ANODIZED STEEL|-|9|-|4 +Brand#12|-|ECONOMY BRUSHED COPPER|-|36|-|4 +Brand#12|-|ECONOMY BRUSHED NICKEL|-|49|-|4 +Brand#12|-|ECONOMY BRUSHED STEEL|-|49|-|4 +Brand#12|-|ECONOMY BURNISHED COPPER|-|45|-|4 +Brand#12|-|ECONOMY PLATED COPPER|-|23|-|4 +Brand#12|-|ECONOMY PLATED STEEL|-|23|-|4 +Brand#12|-|ECONOMY PLATED TIN|-|36|-|4 +Brand#12|-|ECONOMY POLISHED BRASS|-|14|-|4 +Brand#12|-|ECONOMY POLISHED COPPER|-|45|-|4 +Brand#12|-|ECONOMY POLISHED NICKEL|-|9|-|4 +Brand#12|-|LARGE ANODIZED NICKEL|-|9|-|4 +Brand#12|-|LARGE ANODIZED NICKEL|-|49|-|4 +Brand#12|-|LARGE ANODIZED STEEL|-|49|-|4 +Brand#12|-|LARGE ANODIZED TIN|-|36|-|4 +Brand#12|-|LARGE ANODIZED TIN|-|45|-|4 +Brand#12|-|LARGE BURNISHED BRASS|-|14|-|4 +Brand#12|-|LARGE BURNISHED BRASS|-|19|-|4 +Brand#12|-|LARGE BURNISHED COPPER|-|9|-|4 +Brand#12|-|LARGE BURNISHED NICKEL|-|45|-|4 +Brand#12|-|LARGE BURNISHED TIN|-|36|-|4 +Brand#12|-|LARGE PLATED BRASS|-|3|-|4 +Brand#12|-|LARGE PLATED STEEL|-|36|-|4 +Brand#12|-|LARGE PLATED STEEL|-|45|-|4 +Brand#12|-|LARGE PLATED TIN|-|23|-|4 +Brand#12|-|LARGE POLISHED COPPER|-|14|-|4 +Brand#12|-|LARGE POLISHED COPPER|-|19|-|4 +Brand#12|-|LARGE POLISHED COPPER|-|49|-|4 +Brand#12|-|LARGE POLISHED STEEL|-|3|-|4 +Brand#12|-|MEDIUM ANODIZED COPPER|-|9|-|4 +Brand#12|-|MEDIUM ANODIZED COPPER|-|45|-|4 +Brand#12|-|MEDIUM ANODIZED NICKEL|-|45|-|4 +Brand#12|-|MEDIUM BRUSHED BRASS|-|19|-|4 +Brand#12|-|MEDIUM BRUSHED COPPER|-|9|-|4 +Brand#12|-|MEDIUM BRUSHED COPPER|-|36|-|4 +Brand#12|-|MEDIUM BRUSHED COPPER|-|49|-|4 +Brand#12|-|MEDIUM BRUSHED NICKEL|-|3|-|4 +Brand#12|-|MEDIUM BRUSHED NICKEL|-|14|-|4 +Brand#12|-|MEDIUM BRUSHED NICKEL|-|23|-|4 +Brand#12|-|MEDIUM BURNISHED BRASS|-|3|-|4 +Brand#12|-|MEDIUM BURNISHED COPPER|-|36|-|4 +Brand#12|-|MEDIUM BURNISHED NICKEL|-|19|-|4 +Brand#12|-|MEDIUM BURNISHED TIN|-|14|-|4 +Brand#12|-|MEDIUM PLATED BRASS|-|23|-|4 +Brand#12|-|MEDIUM PLATED TIN|-|19|-|4 +Brand#12|-|MEDIUM PLATED TIN|-|23|-|4 +Brand#12|-|MEDIUM PLATED TIN|-|49|-|4 +Brand#12|-|PROMO ANODIZED BRASS|-|9|-|4 +Brand#12|-|PROMO ANODIZED BRASS|-|45|-|4 +Brand#12|-|PROMO ANODIZED NICKEL|-|14|-|4 +Brand#12|-|PROMO ANODIZED STEEL|-|49|-|4 +Brand#12|-|PROMO ANODIZED TIN|-|3|-|4 +Brand#12|-|PROMO ANODIZED TIN|-|19|-|4 +Brand#12|-|PROMO BRUSHED COPPER|-|14|-|4 +Brand#12|-|PROMO BRUSHED COPPER|-|19|-|4 +Brand#12|-|PROMO BRUSHED NICKEL|-|23|-|4 +Brand#12|-|PROMO BRUSHED STEEL|-|23|-|4 +Brand#12|-|PROMO BRUSHED STEEL|-|36|-|4 +Brand#12|-|PROMO BURNISHED BRASS|-|49|-|4 +Brand#12|-|PROMO BURNISHED TIN|-|9|-|4 +Brand#12|-|PROMO BURNISHED TIN|-|14|-|4 +Brand#12|-|PROMO PLATED BRASS|-|36|-|4 +Brand#12|-|PROMO POLISHED COPPER|-|23|-|4 +Brand#12|-|PROMO POLISHED NICKEL|-|3|-|4 +Brand#12|-|PROMO POLISHED NICKEL|-|9|-|4 +Brand#12|-|PROMO POLISHED STEEL|-|14|-|4 +Brand#12|-|PROMO POLISHED TIN|-|23|-|4 +Brand#12|-|PROMO POLISHED TIN|-|36|-|4 +Brand#12|-|SMALL ANODIZED BRASS|-|36|-|4 +Brand#12|-|SMALL ANODIZED COPPER|-|23|-|4 +Brand#12|-|SMALL ANODIZED STEEL|-|36|-|4 +Brand#12|-|SMALL ANODIZED TIN|-|14|-|4 +Brand#12|-|SMALL BRUSHED COPPER|-|19|-|4 +Brand#12|-|SMALL BRUSHED COPPER|-|36|-|4 +Brand#12|-|SMALL BRUSHED TIN|-|36|-|4 +Brand#12|-|SMALL BURNISHED BRASS|-|14|-|4 +Brand#12|-|SMALL BURNISHED COPPER|-|9|-|4 +Brand#12|-|SMALL BURNISHED COPPER|-|36|-|4 +Brand#12|-|SMALL PLATED BRASS|-|9|-|4 +Brand#12|-|SMALL POLISHED BRASS|-|49|-|4 +Brand#12|-|SMALL POLISHED NICKEL|-|19|-|4 +Brand#12|-|SMALL POLISHED TIN|-|3|-|4 +Brand#12|-|STANDARD ANODIZED BRASS|-|19|-|4 +Brand#12|-|STANDARD ANODIZED NICKEL|-|19|-|4 +Brand#12|-|STANDARD ANODIZED STEEL|-|19|-|4 +Brand#12|-|STANDARD BRUSHED COPPER|-|36|-|4 +Brand#12|-|STANDARD BRUSHED NICKEL|-|23|-|4 +Brand#12|-|STANDARD BRUSHED STEEL|-|49|-|4 +Brand#12|-|STANDARD BURNISHED BRASS|-|23|-|4 +Brand#12|-|STANDARD BURNISHED COPPER|-|14|-|4 +Brand#12|-|STANDARD BURNISHED NICKEL|-|45|-|4 +Brand#12|-|STANDARD BURNISHED NICKEL|-|49|-|4 +Brand#12|-|STANDARD BURNISHED TIN|-|3|-|4 +Brand#12|-|STANDARD BURNISHED TIN|-|14|-|4 +Brand#12|-|STANDARD PLATED BRASS|-|19|-|4 +Brand#12|-|STANDARD PLATED NICKEL|-|45|-|4 +Brand#12|-|STANDARD PLATED STEEL|-|36|-|4 +Brand#12|-|STANDARD PLATED STEEL|-|45|-|4 +Brand#12|-|STANDARD PLATED TIN|-|9|-|4 +Brand#12|-|STANDARD POLISHED BRASS|-|49|-|4 +Brand#12|-|STANDARD POLISHED COPPER|-|3|-|4 +Brand#12|-|STANDARD POLISHED NICKEL|-|23|-|4 +Brand#12|-|STANDARD POLISHED TIN|-|14|-|4 +Brand#13|-|ECONOMY ANODIZED NICKEL|-|14|-|4 +Brand#13|-|ECONOMY ANODIZED NICKEL|-|19|-|4 +Brand#13|-|ECONOMY ANODIZED STEEL|-|45|-|4 +Brand#13|-|ECONOMY ANODIZED STEEL|-|49|-|4 +Brand#13|-|ECONOMY BRUSHED BRASS|-|3|-|4 +Brand#13|-|ECONOMY BURNISHED STEEL|-|14|-|4 +Brand#13|-|ECONOMY BURNISHED TIN|-|19|-|4 +Brand#13|-|ECONOMY BURNISHED TIN|-|45|-|4 +Brand#13|-|ECONOMY PLATED COPPER|-|19|-|4 +Brand#13|-|ECONOMY PLATED NICKEL|-|3|-|4 +Brand#13|-|ECONOMY PLATED STEEL|-|23|-|4 +Brand#13|-|ECONOMY PLATED TIN|-|3|-|4 +Brand#13|-|ECONOMY POLISHED BRASS|-|3|-|4 +Brand#13|-|ECONOMY POLISHED COPPER|-|9|-|4 +Brand#13|-|ECONOMY POLISHED COPPER|-|49|-|4 +Brand#13|-|ECONOMY POLISHED STEEL|-|23|-|4 +Brand#13|-|ECONOMY POLISHED STEEL|-|49|-|4 +Brand#13|-|LARGE ANODIZED BRASS|-|23|-|4 +Brand#13|-|LARGE ANODIZED COPPER|-|19|-|4 +Brand#13|-|LARGE ANODIZED NICKEL|-|9|-|4 +Brand#13|-|LARGE ANODIZED STEEL|-|45|-|4 +Brand#13|-|LARGE ANODIZED TIN|-|19|-|4 +Brand#13|-|LARGE BRUSHED BRASS|-|3|-|4 +Brand#13|-|LARGE BRUSHED BRASS|-|9|-|4 +Brand#13|-|LARGE BRUSHED BRASS|-|19|-|4 +Brand#13|-|LARGE BRUSHED COPPER|-|9|-|4 +Brand#13|-|LARGE BRUSHED COPPER|-|36|-|4 +Brand#13|-|LARGE BRUSHED NICKEL|-|3|-|4 +Brand#13|-|LARGE BRUSHED NICKEL|-|9|-|4 +Brand#13|-|LARGE BRUSHED NICKEL|-|14|-|4 +Brand#13|-|LARGE BRUSHED NICKEL|-|23|-|4 +Brand#13|-|LARGE BRUSHED STEEL|-|19|-|4 +Brand#13|-|LARGE BRUSHED TIN|-|49|-|4 +Brand#13|-|LARGE BURNISHED BRASS|-|49|-|4 +Brand#13|-|LARGE BURNISHED TIN|-|49|-|4 +Brand#13|-|LARGE PLATED COPPER|-|23|-|4 +Brand#13|-|LARGE PLATED STEEL|-|14|-|4 +Brand#13|-|LARGE PLATED STEEL|-|19|-|4 +Brand#13|-|LARGE PLATED STEEL|-|36|-|4 +Brand#13|-|LARGE PLATED TIN|-|14|-|4 +Brand#13|-|LARGE PLATED TIN|-|45|-|4 +Brand#13|-|LARGE POLISHED BRASS|-|3|-|4 +Brand#13|-|LARGE POLISHED BRASS|-|23|-|4 +Brand#13|-|LARGE POLISHED BRASS|-|49|-|4 +Brand#13|-|MEDIUM ANODIZED BRASS|-|3|-|4 +Brand#13|-|MEDIUM ANODIZED BRASS|-|36|-|4 +Brand#13|-|MEDIUM ANODIZED COPPER|-|14|-|4 +Brand#13|-|MEDIUM ANODIZED NICKEL|-|3|-|4 +Brand#13|-|MEDIUM ANODIZED STEEL|-|14|-|4 +Brand#13|-|MEDIUM ANODIZED STEEL|-|19|-|4 +Brand#13|-|MEDIUM ANODIZED STEEL|-|36|-|4 +Brand#13|-|MEDIUM BRUSHED BRASS|-|49|-|4 +Brand#13|-|MEDIUM BRUSHED COPPER|-|23|-|4 +Brand#13|-|MEDIUM BRUSHED NICKEL|-|45|-|4 +Brand#13|-|MEDIUM BURNISHED BRASS|-|9|-|4 +Brand#13|-|MEDIUM BURNISHED STEEL|-|19|-|4 +Brand#13|-|MEDIUM BURNISHED STEEL|-|49|-|4 +Brand#13|-|MEDIUM PLATED BRASS|-|3|-|4 +Brand#13|-|MEDIUM PLATED BRASS|-|23|-|4 +Brand#13|-|MEDIUM PLATED BRASS|-|36|-|4 +Brand#13|-|MEDIUM PLATED COPPER|-|19|-|4 +Brand#13|-|MEDIUM PLATED COPPER|-|23|-|4 +Brand#13|-|MEDIUM PLATED STEEL|-|3|-|4 +Brand#13|-|PROMO ANODIZED BRASS|-|14|-|4 +Brand#13|-|PROMO ANODIZED COPPER|-|9|-|4 +Brand#13|-|PROMO ANODIZED COPPER|-|45|-|4 +Brand#13|-|PROMO ANODIZED STEEL|-|23|-|4 +Brand#13|-|PROMO BRUSHED COPPER|-|49|-|4 +Brand#13|-|PROMO BURNISHED COPPER|-|19|-|4 +Brand#13|-|PROMO BURNISHED NICKEL|-|9|-|4 +Brand#13|-|PROMO BURNISHED STEEL|-|23|-|4 +Brand#13|-|PROMO BURNISHED STEEL|-|45|-|4 +Brand#13|-|PROMO BURNISHED TIN|-|19|-|4 +Brand#13|-|PROMO PLATED BRASS|-|14|-|4 +Brand#13|-|PROMO PLATED BRASS|-|19|-|4 +Brand#13|-|PROMO PLATED COPPER|-|3|-|4 +Brand#13|-|PROMO PLATED COPPER|-|19|-|4 +Brand#13|-|PROMO PLATED TIN|-|19|-|4 +Brand#13|-|PROMO POLISHED BRASS|-|49|-|4 +Brand#13|-|PROMO POLISHED STEEL|-|45|-|4 +Brand#13|-|PROMO POLISHED TIN|-|14|-|4 +Brand#13|-|SMALL ANODIZED STEEL|-|23|-|4 +Brand#13|-|SMALL ANODIZED TIN|-|3|-|4 +Brand#13|-|SMALL ANODIZED TIN|-|45|-|4 +Brand#13|-|SMALL BRUSHED COPPER|-|3|-|4 +Brand#13|-|SMALL BRUSHED NICKEL|-|19|-|4 +Brand#13|-|SMALL BRUSHED TIN|-|9|-|4 +Brand#13|-|SMALL BRUSHED TIN|-|45|-|4 +Brand#13|-|SMALL BURNISHED BRASS|-|19|-|4 +Brand#13|-|SMALL BURNISHED BRASS|-|45|-|4 +Brand#13|-|SMALL PLATED BRASS|-|9|-|4 +Brand#13|-|SMALL PLATED TIN|-|45|-|4 +Brand#13|-|SMALL POLISHED NICKEL|-|19|-|4 +Brand#13|-|SMALL POLISHED STEEL|-|49|-|4 +Brand#13|-|STANDARD ANODIZED COPPER|-|45|-|4 +Brand#13|-|STANDARD ANODIZED NICKEL|-|9|-|4 +Brand#13|-|STANDARD ANODIZED NICKEL|-|19|-|4 +Brand#13|-|STANDARD ANODIZED STEEL|-|14|-|4 +Brand#13|-|STANDARD ANODIZED TIN|-|9|-|4 +Brand#13|-|STANDARD ANODIZED TIN|-|36|-|4 +Brand#13|-|STANDARD BRUSHED BRASS|-|19|-|4 +Brand#13|-|STANDARD BRUSHED TIN|-|9|-|4 +Brand#13|-|STANDARD BURNISHED BRASS|-|9|-|4 +Brand#13|-|STANDARD BURNISHED BRASS|-|14|-|4 +Brand#13|-|STANDARD BURNISHED COPPER|-|45|-|4 +Brand#13|-|STANDARD PLATED BRASS|-|49|-|4 +Brand#13|-|STANDARD PLATED COPPER|-|19|-|4 +Brand#13|-|STANDARD PLATED NICKEL|-|23|-|4 +Brand#13|-|STANDARD PLATED TIN|-|9|-|4 +Brand#13|-|STANDARD POLISHED BRASS|-|49|-|4 +Brand#13|-|STANDARD POLISHED COPPER|-|9|-|4 +Brand#13|-|STANDARD POLISHED COPPER|-|49|-|4 +Brand#13|-|STANDARD POLISHED NICKEL|-|14|-|4 +Brand#13|-|STANDARD POLISHED NICKEL|-|19|-|4 +Brand#13|-|STANDARD POLISHED STEEL|-|23|-|4 +Brand#14|-|ECONOMY ANODIZED BRASS|-|19|-|4 +Brand#14|-|ECONOMY ANODIZED COPPER|-|9|-|4 +Brand#14|-|ECONOMY ANODIZED STEEL|-|19|-|4 +Brand#14|-|ECONOMY ANODIZED STEEL|-|45|-|4 +Brand#14|-|ECONOMY BRUSHED BRASS|-|19|-|4 +Brand#14|-|ECONOMY BRUSHED COPPER|-|45|-|4 +Brand#14|-|ECONOMY BRUSHED NICKEL|-|14|-|4 +Brand#14|-|ECONOMY BRUSHED TIN|-|14|-|4 +Brand#14|-|ECONOMY BURNISHED COPPER|-|9|-|4 +Brand#14|-|ECONOMY BURNISHED COPPER|-|19|-|4 +Brand#14|-|ECONOMY BURNISHED STEEL|-|36|-|4 +Brand#14|-|ECONOMY BURNISHED TIN|-|3|-|4 +Brand#14|-|ECONOMY PLATED BRASS|-|36|-|4 +Brand#14|-|ECONOMY PLATED COPPER|-|49|-|4 +Brand#14|-|ECONOMY PLATED STEEL|-|45|-|4 +Brand#14|-|ECONOMY PLATED TIN|-|9|-|4 +Brand#14|-|ECONOMY POLISHED COPPER|-|3|-|4 +Brand#14|-|ECONOMY POLISHED TIN|-|19|-|4 +Brand#14|-|LARGE ANODIZED COPPER|-|9|-|4 +Brand#14|-|LARGE ANODIZED COPPER|-|23|-|4 +Brand#14|-|LARGE ANODIZED NICKEL|-|3|-|4 +Brand#14|-|LARGE ANODIZED NICKEL|-|9|-|4 +Brand#14|-|LARGE ANODIZED NICKEL|-|19|-|4 +Brand#14|-|LARGE ANODIZED TIN|-|9|-|4 +Brand#14|-|LARGE BRUSHED COPPER|-|14|-|4 +Brand#14|-|LARGE BRUSHED NICKEL|-|45|-|4 +Brand#14|-|LARGE PLATED BRASS|-|3|-|4 +Brand#14|-|LARGE PLATED NICKEL|-|3|-|4 +Brand#14|-|LARGE PLATED NICKEL|-|14|-|4 +Brand#14|-|LARGE PLATED NICKEL|-|49|-|4 +Brand#14|-|LARGE PLATED TIN|-|49|-|4 +Brand#14|-|LARGE POLISHED BRASS|-|9|-|4 +Brand#14|-|LARGE POLISHED BRASS|-|14|-|4 +Brand#14|-|LARGE POLISHED BRASS|-|36|-|4 +Brand#14|-|LARGE POLISHED NICKEL|-|3|-|4 +Brand#14|-|LARGE POLISHED NICKEL|-|14|-|4 +Brand#14|-|LARGE POLISHED STEEL|-|9|-|4 +Brand#14|-|LARGE POLISHED STEEL|-|23|-|4 +Brand#14|-|LARGE POLISHED STEEL|-|36|-|4 +Brand#14|-|MEDIUM ANODIZED NICKEL|-|3|-|4 +Brand#14|-|MEDIUM ANODIZED NICKEL|-|49|-|4 +Brand#14|-|MEDIUM ANODIZED STEEL|-|23|-|4 +Brand#14|-|MEDIUM ANODIZED STEEL|-|36|-|4 +Brand#14|-|MEDIUM BRUSHED BRASS|-|9|-|4 +Brand#14|-|MEDIUM BRUSHED COPPER|-|23|-|4 +Brand#14|-|MEDIUM BRUSHED STEEL|-|14|-|4 +Brand#14|-|MEDIUM BURNISHED COPPER|-|14|-|4 +Brand#14|-|MEDIUM BURNISHED STEEL|-|3|-|4 +Brand#14|-|MEDIUM BURNISHED STEEL|-|49|-|4 +Brand#14|-|MEDIUM PLATED BRASS|-|36|-|4 +Brand#14|-|MEDIUM PLATED STEEL|-|49|-|4 +Brand#14|-|MEDIUM PLATED TIN|-|14|-|4 +Brand#14|-|PROMO ANODIZED BRASS|-|49|-|4 +Brand#14|-|PROMO ANODIZED STEEL|-|36|-|4 +Brand#14|-|PROMO BRUSHED STEEL|-|19|-|4 +Brand#14|-|PROMO BURNISHED BRASS|-|23|-|4 +Brand#14|-|PROMO BURNISHED STEEL|-|36|-|4 +Brand#14|-|PROMO PLATED BRASS|-|9|-|4 +Brand#14|-|PROMO PLATED BRASS|-|45|-|4 +Brand#14|-|PROMO PLATED COPPER|-|45|-|4 +Brand#14|-|PROMO PLATED STEEL|-|3|-|4 +Brand#14|-|PROMO POLISHED BRASS|-|9|-|4 +Brand#14|-|PROMO POLISHED COPPER|-|49|-|4 +Brand#14|-|PROMO POLISHED STEEL|-|19|-|4 +Brand#14|-|SMALL ANODIZED STEEL|-|23|-|4 +Brand#14|-|SMALL ANODIZED TIN|-|23|-|4 +Brand#14|-|SMALL BRUSHED BRASS|-|19|-|4 +Brand#14|-|SMALL BRUSHED BRASS|-|36|-|4 +Brand#14|-|SMALL BRUSHED COPPER|-|9|-|4 +Brand#14|-|SMALL BRUSHED TIN|-|36|-|4 +Brand#14|-|SMALL BURNISHED BRASS|-|45|-|4 +Brand#14|-|SMALL BURNISHED COPPER|-|9|-|4 +Brand#14|-|SMALL BURNISHED COPPER|-|14|-|4 +Brand#14|-|SMALL BURNISHED COPPER|-|45|-|4 +Brand#14|-|SMALL BURNISHED NICKEL|-|36|-|4 +Brand#14|-|SMALL BURNISHED STEEL|-|36|-|4 +Brand#14|-|SMALL BURNISHED TIN|-|23|-|4 +Brand#14|-|SMALL PLATED NICKEL|-|3|-|4 +Brand#14|-|SMALL PLATED NICKEL|-|9|-|4 +Brand#14|-|SMALL PLATED STEEL|-|14|-|4 +Brand#14|-|SMALL POLISHED BRASS|-|36|-|4 +Brand#14|-|SMALL POLISHED COPPER|-|36|-|4 +Brand#14|-|SMALL POLISHED NICKEL|-|9|-|4 +Brand#14|-|SMALL POLISHED STEEL|-|14|-|4 +Brand#14|-|SMALL POLISHED TIN|-|14|-|4 +Brand#14|-|STANDARD ANODIZED BRASS|-|19|-|4 +Brand#14|-|STANDARD ANODIZED NICKEL|-|14|-|4 +Brand#14|-|STANDARD ANODIZED STEEL|-|9|-|4 +Brand#14|-|STANDARD BRUSHED COPPER|-|45|-|4 +Brand#14|-|STANDARD BRUSHED NICKEL|-|45|-|4 +Brand#14|-|STANDARD BRUSHED TIN|-|45|-|4 +Brand#14|-|STANDARD BURNISHED BRASS|-|9|-|4 +Brand#14|-|STANDARD BURNISHED BRASS|-|23|-|4 +Brand#14|-|STANDARD BURNISHED BRASS|-|49|-|4 +Brand#14|-|STANDARD BURNISHED NICKEL|-|9|-|4 +Brand#14|-|STANDARD PLATED BRASS|-|36|-|4 +Brand#14|-|STANDARD PLATED COPPER|-|45|-|4 +Brand#14|-|STANDARD POLISHED NICKEL|-|3|-|4 +Brand#14|-|STANDARD POLISHED NICKEL|-|9|-|4 +Brand#14|-|STANDARD POLISHED TIN|-|19|-|4 +Brand#15|-|ECONOMY ANODIZED COPPER|-|14|-|4 +Brand#15|-|ECONOMY ANODIZED STEEL|-|19|-|4 +Brand#15|-|ECONOMY ANODIZED STEEL|-|36|-|4 +Brand#15|-|ECONOMY BRUSHED BRASS|-|36|-|4 +Brand#15|-|ECONOMY BRUSHED COPPER|-|14|-|4 +Brand#15|-|ECONOMY BRUSHED NICKEL|-|14|-|4 +Brand#15|-|ECONOMY BRUSHED STEEL|-|3|-|4 +Brand#15|-|ECONOMY BRUSHED TIN|-|3|-|4 +Brand#15|-|ECONOMY BURNISHED BRASS|-|14|-|4 +Brand#15|-|ECONOMY BURNISHED COPPER|-|3|-|4 +Brand#15|-|ECONOMY BURNISHED COPPER|-|23|-|4 +Brand#15|-|ECONOMY PLATED NICKEL|-|49|-|4 +Brand#15|-|ECONOMY PLATED STEEL|-|3|-|4 +Brand#15|-|ECONOMY PLATED STEEL|-|19|-|4 +Brand#15|-|ECONOMY PLATED STEEL|-|45|-|4 +Brand#15|-|LARGE ANODIZED BRASS|-|19|-|4 +Brand#15|-|LARGE ANODIZED BRASS|-|36|-|4 +Brand#15|-|LARGE ANODIZED BRASS|-|45|-|4 +Brand#15|-|LARGE ANODIZED COPPER|-|3|-|4 +Brand#15|-|LARGE ANODIZED NICKEL|-|9|-|4 +Brand#15|-|LARGE ANODIZED TIN|-|19|-|4 +Brand#15|-|LARGE BRUSHED BRASS|-|9|-|4 +Brand#15|-|LARGE BRUSHED BRASS|-|19|-|4 +Brand#15|-|LARGE BRUSHED COPPER|-|14|-|4 +Brand#15|-|LARGE BRUSHED STEEL|-|9|-|4 +Brand#15|-|LARGE BRUSHED STEEL|-|14|-|4 +Brand#15|-|LARGE BRUSHED STEEL|-|19|-|4 +Brand#15|-|LARGE BRUSHED STEEL|-|36|-|4 +Brand#15|-|LARGE BURNISHED BRASS|-|14|-|4 +Brand#15|-|LARGE BURNISHED BRASS|-|19|-|4 +Brand#15|-|LARGE BURNISHED COPPER|-|9|-|4 +Brand#15|-|LARGE BURNISHED COPPER|-|45|-|4 +Brand#15|-|LARGE BURNISHED TIN|-|49|-|4 +Brand#15|-|LARGE PLATED BRASS|-|19|-|4 +Brand#15|-|LARGE PLATED COPPER|-|3|-|4 +Brand#15|-|LARGE PLATED COPPER|-|23|-|4 +Brand#15|-|LARGE PLATED NICKEL|-|36|-|4 +Brand#15|-|MEDIUM ANODIZED BRASS|-|23|-|4 +Brand#15|-|MEDIUM ANODIZED COPPER|-|9|-|4 +Brand#15|-|MEDIUM ANODIZED NICKEL|-|3|-|4 +Brand#15|-|MEDIUM ANODIZED TIN|-|19|-|4 +Brand#15|-|MEDIUM BRUSHED BRASS|-|9|-|4 +Brand#15|-|MEDIUM BRUSHED TIN|-|23|-|4 +Brand#15|-|MEDIUM BURNISHED COPPER|-|36|-|4 +Brand#15|-|MEDIUM BURNISHED TIN|-|45|-|4 +Brand#15|-|MEDIUM PLATED COPPER|-|9|-|4 +Brand#15|-|MEDIUM PLATED NICKEL|-|9|-|4 +Brand#15|-|MEDIUM PLATED NICKEL|-|19|-|4 +Brand#15|-|MEDIUM PLATED STEEL|-|36|-|4 +Brand#15|-|MEDIUM PLATED STEEL|-|49|-|4 +Brand#15|-|MEDIUM PLATED TIN|-|9|-|4 +Brand#15|-|MEDIUM PLATED TIN|-|14|-|4 +Brand#15|-|MEDIUM PLATED TIN|-|23|-|4 +Brand#15|-|PROMO ANODIZED COPPER|-|23|-|4 +Brand#15|-|PROMO ANODIZED STEEL|-|14|-|4 +Brand#15|-|PROMO ANODIZED TIN|-|45|-|4 +Brand#15|-|PROMO BRUSHED COPPER|-|14|-|4 +Brand#15|-|PROMO BRUSHED COPPER|-|19|-|4 +Brand#15|-|PROMO BRUSHED NICKEL|-|19|-|4 +Brand#15|-|PROMO BRUSHED NICKEL|-|23|-|4 +Brand#15|-|PROMO BRUSHED STEEL|-|14|-|4 +Brand#15|-|PROMO BRUSHED TIN|-|36|-|4 +Brand#15|-|PROMO BURNISHED NICKEL|-|9|-|4 +Brand#15|-|PROMO BURNISHED STEEL|-|45|-|4 +Brand#15|-|PROMO PLATED COPPER|-|3|-|4 +Brand#15|-|PROMO PLATED COPPER|-|36|-|4 +Brand#15|-|PROMO PLATED STEEL|-|3|-|4 +Brand#15|-|PROMO PLATED TIN|-|49|-|4 +Brand#15|-|PROMO POLISHED COPPER|-|3|-|4 +Brand#15|-|PROMO POLISHED NICKEL|-|36|-|4 +Brand#15|-|PROMO POLISHED STEEL|-|36|-|4 +Brand#15|-|PROMO POLISHED TIN|-|49|-|4 +Brand#15|-|SMALL ANODIZED BRASS|-|14|-|4 +Brand#15|-|SMALL ANODIZED BRASS|-|19|-|4 +Brand#15|-|SMALL ANODIZED COPPER|-|9|-|4 +Brand#15|-|SMALL ANODIZED TIN|-|45|-|4 +Brand#15|-|SMALL BRUSHED BRASS|-|3|-|4 +Brand#15|-|SMALL BRUSHED COPPER|-|19|-|4 +Brand#15|-|SMALL BRUSHED STEEL|-|23|-|4 +Brand#15|-|SMALL BRUSHED TIN|-|45|-|4 +Brand#15|-|SMALL BURNISHED BRASS|-|19|-|4 +Brand#15|-|SMALL BURNISHED COPPER|-|14|-|4 +Brand#15|-|SMALL BURNISHED NICKEL|-|19|-|4 +Brand#15|-|SMALL BURNISHED NICKEL|-|49|-|4 +Brand#15|-|SMALL BURNISHED STEEL|-|9|-|4 +Brand#15|-|SMALL BURNISHED TIN|-|19|-|4 +Brand#15|-|SMALL BURNISHED TIN|-|23|-|4 +Brand#15|-|SMALL BURNISHED TIN|-|36|-|4 +Brand#15|-|SMALL PLATED BRASS|-|3|-|4 +Brand#15|-|SMALL PLATED COPPER|-|23|-|4 +Brand#15|-|SMALL PLATED COPPER|-|49|-|4 +Brand#15|-|SMALL PLATED NICKEL|-|36|-|4 +Brand#15|-|SMALL PLATED NICKEL|-|45|-|4 +Brand#15|-|SMALL PLATED STEEL|-|3|-|4 +Brand#15|-|SMALL PLATED TIN|-|9|-|4 +Brand#15|-|SMALL POLISHED COPPER|-|9|-|4 +Brand#15|-|SMALL POLISHED NICKEL|-|3|-|4 +Brand#15|-|SMALL POLISHED STEEL|-|19|-|4 +Brand#15|-|SMALL POLISHED STEEL|-|36|-|4 +Brand#15|-|SMALL POLISHED TIN|-|19|-|4 +Brand#15|-|SMALL POLISHED TIN|-|49|-|4 +Brand#15|-|STANDARD ANODIZED NICKEL|-|19|-|4 +Brand#15|-|STANDARD ANODIZED NICKEL|-|49|-|4 +Brand#15|-|STANDARD ANODIZED TIN|-|36|-|4 +Brand#15|-|STANDARD BRUSHED NICKEL|-|3|-|4 +Brand#15|-|STANDARD BURNISHED BRASS|-|23|-|4 +Brand#15|-|STANDARD BURNISHED STEEL|-|3|-|4 +Brand#15|-|STANDARD BURNISHED STEEL|-|45|-|4 +Brand#15|-|STANDARD PLATED BRASS|-|36|-|4 +Brand#15|-|STANDARD PLATED COPPER|-|14|-|4 +Brand#15|-|STANDARD PLATED COPPER|-|23|-|4 +Brand#15|-|STANDARD PLATED NICKEL|-|19|-|4 +Brand#15|-|STANDARD PLATED TIN|-|45|-|4 +Brand#15|-|STANDARD POLISHED BRASS|-|14|-|4 +Brand#15|-|STANDARD POLISHED COPPER|-|23|-|4 +Brand#15|-|STANDARD POLISHED NICKEL|-|45|-|4 +Brand#21|-|ECONOMY ANODIZED BRASS|-|3|-|4 +Brand#21|-|ECONOMY ANODIZED NICKEL|-|14|-|4 +Brand#21|-|ECONOMY ANODIZED STEEL|-|19|-|4 +Brand#21|-|ECONOMY ANODIZED STEEL|-|23|-|4 +Brand#21|-|ECONOMY ANODIZED STEEL|-|49|-|4 +Brand#21|-|ECONOMY ANODIZED TIN|-|19|-|4 +Brand#21|-|ECONOMY BRUSHED BRASS|-|9|-|4 +Brand#21|-|ECONOMY BRUSHED BRASS|-|14|-|4 +Brand#21|-|ECONOMY BRUSHED BRASS|-|36|-|4 +Brand#21|-|ECONOMY BRUSHED COPPER|-|49|-|4 +Brand#21|-|ECONOMY BRUSHED STEEL|-|45|-|4 +Brand#21|-|ECONOMY BRUSHED TIN|-|49|-|4 +Brand#21|-|ECONOMY BURNISHED BRASS|-|3|-|4 +Brand#21|-|ECONOMY BURNISHED COPPER|-|45|-|4 +Brand#21|-|ECONOMY BURNISHED STEEL|-|19|-|4 +Brand#21|-|ECONOMY BURNISHED STEEL|-|36|-|4 +Brand#21|-|ECONOMY PLATED BRASS|-|36|-|4 +Brand#21|-|ECONOMY PLATED COPPER|-|3|-|4 +Brand#21|-|ECONOMY PLATED COPPER|-|14|-|4 +Brand#21|-|ECONOMY PLATED NICKEL|-|49|-|4 +Brand#21|-|ECONOMY POLISHED NICKEL|-|3|-|4 +Brand#21|-|ECONOMY POLISHED NICKEL|-|9|-|4 +Brand#21|-|LARGE ANODIZED COPPER|-|3|-|4 +Brand#21|-|LARGE ANODIZED COPPER|-|9|-|4 +Brand#21|-|LARGE ANODIZED STEEL|-|36|-|4 +Brand#21|-|LARGE ANODIZED TIN|-|45|-|4 +Brand#21|-|LARGE BRUSHED COPPER|-|45|-|4 +Brand#21|-|LARGE BRUSHED STEEL|-|23|-|4 +Brand#21|-|LARGE BURNISHED BRASS|-|49|-|4 +Brand#21|-|LARGE BURNISHED COPPER|-|19|-|4 +Brand#21|-|LARGE BURNISHED STEEL|-|49|-|4 +Brand#21|-|LARGE BURNISHED TIN|-|49|-|4 +Brand#21|-|LARGE PLATED BRASS|-|19|-|4 +Brand#21|-|LARGE PLATED NICKEL|-|23|-|4 +Brand#21|-|LARGE PLATED NICKEL|-|49|-|4 +Brand#21|-|LARGE PLATED TIN|-|19|-|4 +Brand#21|-|LARGE POLISHED BRASS|-|49|-|4 +Brand#21|-|LARGE POLISHED COPPER|-|14|-|4 +Brand#21|-|LARGE POLISHED NICKEL|-|3|-|4 +Brand#21|-|LARGE POLISHED NICKEL|-|14|-|4 +Brand#21|-|LARGE POLISHED STEEL|-|14|-|4 +Brand#21|-|LARGE POLISHED TIN|-|49|-|4 +Brand#21|-|MEDIUM ANODIZED COPPER|-|14|-|4 +Brand#21|-|MEDIUM ANODIZED NICKEL|-|49|-|4 +Brand#21|-|MEDIUM BRUSHED COPPER|-|3|-|4 +Brand#21|-|MEDIUM BRUSHED COPPER|-|49|-|4 +Brand#21|-|MEDIUM BRUSHED STEEL|-|23|-|4 +Brand#21|-|MEDIUM BRUSHED TIN|-|3|-|4 +Brand#21|-|MEDIUM BRUSHED TIN|-|14|-|4 +Brand#21|-|MEDIUM BURNISHED NICKEL|-|14|-|4 +Brand#21|-|MEDIUM BURNISHED STEEL|-|23|-|4 +Brand#21|-|MEDIUM BURNISHED TIN|-|3|-|4 +Brand#21|-|MEDIUM PLATED BRASS|-|3|-|4 +Brand#21|-|MEDIUM PLATED BRASS|-|19|-|4 +Brand#21|-|MEDIUM PLATED STEEL|-|36|-|4 +Brand#21|-|PROMO ANODIZED BRASS|-|9|-|4 +Brand#21|-|PROMO ANODIZED COPPER|-|14|-|4 +Brand#21|-|PROMO ANODIZED NICKEL|-|23|-|4 +Brand#21|-|PROMO ANODIZED STEEL|-|3|-|4 +Brand#21|-|PROMO ANODIZED STEEL|-|14|-|4 +Brand#21|-|PROMO ANODIZED STEEL|-|36|-|4 +Brand#21|-|PROMO BRUSHED NICKEL|-|45|-|4 +Brand#21|-|PROMO BRUSHED STEEL|-|14|-|4 +Brand#21|-|PROMO BRUSHED STEEL|-|23|-|4 +Brand#21|-|PROMO BRUSHED STEEL|-|45|-|4 +Brand#21|-|PROMO BURNISHED BRASS|-|19|-|4 +Brand#21|-|PROMO BURNISHED COPPER|-|19|-|4 +Brand#21|-|PROMO BURNISHED NICKEL|-|9|-|4 +Brand#21|-|PROMO BURNISHED TIN|-|19|-|4 +Brand#21|-|PROMO PLATED NICKEL|-|9|-|4 +Brand#21|-|PROMO PLATED NICKEL|-|36|-|4 +Brand#21|-|PROMO PLATED STEEL|-|49|-|4 +Brand#21|-|PROMO PLATED TIN|-|3|-|4 +Brand#21|-|PROMO POLISHED NICKEL|-|23|-|4 +Brand#21|-|PROMO POLISHED TIN|-|14|-|4 +Brand#21|-|PROMO POLISHED TIN|-|19|-|4 +Brand#21|-|PROMO POLISHED TIN|-|23|-|4 +Brand#21|-|SMALL BRUSHED BRASS|-|23|-|4 +Brand#21|-|SMALL BRUSHED COPPER|-|49|-|4 +Brand#21|-|SMALL BURNISHED BRASS|-|23|-|4 +Brand#21|-|SMALL BURNISHED BRASS|-|36|-|4 +Brand#21|-|SMALL BURNISHED STEEL|-|19|-|4 +Brand#21|-|SMALL BURNISHED TIN|-|19|-|4 +Brand#21|-|SMALL PLATED BRASS|-|45|-|4 +Brand#21|-|SMALL PLATED COPPER|-|45|-|4 +Brand#21|-|SMALL PLATED STEEL|-|45|-|4 +Brand#21|-|SMALL PLATED TIN|-|14|-|4 +Brand#21|-|SMALL PLATED TIN|-|45|-|4 +Brand#21|-|SMALL POLISHED COPPER|-|9|-|4 +Brand#21|-|SMALL POLISHED NICKEL|-|23|-|4 +Brand#21|-|SMALL POLISHED TIN|-|3|-|4 +Brand#21|-|STANDARD ANODIZED BRASS|-|9|-|4 +Brand#21|-|STANDARD ANODIZED NICKEL|-|19|-|4 +Brand#21|-|STANDARD ANODIZED TIN|-|45|-|4 +Brand#21|-|STANDARD BURNISHED COPPER|-|36|-|4 +Brand#21|-|STANDARD BURNISHED NICKEL|-|23|-|4 +Brand#21|-|STANDARD BURNISHED TIN|-|9|-|4 +Brand#21|-|STANDARD PLATED BRASS|-|14|-|4 +Brand#21|-|STANDARD PLATED COPPER|-|19|-|4 +Brand#21|-|STANDARD PLATED NICKEL|-|3|-|4 +Brand#21|-|STANDARD PLATED STEEL|-|9|-|4 +Brand#21|-|STANDARD PLATED TIN|-|9|-|4 +Brand#21|-|STANDARD POLISHED BRASS|-|9|-|4 +Brand#21|-|STANDARD POLISHED COPPER|-|49|-|4 +Brand#21|-|STANDARD POLISHED STEEL|-|36|-|4 +Brand#21|-|STANDARD POLISHED TIN|-|36|-|4 +Brand#22|-|ECONOMY ANODIZED STEEL|-|9|-|4 +Brand#22|-|ECONOMY ANODIZED STEEL|-|14|-|4 +Brand#22|-|ECONOMY ANODIZED STEEL|-|23|-|4 +Brand#22|-|ECONOMY ANODIZED TIN|-|9|-|4 +Brand#22|-|ECONOMY ANODIZED TIN|-|36|-|4 +Brand#22|-|ECONOMY BRUSHED NICKEL|-|36|-|4 +Brand#22|-|ECONOMY BRUSHED NICKEL|-|45|-|4 +Brand#22|-|ECONOMY BURNISHED BRASS|-|9|-|4 +Brand#22|-|ECONOMY BURNISHED BRASS|-|23|-|4 +Brand#22|-|ECONOMY BURNISHED BRASS|-|45|-|4 +Brand#22|-|ECONOMY BURNISHED NICKEL|-|19|-|4 +Brand#22|-|ECONOMY BURNISHED NICKEL|-|49|-|4 +Brand#22|-|ECONOMY BURNISHED STEEL|-|9|-|4 +Brand#22|-|ECONOMY BURNISHED STEEL|-|14|-|4 +Brand#22|-|ECONOMY BURNISHED STEEL|-|23|-|4 +Brand#22|-|ECONOMY PLATED BRASS|-|36|-|4 +Brand#22|-|ECONOMY PLATED COPPER|-|23|-|4 +Brand#22|-|ECONOMY PLATED TIN|-|3|-|4 +Brand#22|-|ECONOMY POLISHED TIN|-|49|-|4 +Brand#22|-|LARGE ANODIZED BRASS|-|19|-|4 +Brand#22|-|LARGE ANODIZED COPPER|-|36|-|4 +Brand#22|-|LARGE ANODIZED STEEL|-|3|-|4 +Brand#22|-|LARGE BRUSHED BRASS|-|23|-|4 +Brand#22|-|LARGE BRUSHED BRASS|-|49|-|4 +Brand#22|-|LARGE BRUSHED STEEL|-|49|-|4 +Brand#22|-|LARGE BURNISHED COPPER|-|19|-|4 +Brand#22|-|LARGE BURNISHED STEEL|-|23|-|4 +Brand#22|-|LARGE BURNISHED STEEL|-|45|-|4 +Brand#22|-|LARGE BURNISHED TIN|-|45|-|4 +Brand#22|-|LARGE PLATED COPPER|-|14|-|4 +Brand#22|-|LARGE PLATED STEEL|-|49|-|4 +Brand#22|-|LARGE POLISHED BRASS|-|19|-|4 +Brand#22|-|LARGE POLISHED COPPER|-|19|-|4 +Brand#22|-|LARGE POLISHED COPPER|-|23|-|4 +Brand#22|-|LARGE POLISHED NICKEL|-|19|-|4 +Brand#22|-|LARGE POLISHED TIN|-|49|-|4 +Brand#22|-|MEDIUM ANODIZED BRASS|-|45|-|4 +Brand#22|-|MEDIUM ANODIZED COPPER|-|19|-|4 +Brand#22|-|MEDIUM ANODIZED COPPER|-|49|-|4 +Brand#22|-|MEDIUM ANODIZED NICKEL|-|9|-|4 +Brand#22|-|MEDIUM ANODIZED NICKEL|-|14|-|4 +Brand#22|-|MEDIUM ANODIZED NICKEL|-|36|-|4 +Brand#22|-|MEDIUM ANODIZED TIN|-|3|-|4 +Brand#22|-|MEDIUM ANODIZED TIN|-|9|-|4 +Brand#22|-|MEDIUM BRUSHED BRASS|-|3|-|4 +Brand#22|-|MEDIUM BRUSHED BRASS|-|14|-|4 +Brand#22|-|MEDIUM BRUSHED COPPER|-|3|-|4 +Brand#22|-|MEDIUM BRUSHED COPPER|-|45|-|4 +Brand#22|-|MEDIUM BRUSHED NICKEL|-|14|-|4 +Brand#22|-|MEDIUM BRUSHED TIN|-|45|-|4 +Brand#22|-|MEDIUM BURNISHED COPPER|-|36|-|4 +Brand#22|-|MEDIUM BURNISHED TIN|-|19|-|4 +Brand#22|-|MEDIUM BURNISHED TIN|-|23|-|4 +Brand#22|-|MEDIUM BURNISHED TIN|-|49|-|4 +Brand#22|-|MEDIUM PLATED BRASS|-|49|-|4 +Brand#22|-|MEDIUM PLATED COPPER|-|9|-|4 +Brand#22|-|MEDIUM PLATED STEEL|-|3|-|4 +Brand#22|-|PROMO ANODIZED BRASS|-|9|-|4 +Brand#22|-|PROMO ANODIZED STEEL|-|36|-|4 +Brand#22|-|PROMO ANODIZED TIN|-|45|-|4 +Brand#22|-|PROMO BRUSHED BRASS|-|3|-|4 +Brand#22|-|PROMO BRUSHED BRASS|-|9|-|4 +Brand#22|-|PROMO BRUSHED BRASS|-|36|-|4 +Brand#22|-|PROMO BRUSHED STEEL|-|36|-|4 +Brand#22|-|PROMO BURNISHED BRASS|-|23|-|4 +Brand#22|-|PROMO BURNISHED COPPER|-|9|-|4 +Brand#22|-|PROMO PLATED BRASS|-|14|-|4 +Brand#22|-|PROMO PLATED BRASS|-|45|-|4 +Brand#22|-|PROMO PLATED NICKEL|-|3|-|4 +Brand#22|-|PROMO PLATED STEEL|-|19|-|4 +Brand#22|-|PROMO POLISHED BRASS|-|3|-|4 +Brand#22|-|PROMO POLISHED STEEL|-|14|-|4 +Brand#22|-|PROMO POLISHED STEEL|-|23|-|4 +Brand#22|-|SMALL ANODIZED TIN|-|36|-|4 +Brand#22|-|SMALL ANODIZED TIN|-|49|-|4 +Brand#22|-|SMALL BRUSHED NICKEL|-|3|-|4 +Brand#22|-|SMALL BRUSHED NICKEL|-|36|-|4 +Brand#22|-|SMALL BRUSHED NICKEL|-|45|-|4 +Brand#22|-|SMALL BRUSHED TIN|-|45|-|4 +Brand#22|-|SMALL BURNISHED STEEL|-|23|-|4 +Brand#22|-|SMALL BURNISHED TIN|-|14|-|4 +Brand#22|-|SMALL PLATED STEEL|-|3|-|4 +Brand#22|-|SMALL PLATED TIN|-|9|-|4 +Brand#22|-|SMALL PLATED TIN|-|36|-|4 +Brand#22|-|SMALL POLISHED BRASS|-|23|-|4 +Brand#22|-|SMALL POLISHED NICKEL|-|19|-|4 +Brand#22|-|STANDARD ANODIZED BRASS|-|14|-|4 +Brand#22|-|STANDARD ANODIZED BRASS|-|23|-|4 +Brand#22|-|STANDARD BRUSHED COPPER|-|49|-|4 +Brand#22|-|STANDARD BRUSHED NICKEL|-|3|-|4 +Brand#22|-|STANDARD BRUSHED NICKEL|-|23|-|4 +Brand#22|-|STANDARD BRUSHED STEEL|-|9|-|4 +Brand#22|-|STANDARD BRUSHED TIN|-|19|-|4 +Brand#22|-|STANDARD BURNISHED COPPER|-|45|-|4 +Brand#22|-|STANDARD BURNISHED NICKEL|-|3|-|4 +Brand#22|-|STANDARD BURNISHED NICKEL|-|14|-|4 +Brand#22|-|STANDARD BURNISHED NICKEL|-|45|-|4 +Brand#22|-|STANDARD BURNISHED STEEL|-|3|-|4 +Brand#22|-|STANDARD BURNISHED STEEL|-|36|-|4 +Brand#22|-|STANDARD BURNISHED STEEL|-|45|-|4 +Brand#22|-|STANDARD BURNISHED STEEL|-|49|-|4 +Brand#22|-|STANDARD PLATED BRASS|-|45|-|4 +Brand#22|-|STANDARD PLATED NICKEL|-|3|-|4 +Brand#22|-|STANDARD PLATED NICKEL|-|45|-|4 +Brand#22|-|STANDARD PLATED STEEL|-|14|-|4 +Brand#22|-|STANDARD PLATED TIN|-|19|-|4 +Brand#22|-|STANDARD PLATED TIN|-|49|-|4 +Brand#22|-|STANDARD POLISHED COPPER|-|9|-|4 +Brand#22|-|STANDARD POLISHED STEEL|-|49|-|4 +Brand#22|-|STANDARD POLISHED TIN|-|45|-|4 +Brand#23|-|ECONOMY ANODIZED NICKEL|-|49|-|4 +Brand#23|-|ECONOMY ANODIZED STEEL|-|14|-|4 +Brand#23|-|ECONOMY ANODIZED STEEL|-|49|-|4 +Brand#23|-|ECONOMY ANODIZED TIN|-|49|-|4 +Brand#23|-|ECONOMY BRUSHED BRASS|-|3|-|4 +Brand#23|-|ECONOMY BRUSHED COPPER|-|9|-|4 +Brand#23|-|ECONOMY BRUSHED TIN|-|9|-|4 +Brand#23|-|ECONOMY BURNISHED STEEL|-|49|-|4 +Brand#23|-|ECONOMY PLATED COPPER|-|14|-|4 +Brand#23|-|ECONOMY PLATED NICKEL|-|23|-|4 +Brand#23|-|ECONOMY PLATED STEEL|-|14|-|4 +Brand#23|-|ECONOMY POLISHED NICKEL|-|9|-|4 +Brand#23|-|LARGE ANODIZED BRASS|-|14|-|4 +Brand#23|-|LARGE ANODIZED COPPER|-|9|-|4 +Brand#23|-|LARGE ANODIZED COPPER|-|14|-|4 +Brand#23|-|LARGE ANODIZED COPPER|-|45|-|4 +Brand#23|-|LARGE ANODIZED STEEL|-|19|-|4 +Brand#23|-|LARGE ANODIZED STEEL|-|36|-|4 +Brand#23|-|LARGE ANODIZED STEEL|-|49|-|4 +Brand#23|-|LARGE ANODIZED TIN|-|9|-|4 +Brand#23|-|LARGE PLATED BRASS|-|9|-|4 +Brand#23|-|LARGE PLATED BRASS|-|49|-|4 +Brand#23|-|LARGE PLATED COPPER|-|3|-|4 +Brand#23|-|LARGE POLISHED BRASS|-|45|-|4 +Brand#23|-|LARGE POLISHED STEEL|-|9|-|4 +Brand#23|-|MEDIUM ANODIZED BRASS|-|19|-|4 +Brand#23|-|MEDIUM ANODIZED NICKEL|-|3|-|4 +Brand#23|-|MEDIUM ANODIZED NICKEL|-|14|-|4 +Brand#23|-|MEDIUM ANODIZED STEEL|-|45|-|4 +Brand#23|-|MEDIUM ANODIZED TIN|-|36|-|4 +Brand#23|-|MEDIUM ANODIZED TIN|-|45|-|4 +Brand#23|-|MEDIUM BRUSHED COPPER|-|3|-|4 +Brand#23|-|MEDIUM BRUSHED COPPER|-|23|-|4 +Brand#23|-|MEDIUM BRUSHED NICKEL|-|3|-|4 +Brand#23|-|MEDIUM BRUSHED TIN|-|14|-|4 +Brand#23|-|MEDIUM BURNISHED BRASS|-|9|-|4 +Brand#23|-|MEDIUM BURNISHED BRASS|-|45|-|4 +Brand#23|-|MEDIUM BURNISHED COPPER|-|19|-|4 +Brand#23|-|MEDIUM PLATED COPPER|-|19|-|4 +Brand#23|-|MEDIUM PLATED COPPER|-|36|-|4 +Brand#23|-|MEDIUM PLATED COPPER|-|45|-|4 +Brand#23|-|MEDIUM PLATED NICKEL|-|9|-|4 +Brand#23|-|MEDIUM PLATED NICKEL|-|14|-|4 +Brand#23|-|PROMO ANODIZED COPPER|-|9|-|4 +Brand#23|-|PROMO ANODIZED COPPER|-|19|-|4 +Brand#23|-|PROMO ANODIZED STEEL|-|36|-|4 +Brand#23|-|PROMO ANODIZED TIN|-|14|-|4 +Brand#23|-|PROMO BRUSHED BRASS|-|3|-|4 +Brand#23|-|PROMO BRUSHED BRASS|-|19|-|4 +Brand#23|-|PROMO BRUSHED BRASS|-|36|-|4 +Brand#23|-|PROMO BRUSHED COPPER|-|3|-|4 +Brand#23|-|PROMO BRUSHED TIN|-|49|-|4 +Brand#23|-|PROMO BURNISHED BRASS|-|14|-|4 +Brand#23|-|PROMO BURNISHED BRASS|-|45|-|4 +Brand#23|-|PROMO BURNISHED COPPER|-|14|-|4 +Brand#23|-|PROMO PLATED BRASS|-|23|-|4 +Brand#23|-|PROMO POLISHED BRASS|-|14|-|4 +Brand#23|-|PROMO POLISHED BRASS|-|23|-|4 +Brand#23|-|PROMO POLISHED COPPER|-|36|-|4 +Brand#23|-|PROMO POLISHED STEEL|-|36|-|4 +Brand#23|-|SMALL ANODIZED BRASS|-|23|-|4 +Brand#23|-|SMALL ANODIZED STEEL|-|23|-|4 +Brand#23|-|SMALL BRUSHED BRASS|-|49|-|4 +Brand#23|-|SMALL BRUSHED COPPER|-|45|-|4 +Brand#23|-|SMALL BRUSHED STEEL|-|3|-|4 +Brand#23|-|SMALL BRUSHED STEEL|-|19|-|4 +Brand#23|-|SMALL BURNISHED BRASS|-|36|-|4 +Brand#23|-|SMALL BURNISHED COPPER|-|45|-|4 +Brand#23|-|SMALL BURNISHED COPPER|-|49|-|4 +Brand#23|-|SMALL BURNISHED STEEL|-|45|-|4 +Brand#23|-|SMALL PLATED BRASS|-|36|-|4 +Brand#23|-|SMALL PLATED BRASS|-|49|-|4 +Brand#23|-|SMALL PLATED COPPER|-|14|-|4 +Brand#23|-|SMALL PLATED TIN|-|14|-|4 +Brand#23|-|SMALL POLISHED BRASS|-|9|-|4 +Brand#23|-|SMALL POLISHED BRASS|-|14|-|4 +Brand#23|-|SMALL POLISHED NICKEL|-|3|-|4 +Brand#23|-|SMALL POLISHED STEEL|-|14|-|4 +Brand#23|-|SMALL POLISHED TIN|-|9|-|4 +Brand#23|-|STANDARD ANODIZED BRASS|-|19|-|4 +Brand#23|-|STANDARD ANODIZED BRASS|-|45|-|4 +Brand#23|-|STANDARD ANODIZED COPPER|-|19|-|4 +Brand#23|-|STANDARD ANODIZED TIN|-|3|-|4 +Brand#23|-|STANDARD BRUSHED COPPER|-|36|-|4 +Brand#23|-|STANDARD BRUSHED NICKEL|-|19|-|4 +Brand#23|-|STANDARD BRUSHED STEEL|-|49|-|4 +Brand#23|-|STANDARD BURNISHED COPPER|-|19|-|4 +Brand#23|-|STANDARD PLATED BRASS|-|3|-|4 +Brand#23|-|STANDARD PLATED BRASS|-|9|-|4 +Brand#23|-|STANDARD PLATED STEEL|-|36|-|4 +Brand#23|-|STANDARD PLATED TIN|-|19|-|4 +Brand#23|-|STANDARD POLISHED BRASS|-|9|-|4 +Brand#23|-|STANDARD POLISHED BRASS|-|49|-|4 +Brand#23|-|STANDARD POLISHED STEEL|-|19|-|4 +Brand#23|-|STANDARD POLISHED STEEL|-|49|-|4 +Brand#23|-|STANDARD POLISHED TIN|-|23|-|4 +Brand#24|-|ECONOMY ANODIZED BRASS|-|3|-|4 +Brand#24|-|ECONOMY ANODIZED BRASS|-|9|-|4 +Brand#24|-|ECONOMY ANODIZED BRASS|-|23|-|4 +Brand#24|-|ECONOMY ANODIZED COPPER|-|9|-|4 +Brand#24|-|ECONOMY ANODIZED COPPER|-|49|-|4 +Brand#24|-|ECONOMY BRUSHED BRASS|-|36|-|4 +Brand#24|-|ECONOMY BRUSHED COPPER|-|23|-|4 +Brand#24|-|ECONOMY BURNISHED COPPER|-|3|-|4 +Brand#24|-|ECONOMY BURNISHED NICKEL|-|19|-|4 +Brand#24|-|ECONOMY BURNISHED STEEL|-|45|-|4 +Brand#24|-|ECONOMY PLATED BRASS|-|23|-|4 +Brand#24|-|ECONOMY PLATED COPPER|-|36|-|4 +Brand#24|-|ECONOMY PLATED STEEL|-|45|-|4 +Brand#24|-|ECONOMY POLISHED BRASS|-|23|-|4 +Brand#24|-|ECONOMY POLISHED COPPER|-|45|-|4 +Brand#24|-|ECONOMY POLISHED NICKEL|-|36|-|4 +Brand#24|-|ECONOMY POLISHED STEEL|-|14|-|4 +Brand#24|-|ECONOMY POLISHED STEEL|-|36|-|4 +Brand#24|-|LARGE ANODIZED NICKEL|-|23|-|4 +Brand#24|-|LARGE ANODIZED NICKEL|-|45|-|4 +Brand#24|-|LARGE ANODIZED TIN|-|45|-|4 +Brand#24|-|LARGE BRUSHED BRASS|-|14|-|4 +Brand#24|-|LARGE BRUSHED BRASS|-|23|-|4 +Brand#24|-|LARGE BRUSHED STEEL|-|9|-|4 +Brand#24|-|LARGE BRUSHED STEEL|-|23|-|4 +Brand#24|-|LARGE BRUSHED STEEL|-|45|-|4 +Brand#24|-|LARGE BRUSHED TIN|-|49|-|4 +Brand#24|-|LARGE BURNISHED BRASS|-|3|-|4 +Brand#24|-|LARGE BURNISHED NICKEL|-|19|-|4 +Brand#24|-|LARGE PLATED BRASS|-|9|-|4 +Brand#24|-|LARGE PLATED NICKEL|-|36|-|4 +Brand#24|-|LARGE PLATED NICKEL|-|49|-|4 +Brand#24|-|LARGE PLATED TIN|-|9|-|4 +Brand#24|-|LARGE PLATED TIN|-|19|-|4 +Brand#24|-|LARGE PLATED TIN|-|36|-|4 +Brand#24|-|LARGE PLATED TIN|-|49|-|4 +Brand#24|-|LARGE POLISHED BRASS|-|9|-|4 +Brand#24|-|LARGE POLISHED COPPER|-|9|-|4 +Brand#24|-|LARGE POLISHED COPPER|-|49|-|4 +Brand#24|-|LARGE POLISHED NICKEL|-|19|-|4 +Brand#24|-|LARGE POLISHED STEEL|-|23|-|4 +Brand#24|-|LARGE POLISHED TIN|-|14|-|4 +Brand#24|-|MEDIUM ANODIZED COPPER|-|45|-|4 +Brand#24|-|MEDIUM BRUSHED COPPER|-|9|-|4 +Brand#24|-|MEDIUM BRUSHED COPPER|-|14|-|4 +Brand#24|-|MEDIUM BRUSHED NICKEL|-|9|-|4 +Brand#24|-|MEDIUM BRUSHED NICKEL|-|23|-|4 +Brand#24|-|MEDIUM BRUSHED STEEL|-|14|-|4 +Brand#24|-|MEDIUM BRUSHED STEEL|-|45|-|4 +Brand#24|-|MEDIUM BRUSHED STEEL|-|49|-|4 +Brand#24|-|MEDIUM BURNISHED BRASS|-|36|-|4 +Brand#24|-|MEDIUM BURNISHED NICKEL|-|36|-|4 +Brand#24|-|MEDIUM BURNISHED STEEL|-|36|-|4 +Brand#24|-|MEDIUM PLATED COPPER|-|14|-|4 +Brand#24|-|MEDIUM PLATED STEEL|-|3|-|4 +Brand#24|-|MEDIUM PLATED STEEL|-|19|-|4 +Brand#24|-|PROMO ANODIZED NICKEL|-|9|-|4 +Brand#24|-|PROMO ANODIZED NICKEL|-|19|-|4 +Brand#24|-|PROMO ANODIZED NICKEL|-|45|-|4 +Brand#24|-|PROMO ANODIZED STEEL|-|3|-|4 +Brand#24|-|PROMO ANODIZED TIN|-|45|-|4 +Brand#24|-|PROMO BRUSHED BRASS|-|19|-|4 +Brand#24|-|PROMO BRUSHED NICKEL|-|19|-|4 +Brand#24|-|PROMO BRUSHED NICKEL|-|45|-|4 +Brand#24|-|PROMO BRUSHED STEEL|-|49|-|4 +Brand#24|-|PROMO BURNISHED BRASS|-|3|-|4 +Brand#24|-|PROMO BURNISHED BRASS|-|45|-|4 +Brand#24|-|PROMO BURNISHED STEEL|-|49|-|4 +Brand#24|-|PROMO PLATED BRASS|-|3|-|4 +Brand#24|-|PROMO PLATED COPPER|-|23|-|4 +Brand#24|-|PROMO PLATED COPPER|-|49|-|4 +Brand#24|-|PROMO POLISHED BRASS|-|3|-|4 +Brand#24|-|PROMO POLISHED BRASS|-|14|-|4 +Brand#24|-|PROMO POLISHED NICKEL|-|3|-|4 +Brand#24|-|PROMO POLISHED STEEL|-|14|-|4 +Brand#24|-|PROMO POLISHED STEEL|-|19|-|4 +Brand#24|-|PROMO POLISHED STEEL|-|23|-|4 +Brand#24|-|SMALL ANODIZED BRASS|-|19|-|4 +Brand#24|-|SMALL ANODIZED COPPER|-|3|-|4 +Brand#24|-|SMALL ANODIZED NICKEL|-|14|-|4 +Brand#24|-|SMALL ANODIZED STEEL|-|36|-|4 +Brand#24|-|SMALL ANODIZED TIN|-|3|-|4 +Brand#24|-|SMALL ANODIZED TIN|-|36|-|4 +Brand#24|-|SMALL BRUSHED COPPER|-|49|-|4 +Brand#24|-|SMALL BRUSHED NICKEL|-|49|-|4 +Brand#24|-|SMALL BURNISHED BRASS|-|14|-|4 +Brand#24|-|SMALL BURNISHED BRASS|-|19|-|4 +Brand#24|-|SMALL BURNISHED TIN|-|9|-|4 +Brand#24|-|SMALL PLATED BRASS|-|3|-|4 +Brand#24|-|SMALL PLATED COPPER|-|14|-|4 +Brand#24|-|SMALL PLATED COPPER|-|36|-|4 +Brand#24|-|SMALL PLATED NICKEL|-|14|-|4 +Brand#24|-|SMALL PLATED NICKEL|-|49|-|4 +Brand#24|-|SMALL POLISHED BRASS|-|3|-|4 +Brand#24|-|SMALL POLISHED NICKEL|-|9|-|4 +Brand#24|-|SMALL POLISHED NICKEL|-|19|-|4 +Brand#24|-|SMALL POLISHED NICKEL|-|36|-|4 +Brand#24|-|SMALL POLISHED STEEL|-|9|-|4 +Brand#24|-|SMALL POLISHED STEEL|-|36|-|4 +Brand#24|-|STANDARD ANODIZED TIN|-|9|-|4 +Brand#24|-|STANDARD ANODIZED TIN|-|49|-|4 +Brand#24|-|STANDARD BRUSHED BRASS|-|14|-|4 +Brand#24|-|STANDARD BRUSHED COPPER|-|23|-|4 +Brand#24|-|STANDARD BRUSHED NICKEL|-|19|-|4 +Brand#24|-|STANDARD BRUSHED STEEL|-|14|-|4 +Brand#24|-|STANDARD BRUSHED TIN|-|36|-|4 +Brand#24|-|STANDARD BURNISHED COPPER|-|19|-|4 +Brand#24|-|STANDARD BURNISHED COPPER|-|36|-|4 +Brand#24|-|STANDARD BURNISHED NICKEL|-|45|-|4 +Brand#24|-|STANDARD PLATED BRASS|-|36|-|4 +Brand#24|-|STANDARD PLATED COPPER|-|45|-|4 +Brand#24|-|STANDARD PLATED NICKEL|-|36|-|4 +Brand#24|-|STANDARD PLATED TIN|-|36|-|4 +Brand#24|-|STANDARD POLISHED COPPER|-|45|-|4 +Brand#24|-|STANDARD POLISHED NICKEL|-|14|-|4 +Brand#25|-|ECONOMY ANODIZED BRASS|-|14|-|4 +Brand#25|-|ECONOMY ANODIZED BRASS|-|49|-|4 +Brand#25|-|ECONOMY ANODIZED TIN|-|9|-|4 +Brand#25|-|ECONOMY ANODIZED TIN|-|19|-|4 +Brand#25|-|ECONOMY ANODIZED TIN|-|49|-|4 +Brand#25|-|ECONOMY BRUSHED COPPER|-|36|-|4 +Brand#25|-|ECONOMY BURNISHED COPPER|-|45|-|4 +Brand#25|-|ECONOMY BURNISHED TIN|-|19|-|4 +Brand#25|-|ECONOMY PLATED NICKEL|-|23|-|4 +Brand#25|-|ECONOMY PLATED TIN|-|14|-|4 +Brand#25|-|ECONOMY POLISHED BRASS|-|23|-|4 +Brand#25|-|ECONOMY POLISHED COPPER|-|9|-|4 +Brand#25|-|ECONOMY POLISHED NICKEL|-|3|-|4 +Brand#25|-|ECONOMY POLISHED TIN|-|9|-|4 +Brand#25|-|ECONOMY POLISHED TIN|-|45|-|4 +Brand#25|-|LARGE ANODIZED BRASS|-|3|-|4 +Brand#25|-|LARGE ANODIZED BRASS|-|14|-|4 +Brand#25|-|LARGE ANODIZED COPPER|-|36|-|4 +Brand#25|-|LARGE ANODIZED NICKEL|-|23|-|4 +Brand#25|-|LARGE ANODIZED STEEL|-|23|-|4 +Brand#25|-|LARGE BRUSHED NICKEL|-|19|-|4 +Brand#25|-|LARGE BRUSHED NICKEL|-|49|-|4 +Brand#25|-|LARGE BRUSHED TIN|-|3|-|4 +Brand#25|-|LARGE BRUSHED TIN|-|9|-|4 +Brand#25|-|LARGE BURNISHED BRASS|-|19|-|4 +Brand#25|-|LARGE BURNISHED BRASS|-|23|-|4 +Brand#25|-|LARGE BURNISHED BRASS|-|49|-|4 +Brand#25|-|LARGE BURNISHED NICKEL|-|14|-|4 +Brand#25|-|LARGE BURNISHED TIN|-|49|-|4 +Brand#25|-|LARGE PLATED BRASS|-|14|-|4 +Brand#25|-|LARGE PLATED NICKEL|-|23|-|4 +Brand#25|-|LARGE PLATED NICKEL|-|45|-|4 +Brand#25|-|LARGE PLATED TIN|-|19|-|4 +Brand#25|-|LARGE PLATED TIN|-|23|-|4 +Brand#25|-|LARGE POLISHED BRASS|-|9|-|4 +Brand#25|-|LARGE POLISHED COPPER|-|14|-|4 +Brand#25|-|LARGE POLISHED COPPER|-|36|-|4 +Brand#25|-|MEDIUM ANODIZED TIN|-|36|-|4 +Brand#25|-|MEDIUM BRUSHED COPPER|-|9|-|4 +Brand#25|-|MEDIUM BRUSHED COPPER|-|36|-|4 +Brand#25|-|MEDIUM BRUSHED COPPER|-|49|-|4 +Brand#25|-|MEDIUM BURNISHED COPPER|-|49|-|4 +Brand#25|-|MEDIUM BURNISHED NICKEL|-|9|-|4 +Brand#25|-|MEDIUM BURNISHED NICKEL|-|49|-|4 +Brand#25|-|MEDIUM BURNISHED STEEL|-|3|-|4 +Brand#25|-|MEDIUM BURNISHED STEEL|-|36|-|4 +Brand#25|-|MEDIUM BURNISHED STEEL|-|45|-|4 +Brand#25|-|MEDIUM BURNISHED STEEL|-|49|-|4 +Brand#25|-|MEDIUM BURNISHED TIN|-|9|-|4 +Brand#25|-|MEDIUM BURNISHED TIN|-|36|-|4 +Brand#25|-|MEDIUM PLATED BRASS|-|45|-|4 +Brand#25|-|MEDIUM PLATED COPPER|-|14|-|4 +Brand#25|-|MEDIUM PLATED NICKEL|-|45|-|4 +Brand#25|-|MEDIUM PLATED STEEL|-|9|-|4 +Brand#25|-|MEDIUM PLATED STEEL|-|36|-|4 +Brand#25|-|PROMO ANODIZED COPPER|-|14|-|4 +Brand#25|-|PROMO ANODIZED COPPER|-|19|-|4 +Brand#25|-|PROMO ANODIZED STEEL|-|36|-|4 +Brand#25|-|PROMO ANODIZED TIN|-|3|-|4 +Brand#25|-|PROMO ANODIZED TIN|-|14|-|4 +Brand#25|-|PROMO BRUSHED NICKEL|-|3|-|4 +Brand#25|-|PROMO BRUSHED STEEL|-|19|-|4 +Brand#25|-|PROMO BRUSHED TIN|-|14|-|4 +Brand#25|-|PROMO BRUSHED TIN|-|36|-|4 +Brand#25|-|PROMO BURNISHED COPPER|-|19|-|4 +Brand#25|-|PROMO BURNISHED COPPER|-|45|-|4 +Brand#25|-|PROMO BURNISHED COPPER|-|49|-|4 +Brand#25|-|PROMO BURNISHED NICKEL|-|36|-|4 +Brand#25|-|PROMO BURNISHED TIN|-|3|-|4 +Brand#25|-|PROMO PLATED BRASS|-|45|-|4 +Brand#25|-|PROMO PLATED COPPER|-|19|-|4 +Brand#25|-|PROMO PLATED NICKEL|-|45|-|4 +Brand#25|-|PROMO PLATED NICKEL|-|49|-|4 +Brand#25|-|PROMO PLATED STEEL|-|23|-|4 +Brand#25|-|PROMO POLISHED BRASS|-|23|-|4 +Brand#25|-|SMALL ANODIZED BRASS|-|45|-|4 +Brand#25|-|SMALL ANODIZED NICKEL|-|19|-|4 +Brand#25|-|SMALL ANODIZED STEEL|-|23|-|4 +Brand#25|-|SMALL ANODIZED TIN|-|14|-|4 +Brand#25|-|SMALL ANODIZED TIN|-|19|-|4 +Brand#25|-|SMALL BRUSHED COPPER|-|45|-|4 +Brand#25|-|SMALL BRUSHED NICKEL|-|9|-|4 +Brand#25|-|SMALL BURNISHED COPPER|-|3|-|4 +Brand#25|-|SMALL BURNISHED STEEL|-|3|-|4 +Brand#25|-|SMALL BURNISHED STEEL|-|14|-|4 +Brand#25|-|SMALL BURNISHED TIN|-|3|-|4 +Brand#25|-|SMALL PLATED BRASS|-|19|-|4 +Brand#25|-|SMALL PLATED COPPER|-|23|-|4 +Brand#25|-|SMALL PLATED STEEL|-|45|-|4 +Brand#25|-|SMALL PLATED TIN|-|36|-|4 +Brand#25|-|SMALL POLISHED BRASS|-|23|-|4 +Brand#25|-|SMALL POLISHED COPPER|-|9|-|4 +Brand#25|-|SMALL POLISHED STEEL|-|14|-|4 +Brand#25|-|STANDARD ANODIZED STEEL|-|3|-|4 +Brand#25|-|STANDARD ANODIZED STEEL|-|19|-|4 +Brand#25|-|STANDARD ANODIZED TIN|-|9|-|4 +Brand#25|-|STANDARD BRUSHED BRASS|-|14|-|4 +Brand#25|-|STANDARD BRUSHED NICKEL|-|19|-|4 +Brand#25|-|STANDARD BRUSHED TIN|-|9|-|4 +Brand#25|-|STANDARD BURNISHED NICKEL|-|9|-|4 +Brand#25|-|STANDARD PLATED BRASS|-|3|-|4 +Brand#25|-|STANDARD PLATED COPPER|-|14|-|4 +Brand#25|-|STANDARD PLATED NICKEL|-|36|-|4 +Brand#25|-|STANDARD POLISHED BRASS|-|45|-|4 +Brand#25|-|STANDARD POLISHED COPPER|-|23|-|4 +Brand#25|-|STANDARD POLISHED NICKEL|-|3|-|4 +Brand#25|-|STANDARD POLISHED NICKEL|-|49|-|4 +Brand#25|-|STANDARD POLISHED TIN|-|36|-|4 +Brand#25|-|STANDARD POLISHED TIN|-|45|-|4 +Brand#31|-|ECONOMY ANODIZED BRASS|-|3|-|4 +Brand#31|-|ECONOMY ANODIZED COPPER|-|45|-|4 +Brand#31|-|ECONOMY ANODIZED STEEL|-|3|-|4 +Brand#31|-|ECONOMY ANODIZED TIN|-|45|-|4 +Brand#31|-|ECONOMY BRUSHED BRASS|-|14|-|4 +Brand#31|-|ECONOMY BRUSHED COPPER|-|19|-|4 +Brand#31|-|ECONOMY BRUSHED NICKEL|-|9|-|4 +Brand#31|-|ECONOMY BRUSHED NICKEL|-|14|-|4 +Brand#31|-|ECONOMY BRUSHED NICKEL|-|49|-|4 +Brand#31|-|ECONOMY BURNISHED COPPER|-|36|-|4 +Brand#31|-|ECONOMY BURNISHED STEEL|-|3|-|4 +Brand#31|-|ECONOMY BURNISHED TIN|-|49|-|4 +Brand#31|-|ECONOMY PLATED COPPER|-|49|-|4 +Brand#31|-|ECONOMY PLATED NICKEL|-|9|-|4 +Brand#31|-|ECONOMY PLATED STEEL|-|23|-|4 +Brand#31|-|ECONOMY PLATED TIN|-|36|-|4 +Brand#31|-|ECONOMY PLATED TIN|-|49|-|4 +Brand#31|-|ECONOMY POLISHED COPPER|-|3|-|4 +Brand#31|-|ECONOMY POLISHED COPPER|-|36|-|4 +Brand#31|-|ECONOMY POLISHED COPPER|-|49|-|4 +Brand#31|-|ECONOMY POLISHED NICKEL|-|3|-|4 +Brand#31|-|LARGE ANODIZED BRASS|-|19|-|4 +Brand#31|-|LARGE ANODIZED STEEL|-|45|-|4 +Brand#31|-|LARGE BRUSHED BRASS|-|36|-|4 +Brand#31|-|LARGE BRUSHED BRASS|-|49|-|4 +Brand#31|-|LARGE BRUSHED TIN|-|3|-|4 +Brand#31|-|LARGE BURNISHED BRASS|-|9|-|4 +Brand#31|-|LARGE PLATED COPPER|-|19|-|4 +Brand#31|-|LARGE PLATED NICKEL|-|14|-|4 +Brand#31|-|LARGE PLATED TIN|-|9|-|4 +Brand#31|-|LARGE PLATED TIN|-|14|-|4 +Brand#31|-|LARGE POLISHED BRASS|-|14|-|4 +Brand#31|-|LARGE POLISHED STEEL|-|14|-|4 +Brand#31|-|LARGE POLISHED STEEL|-|45|-|4 +Brand#31|-|LARGE POLISHED TIN|-|19|-|4 +Brand#31|-|MEDIUM ANODIZED BRASS|-|23|-|4 +Brand#31|-|MEDIUM ANODIZED BRASS|-|36|-|4 +Brand#31|-|MEDIUM ANODIZED COPPER|-|14|-|4 +Brand#31|-|MEDIUM ANODIZED COPPER|-|19|-|4 +Brand#31|-|MEDIUM ANODIZED COPPER|-|36|-|4 +Brand#31|-|MEDIUM ANODIZED STEEL|-|14|-|4 +Brand#31|-|MEDIUM ANODIZED STEEL|-|49|-|4 +Brand#31|-|MEDIUM ANODIZED TIN|-|19|-|4 +Brand#31|-|MEDIUM ANODIZED TIN|-|49|-|4 +Brand#31|-|MEDIUM BRUSHED BRASS|-|36|-|4 +Brand#31|-|MEDIUM BRUSHED STEEL|-|14|-|4 +Brand#31|-|MEDIUM BURNISHED BRASS|-|14|-|4 +Brand#31|-|MEDIUM BURNISHED COPPER|-|3|-|4 +Brand#31|-|MEDIUM BURNISHED NICKEL|-|9|-|4 +Brand#31|-|MEDIUM BURNISHED STEEL|-|9|-|4 +Brand#31|-|MEDIUM BURNISHED TIN|-|14|-|4 +Brand#31|-|MEDIUM BURNISHED TIN|-|23|-|4 +Brand#31|-|MEDIUM PLATED BRASS|-|3|-|4 +Brand#31|-|MEDIUM PLATED TIN|-|9|-|4 +Brand#31|-|MEDIUM PLATED TIN|-|36|-|4 +Brand#31|-|MEDIUM PLATED TIN|-|45|-|4 +Brand#31|-|PROMO ANODIZED BRASS|-|3|-|4 +Brand#31|-|PROMO ANODIZED NICKEL|-|9|-|4 +Brand#31|-|PROMO BRUSHED BRASS|-|3|-|4 +Brand#31|-|PROMO BRUSHED BRASS|-|23|-|4 +Brand#31|-|PROMO BRUSHED COPPER|-|23|-|4 +Brand#31|-|PROMO BRUSHED NICKEL|-|45|-|4 +Brand#31|-|PROMO BURNISHED COPPER|-|36|-|4 +Brand#31|-|PROMO BURNISHED STEEL|-|3|-|4 +Brand#31|-|PROMO BURNISHED TIN|-|3|-|4 +Brand#31|-|PROMO PLATED BRASS|-|19|-|4 +Brand#31|-|PROMO PLATED NICKEL|-|36|-|4 +Brand#31|-|PROMO POLISHED BRASS|-|49|-|4 +Brand#31|-|PROMO POLISHED COPPER|-|14|-|4 +Brand#31|-|PROMO POLISHED NICKEL|-|3|-|4 +Brand#31|-|PROMO POLISHED NICKEL|-|9|-|4 +Brand#31|-|PROMO POLISHED TIN|-|3|-|4 +Brand#31|-|PROMO POLISHED TIN|-|23|-|4 +Brand#31|-|SMALL ANODIZED COPPER|-|45|-|4 +Brand#31|-|SMALL ANODIZED STEEL|-|23|-|4 +Brand#31|-|SMALL ANODIZED TIN|-|3|-|4 +Brand#31|-|SMALL BRUSHED COPPER|-|36|-|4 +Brand#31|-|SMALL BRUSHED COPPER|-|49|-|4 +Brand#31|-|SMALL BRUSHED NICKEL|-|19|-|4 +Brand#31|-|SMALL BRUSHED NICKEL|-|23|-|4 +Brand#31|-|SMALL BURNISHED BRASS|-|45|-|4 +Brand#31|-|SMALL BURNISHED NICKEL|-|9|-|4 +Brand#31|-|SMALL BURNISHED NICKEL|-|36|-|4 +Brand#31|-|SMALL PLATED COPPER|-|36|-|4 +Brand#31|-|SMALL PLATED NICKEL|-|9|-|4 +Brand#31|-|SMALL PLATED NICKEL|-|36|-|4 +Brand#31|-|SMALL POLISHED BRASS|-|3|-|4 +Brand#31|-|SMALL POLISHED COPPER|-|45|-|4 +Brand#31|-|SMALL POLISHED NICKEL|-|45|-|4 +Brand#31|-|SMALL POLISHED TIN|-|23|-|4 +Brand#31|-|SMALL POLISHED TIN|-|49|-|4 +Brand#31|-|STANDARD BRUSHED STEEL|-|23|-|4 +Brand#31|-|STANDARD BRUSHED STEEL|-|49|-|4 +Brand#31|-|STANDARD BURNISHED BRASS|-|14|-|4 +Brand#31|-|STANDARD BURNISHED NICKEL|-|45|-|4 +Brand#31|-|STANDARD PLATED NICKEL|-|3|-|4 +Brand#31|-|STANDARD POLISHED BRASS|-|3|-|4 +Brand#31|-|STANDARD POLISHED BRASS|-|45|-|4 +Brand#31|-|STANDARD POLISHED STEEL|-|36|-|4 +Brand#32|-|ECONOMY ANODIZED BRASS|-|19|-|4 +Brand#32|-|ECONOMY ANODIZED COPPER|-|36|-|4 +Brand#32|-|ECONOMY ANODIZED STEEL|-|23|-|4 +Brand#32|-|ECONOMY ANODIZED STEEL|-|36|-|4 +Brand#32|-|ECONOMY ANODIZED STEEL|-|45|-|4 +Brand#32|-|ECONOMY ANODIZED TIN|-|19|-|4 +Brand#32|-|ECONOMY BRUSHED COPPER|-|45|-|4 +Brand#32|-|ECONOMY BRUSHED TIN|-|45|-|4 +Brand#32|-|ECONOMY BURNISHED BRASS|-|23|-|4 +Brand#32|-|ECONOMY BURNISHED COPPER|-|36|-|4 +Brand#32|-|ECONOMY BURNISHED COPPER|-|45|-|4 +Brand#32|-|ECONOMY BURNISHED STEEL|-|19|-|4 +Brand#32|-|ECONOMY PLATED BRASS|-|9|-|4 +Brand#32|-|ECONOMY PLATED COPPER|-|9|-|4 +Brand#32|-|ECONOMY PLATED NICKEL|-|23|-|4 +Brand#32|-|ECONOMY PLATED TIN|-|45|-|4 +Brand#32|-|ECONOMY POLISHED STEEL|-|3|-|4 +Brand#32|-|LARGE ANODIZED BRASS|-|23|-|4 +Brand#32|-|LARGE ANODIZED BRASS|-|36|-|4 +Brand#32|-|LARGE ANODIZED NICKEL|-|45|-|4 +Brand#32|-|LARGE ANODIZED STEEL|-|3|-|4 +Brand#32|-|LARGE ANODIZED STEEL|-|14|-|4 +Brand#32|-|LARGE BRUSHED STEEL|-|45|-|4 +Brand#32|-|LARGE BRUSHED TIN|-|45|-|4 +Brand#32|-|LARGE BURNISHED NICKEL|-|36|-|4 +Brand#32|-|LARGE BURNISHED TIN|-|19|-|4 +Brand#32|-|LARGE BURNISHED TIN|-|45|-|4 +Brand#32|-|LARGE PLATED BRASS|-|3|-|4 +Brand#32|-|LARGE PLATED NICKEL|-|49|-|4 +Brand#32|-|LARGE PLATED STEEL|-|19|-|4 +Brand#32|-|LARGE PLATED STEEL|-|36|-|4 +Brand#32|-|LARGE POLISHED BRASS|-|45|-|4 +Brand#32|-|LARGE POLISHED COPPER|-|9|-|4 +Brand#32|-|LARGE POLISHED COPPER|-|49|-|4 +Brand#32|-|LARGE POLISHED NICKEL|-|3|-|4 +Brand#32|-|MEDIUM ANODIZED BRASS|-|3|-|4 +Brand#32|-|MEDIUM ANODIZED BRASS|-|9|-|4 +Brand#32|-|MEDIUM ANODIZED TIN|-|23|-|4 +Brand#32|-|MEDIUM BRUSHED BRASS|-|23|-|4 +Brand#32|-|MEDIUM BRUSHED BRASS|-|49|-|4 +Brand#32|-|MEDIUM BRUSHED COPPER|-|9|-|4 +Brand#32|-|MEDIUM BRUSHED COPPER|-|19|-|4 +Brand#32|-|MEDIUM BRUSHED TIN|-|49|-|4 +Brand#32|-|MEDIUM BURNISHED BRASS|-|9|-|4 +Brand#32|-|MEDIUM BURNISHED BRASS|-|36|-|4 +Brand#32|-|MEDIUM BURNISHED BRASS|-|49|-|4 +Brand#32|-|MEDIUM BURNISHED COPPER|-|9|-|4 +Brand#32|-|MEDIUM BURNISHED COPPER|-|45|-|4 +Brand#32|-|MEDIUM BURNISHED NICKEL|-|49|-|4 +Brand#32|-|MEDIUM BURNISHED TIN|-|9|-|4 +Brand#32|-|MEDIUM BURNISHED TIN|-|45|-|4 +Brand#32|-|MEDIUM PLATED BRASS|-|3|-|4 +Brand#32|-|MEDIUM PLATED BRASS|-|49|-|4 +Brand#32|-|MEDIUM PLATED COPPER|-|3|-|4 +Brand#32|-|MEDIUM PLATED STEEL|-|9|-|4 +Brand#32|-|MEDIUM PLATED TIN|-|9|-|4 +Brand#32|-|PROMO ANODIZED BRASS|-|3|-|4 +Brand#32|-|PROMO ANODIZED COPPER|-|19|-|4 +Brand#32|-|PROMO ANODIZED NICKEL|-|23|-|4 +Brand#32|-|PROMO BRUSHED COPPER|-|23|-|4 +Brand#32|-|PROMO BRUSHED NICKEL|-|14|-|4 +Brand#32|-|PROMO BRUSHED NICKEL|-|36|-|4 +Brand#32|-|PROMO BRUSHED STEEL|-|14|-|4 +Brand#32|-|PROMO BRUSHED STEEL|-|23|-|4 +Brand#32|-|PROMO BRUSHED STEEL|-|49|-|4 +Brand#32|-|PROMO BURNISHED BRASS|-|45|-|4 +Brand#32|-|PROMO BURNISHED NICKEL|-|45|-|4 +Brand#32|-|PROMO BURNISHED TIN|-|14|-|4 +Brand#32|-|PROMO BURNISHED TIN|-|45|-|4 +Brand#32|-|PROMO PLATED TIN|-|19|-|4 +Brand#32|-|PROMO POLISHED NICKEL|-|36|-|4 +Brand#32|-|PROMO POLISHED TIN|-|3|-|4 +Brand#32|-|SMALL ANODIZED BRASS|-|3|-|4 +Brand#32|-|SMALL ANODIZED NICKEL|-|3|-|4 +Brand#32|-|SMALL ANODIZED NICKEL|-|14|-|4 +Brand#32|-|SMALL ANODIZED TIN|-|9|-|4 +Brand#32|-|SMALL BRUSHED BRASS|-|9|-|4 +Brand#32|-|SMALL BRUSHED BRASS|-|19|-|4 +Brand#32|-|SMALL BRUSHED COPPER|-|3|-|4 +Brand#32|-|SMALL BRUSHED COPPER|-|23|-|4 +Brand#32|-|SMALL BRUSHED NICKEL|-|9|-|4 +Brand#32|-|SMALL BRUSHED NICKEL|-|45|-|4 +Brand#32|-|SMALL BRUSHED STEEL|-|23|-|4 +Brand#32|-|SMALL BRUSHED TIN|-|9|-|4 +Brand#32|-|SMALL BURNISHED NICKEL|-|36|-|4 +Brand#32|-|SMALL BURNISHED STEEL|-|3|-|4 +Brand#32|-|SMALL BURNISHED TIN|-|23|-|4 +Brand#32|-|SMALL PLATED BRASS|-|49|-|4 +Brand#32|-|SMALL PLATED COPPER|-|36|-|4 +Brand#32|-|SMALL PLATED COPPER|-|45|-|4 +Brand#32|-|SMALL PLATED NICKEL|-|45|-|4 +Brand#32|-|SMALL PLATED STEEL|-|45|-|4 +Brand#32|-|SMALL PLATED TIN|-|23|-|4 +Brand#32|-|SMALL PLATED TIN|-|36|-|4 +Brand#32|-|SMALL PLATED TIN|-|45|-|4 +Brand#32|-|SMALL POLISHED NICKEL|-|36|-|4 +Brand#32|-|SMALL POLISHED STEEL|-|14|-|4 +Brand#32|-|SMALL POLISHED STEEL|-|23|-|4 +Brand#32|-|SMALL POLISHED STEEL|-|36|-|4 +Brand#32|-|SMALL POLISHED TIN|-|36|-|4 +Brand#32|-|SMALL POLISHED TIN|-|45|-|4 +Brand#32|-|STANDARD ANODIZED NICKEL|-|19|-|4 +Brand#32|-|STANDARD ANODIZED TIN|-|9|-|4 +Brand#32|-|STANDARD ANODIZED TIN|-|14|-|4 +Brand#32|-|STANDARD ANODIZED TIN|-|19|-|4 +Brand#32|-|STANDARD BRUSHED NICKEL|-|23|-|4 +Brand#32|-|STANDARD BURNISHED BRASS|-|36|-|4 +Brand#32|-|STANDARD BURNISHED BRASS|-|45|-|4 +Brand#32|-|STANDARD BURNISHED COPPER|-|3|-|4 +Brand#32|-|STANDARD BURNISHED COPPER|-|36|-|4 +Brand#32|-|STANDARD BURNISHED NICKEL|-|49|-|4 +Brand#32|-|STANDARD BURNISHED STEEL|-|49|-|4 +Brand#32|-|STANDARD BURNISHED TIN|-|23|-|4 +Brand#32|-|STANDARD PLATED BRASS|-|9|-|4 +Brand#32|-|STANDARD PLATED BRASS|-|45|-|4 +Brand#32|-|STANDARD PLATED STEEL|-|36|-|4 +Brand#32|-|STANDARD POLISHED BRASS|-|14|-|4 +Brand#32|-|STANDARD POLISHED COPPER|-|36|-|4 +Brand#32|-|STANDARD POLISHED STEEL|-|14|-|4 +Brand#33|-|ECONOMY ANODIZED BRASS|-|23|-|4 +Brand#33|-|ECONOMY ANODIZED COPPER|-|9|-|4 +Brand#33|-|ECONOMY ANODIZED NICKEL|-|3|-|4 +Brand#33|-|ECONOMY ANODIZED NICKEL|-|9|-|4 +Brand#33|-|ECONOMY ANODIZED NICKEL|-|23|-|4 +Brand#33|-|ECONOMY ANODIZED NICKEL|-|36|-|4 +Brand#33|-|ECONOMY BRUSHED BRASS|-|14|-|4 +Brand#33|-|ECONOMY BRUSHED COPPER|-|23|-|4 +Brand#33|-|ECONOMY BURNISHED BRASS|-|49|-|4 +Brand#33|-|ECONOMY BURNISHED COPPER|-|3|-|4 +Brand#33|-|ECONOMY BURNISHED COPPER|-|14|-|4 +Brand#33|-|ECONOMY BURNISHED STEEL|-|3|-|4 +Brand#33|-|ECONOMY BURNISHED TIN|-|36|-|4 +Brand#33|-|ECONOMY BURNISHED TIN|-|45|-|4 +Brand#33|-|ECONOMY PLATED COPPER|-|19|-|4 +Brand#33|-|ECONOMY PLATED COPPER|-|45|-|4 +Brand#33|-|ECONOMY PLATED NICKEL|-|14|-|4 +Brand#33|-|ECONOMY PLATED NICKEL|-|36|-|4 +Brand#33|-|ECONOMY PLATED STEEL|-|3|-|4 +Brand#33|-|ECONOMY PLATED STEEL|-|23|-|4 +Brand#33|-|ECONOMY PLATED STEEL|-|36|-|4 +Brand#33|-|ECONOMY POLISHED BRASS|-|14|-|4 +Brand#33|-|ECONOMY POLISHED NICKEL|-|19|-|4 +Brand#33|-|ECONOMY POLISHED TIN|-|9|-|4 +Brand#33|-|LARGE ANODIZED BRASS|-|36|-|4 +Brand#33|-|LARGE ANODIZED COPPER|-|19|-|4 +Brand#33|-|LARGE ANODIZED COPPER|-|45|-|4 +Brand#33|-|LARGE ANODIZED NICKEL|-|36|-|4 +Brand#33|-|LARGE ANODIZED NICKEL|-|45|-|4 +Brand#33|-|LARGE ANODIZED STEEL|-|3|-|4 +Brand#33|-|LARGE ANODIZED STEEL|-|45|-|4 +Brand#33|-|LARGE ANODIZED TIN|-|45|-|4 +Brand#33|-|LARGE BRUSHED BRASS|-|3|-|4 +Brand#33|-|LARGE BRUSHED BRASS|-|49|-|4 +Brand#33|-|LARGE BRUSHED STEEL|-|19|-|4 +Brand#33|-|LARGE BRUSHED TIN|-|36|-|4 +Brand#33|-|LARGE BURNISHED COPPER|-|45|-|4 +Brand#33|-|LARGE BURNISHED NICKEL|-|23|-|4 +Brand#33|-|LARGE BURNISHED STEEL|-|19|-|4 +Brand#33|-|LARGE PLATED BRASS|-|3|-|4 +Brand#33|-|LARGE PLATED COPPER|-|19|-|4 +Brand#33|-|LARGE PLATED STEEL|-|3|-|4 +Brand#33|-|LARGE PLATED STEEL|-|19|-|4 +Brand#33|-|LARGE PLATED TIN|-|45|-|4 +Brand#33|-|LARGE POLISHED BRASS|-|45|-|4 +Brand#33|-|LARGE POLISHED STEEL|-|14|-|4 +Brand#33|-|LARGE POLISHED STEEL|-|23|-|4 +Brand#33|-|LARGE POLISHED TIN|-|23|-|4 +Brand#33|-|MEDIUM ANODIZED BRASS|-|3|-|4 +Brand#33|-|MEDIUM ANODIZED COPPER|-|9|-|4 +Brand#33|-|MEDIUM ANODIZED COPPER|-|36|-|4 +Brand#33|-|MEDIUM ANODIZED COPPER|-|49|-|4 +Brand#33|-|MEDIUM ANODIZED NICKEL|-|3|-|4 +Brand#33|-|MEDIUM ANODIZED NICKEL|-|19|-|4 +Brand#33|-|MEDIUM BRUSHED BRASS|-|3|-|4 +Brand#33|-|MEDIUM BRUSHED STEEL|-|19|-|4 +Brand#33|-|MEDIUM BRUSHED TIN|-|14|-|4 +Brand#33|-|MEDIUM BURNISHED COPPER|-|14|-|4 +Brand#33|-|MEDIUM BURNISHED COPPER|-|49|-|4 +Brand#33|-|MEDIUM BURNISHED TIN|-|36|-|4 +Brand#33|-|MEDIUM PLATED BRASS|-|3|-|4 +Brand#33|-|MEDIUM PLATED STEEL|-|3|-|4 +Brand#33|-|MEDIUM PLATED STEEL|-|49|-|4 +Brand#33|-|PROMO ANODIZED BRASS|-|3|-|4 +Brand#33|-|PROMO BRUSHED BRASS|-|49|-|4 +Brand#33|-|PROMO BURNISHED COPPER|-|23|-|4 +Brand#33|-|PROMO BURNISHED NICKEL|-|14|-|4 +Brand#33|-|PROMO BURNISHED NICKEL|-|36|-|4 +Brand#33|-|PROMO BURNISHED TIN|-|19|-|4 +Brand#33|-|PROMO BURNISHED TIN|-|23|-|4 +Brand#33|-|PROMO PLATED COPPER|-|14|-|4 +Brand#33|-|PROMO PLATED STEEL|-|45|-|4 +Brand#33|-|PROMO PLATED STEEL|-|49|-|4 +Brand#33|-|PROMO PLATED TIN|-|49|-|4 +Brand#33|-|PROMO POLISHED COPPER|-|3|-|4 +Brand#33|-|PROMO POLISHED STEEL|-|3|-|4 +Brand#33|-|PROMO POLISHED STEEL|-|9|-|4 +Brand#33|-|PROMO POLISHED STEEL|-|23|-|4 +Brand#33|-|SMALL ANODIZED BRASS|-|19|-|4 +Brand#33|-|SMALL ANODIZED COPPER|-|23|-|4 +Brand#33|-|SMALL ANODIZED COPPER|-|49|-|4 +Brand#33|-|SMALL ANODIZED STEEL|-|9|-|4 +Brand#33|-|SMALL BRUSHED BRASS|-|3|-|4 +Brand#33|-|SMALL BRUSHED COPPER|-|3|-|4 +Brand#33|-|SMALL BRUSHED NICKEL|-|45|-|4 +Brand#33|-|SMALL BRUSHED STEEL|-|3|-|4 +Brand#33|-|SMALL BRUSHED TIN|-|9|-|4 +Brand#33|-|SMALL BURNISHED BRASS|-|19|-|4 +Brand#33|-|SMALL BURNISHED NICKEL|-|3|-|4 +Brand#33|-|SMALL PLATED BRASS|-|3|-|4 +Brand#33|-|SMALL PLATED STEEL|-|14|-|4 +Brand#33|-|SMALL PLATED STEEL|-|45|-|4 +Brand#33|-|SMALL PLATED TIN|-|23|-|4 +Brand#33|-|SMALL PLATED TIN|-|36|-|4 +Brand#33|-|SMALL POLISHED NICKEL|-|23|-|4 +Brand#33|-|SMALL POLISHED TIN|-|19|-|4 +Brand#33|-|SMALL POLISHED TIN|-|23|-|4 +Brand#33|-|SMALL POLISHED TIN|-|45|-|4 +Brand#33|-|STANDARD ANODIZED COPPER|-|49|-|4 +Brand#33|-|STANDARD ANODIZED STEEL|-|14|-|4 +Brand#33|-|STANDARD ANODIZED STEEL|-|45|-|4 +Brand#33|-|STANDARD ANODIZED STEEL|-|49|-|4 +Brand#33|-|STANDARD ANODIZED TIN|-|45|-|4 +Brand#33|-|STANDARD BRUSHED BRASS|-|9|-|4 +Brand#33|-|STANDARD BRUSHED NICKEL|-|45|-|4 +Brand#33|-|STANDARD BRUSHED STEEL|-|9|-|4 +Brand#33|-|STANDARD BRUSHED TIN|-|36|-|4 +Brand#33|-|STANDARD BURNISHED BRASS|-|9|-|4 +Brand#33|-|STANDARD BURNISHED BRASS|-|23|-|4 +Brand#33|-|STANDARD BURNISHED NICKEL|-|49|-|4 +Brand#33|-|STANDARD PLATED BRASS|-|49|-|4 +Brand#33|-|STANDARD PLATED COPPER|-|3|-|4 +Brand#33|-|STANDARD PLATED COPPER|-|14|-|4 +Brand#33|-|STANDARD PLATED NICKEL|-|36|-|4 +Brand#33|-|STANDARD PLATED STEEL|-|3|-|4 +Brand#33|-|STANDARD PLATED STEEL|-|36|-|4 +Brand#33|-|STANDARD PLATED TIN|-|14|-|4 +Brand#33|-|STANDARD POLISHED BRASS|-|9|-|4 +Brand#33|-|STANDARD POLISHED BRASS|-|19|-|4 +Brand#33|-|STANDARD POLISHED STEEL|-|3|-|4 +Brand#33|-|STANDARD POLISHED STEEL|-|9|-|4 +Brand#33|-|STANDARD POLISHED STEEL|-|14|-|4 +Brand#34|-|ECONOMY ANODIZED BRASS|-|9|-|4 +Brand#34|-|ECONOMY ANODIZED COPPER|-|3|-|4 +Brand#34|-|ECONOMY ANODIZED COPPER|-|14|-|4 +Brand#34|-|ECONOMY ANODIZED COPPER|-|19|-|4 +Brand#34|-|ECONOMY ANODIZED STEEL|-|9|-|4 +Brand#34|-|ECONOMY ANODIZED TIN|-|49|-|4 +Brand#34|-|ECONOMY BRUSHED BRASS|-|14|-|4 +Brand#34|-|ECONOMY BRUSHED NICKEL|-|49|-|4 +Brand#34|-|ECONOMY BURNISHED COPPER|-|9|-|4 +Brand#34|-|ECONOMY BURNISHED STEEL|-|19|-|4 +Brand#34|-|ECONOMY BURNISHED TIN|-|3|-|4 +Brand#34|-|ECONOMY BURNISHED TIN|-|23|-|4 +Brand#34|-|ECONOMY PLATED BRASS|-|9|-|4 +Brand#34|-|ECONOMY PLATED BRASS|-|14|-|4 +Brand#34|-|ECONOMY PLATED COPPER|-|3|-|4 +Brand#34|-|ECONOMY PLATED NICKEL|-|45|-|4 +Brand#34|-|ECONOMY PLATED TIN|-|14|-|4 +Brand#34|-|ECONOMY PLATED TIN|-|45|-|4 +Brand#34|-|ECONOMY POLISHED BRASS|-|45|-|4 +Brand#34|-|LARGE ANODIZED BRASS|-|14|-|4 +Brand#34|-|LARGE ANODIZED BRASS|-|23|-|4 +Brand#34|-|LARGE ANODIZED BRASS|-|36|-|4 +Brand#34|-|LARGE ANODIZED NICKEL|-|3|-|4 +Brand#34|-|LARGE ANODIZED TIN|-|49|-|4 +Brand#34|-|LARGE BRUSHED BRASS|-|49|-|4 +Brand#34|-|LARGE BRUSHED COPPER|-|23|-|4 +Brand#34|-|LARGE BRUSHED NICKEL|-|23|-|4 +Brand#34|-|LARGE BRUSHED STEEL|-|14|-|4 +Brand#34|-|LARGE BRUSHED STEEL|-|19|-|4 +Brand#34|-|LARGE BRUSHED TIN|-|9|-|4 +Brand#34|-|LARGE BURNISHED BRASS|-|23|-|4 +Brand#34|-|LARGE BURNISHED COPPER|-|3|-|4 +Brand#34|-|LARGE BURNISHED COPPER|-|36|-|4 +Brand#34|-|LARGE BURNISHED NICKEL|-|19|-|4 +Brand#34|-|LARGE PLATED BRASS|-|23|-|4 +Brand#34|-|LARGE PLATED BRASS|-|36|-|4 +Brand#34|-|LARGE PLATED BRASS|-|45|-|4 +Brand#34|-|LARGE PLATED COPPER|-|23|-|4 +Brand#34|-|LARGE PLATED COPPER|-|49|-|4 +Brand#34|-|LARGE PLATED STEEL|-|49|-|4 +Brand#34|-|LARGE POLISHED NICKEL|-|49|-|4 +Brand#34|-|MEDIUM ANODIZED COPPER|-|36|-|4 +Brand#34|-|MEDIUM ANODIZED TIN|-|3|-|4 +Brand#34|-|MEDIUM BRUSHED BRASS|-|49|-|4 +Brand#34|-|MEDIUM BRUSHED COPPER|-|9|-|4 +Brand#34|-|MEDIUM BRUSHED NICKEL|-|9|-|4 +Brand#34|-|MEDIUM BRUSHED NICKEL|-|23|-|4 +Brand#34|-|MEDIUM BRUSHED TIN|-|3|-|4 +Brand#34|-|MEDIUM BRUSHED TIN|-|14|-|4 +Brand#34|-|MEDIUM BURNISHED STEEL|-|45|-|4 +Brand#34|-|MEDIUM BURNISHED STEEL|-|49|-|4 +Brand#34|-|MEDIUM PLATED COPPER|-|36|-|4 +Brand#34|-|MEDIUM PLATED TIN|-|3|-|4 +Brand#34|-|MEDIUM PLATED TIN|-|14|-|4 +Brand#34|-|PROMO ANODIZED COPPER|-|45|-|4 +Brand#34|-|PROMO ANODIZED NICKEL|-|14|-|4 +Brand#34|-|PROMO ANODIZED STEEL|-|49|-|4 +Brand#34|-|PROMO ANODIZED TIN|-|14|-|4 +Brand#34|-|PROMO BRUSHED BRASS|-|9|-|4 +Brand#34|-|PROMO BRUSHED BRASS|-|23|-|4 +Brand#34|-|PROMO BRUSHED COPPER|-|36|-|4 +Brand#34|-|PROMO BRUSHED STEEL|-|36|-|4 +Brand#34|-|PROMO BURNISHED BRASS|-|49|-|4 +Brand#34|-|PROMO BURNISHED STEEL|-|3|-|4 +Brand#34|-|PROMO PLATED BRASS|-|9|-|4 +Brand#34|-|PROMO PLATED STEEL|-|49|-|4 +Brand#34|-|PROMO POLISHED BRASS|-|23|-|4 +Brand#34|-|PROMO POLISHED NICKEL|-|3|-|4 +Brand#34|-|PROMO POLISHED NICKEL|-|36|-|4 +Brand#34|-|SMALL ANODIZED BRASS|-|36|-|4 +Brand#34|-|SMALL ANODIZED COPPER|-|45|-|4 +Brand#34|-|SMALL ANODIZED NICKEL|-|14|-|4 +Brand#34|-|SMALL ANODIZED NICKEL|-|36|-|4 +Brand#34|-|SMALL ANODIZED STEEL|-|3|-|4 +Brand#34|-|SMALL ANODIZED STEEL|-|19|-|4 +Brand#34|-|SMALL ANODIZED STEEL|-|23|-|4 +Brand#34|-|SMALL ANODIZED STEEL|-|36|-|4 +Brand#34|-|SMALL BRUSHED BRASS|-|14|-|4 +Brand#34|-|SMALL BRUSHED BRASS|-|36|-|4 +Brand#34|-|SMALL BRUSHED NICKEL|-|14|-|4 +Brand#34|-|SMALL BRUSHED NICKEL|-|36|-|4 +Brand#34|-|SMALL BRUSHED NICKEL|-|45|-|4 +Brand#34|-|SMALL BRUSHED TIN|-|9|-|4 +Brand#34|-|SMALL BRUSHED TIN|-|23|-|4 +Brand#34|-|SMALL BRUSHED TIN|-|36|-|4 +Brand#34|-|SMALL BURNISHED COPPER|-|9|-|4 +Brand#34|-|SMALL BURNISHED TIN|-|36|-|4 +Brand#34|-|SMALL PLATED BRASS|-|14|-|4 +Brand#34|-|SMALL PLATED COPPER|-|36|-|4 +Brand#34|-|SMALL PLATED TIN|-|45|-|4 +Brand#34|-|SMALL POLISHED NICKEL|-|14|-|4 +Brand#34|-|SMALL POLISHED NICKEL|-|45|-|4 +Brand#34|-|SMALL POLISHED TIN|-|9|-|4 +Brand#34|-|SMALL POLISHED TIN|-|14|-|4 +Brand#34|-|SMALL POLISHED TIN|-|19|-|4 +Brand#34|-|STANDARD ANODIZED BRASS|-|23|-|4 +Brand#34|-|STANDARD ANODIZED BRASS|-|36|-|4 +Brand#34|-|STANDARD ANODIZED COPPER|-|45|-|4 +Brand#34|-|STANDARD ANODIZED NICKEL|-|36|-|4 +Brand#34|-|STANDARD ANODIZED STEEL|-|9|-|4 +Brand#34|-|STANDARD ANODIZED STEEL|-|49|-|4 +Brand#34|-|STANDARD ANODIZED TIN|-|9|-|4 +Brand#34|-|STANDARD BRUSHED BRASS|-|19|-|4 +Brand#34|-|STANDARD BRUSHED BRASS|-|23|-|4 +Brand#34|-|STANDARD BRUSHED NICKEL|-|23|-|4 +Brand#34|-|STANDARD BRUSHED STEEL|-|3|-|4 +Brand#34|-|STANDARD BRUSHED TIN|-|19|-|4 +Brand#34|-|STANDARD BURNISHED COPPER|-|45|-|4 +Brand#34|-|STANDARD BURNISHED NICKEL|-|19|-|4 +Brand#34|-|STANDARD BURNISHED NICKEL|-|45|-|4 +Brand#34|-|STANDARD BURNISHED STEEL|-|36|-|4 +Brand#34|-|STANDARD BURNISHED TIN|-|45|-|4 +Brand#34|-|STANDARD PLATED BRASS|-|9|-|4 +Brand#34|-|STANDARD PLATED COPPER|-|9|-|4 +Brand#34|-|STANDARD PLATED NICKEL|-|36|-|4 +Brand#35|-|ECONOMY ANODIZED COPPER|-|3|-|4 +Brand#35|-|ECONOMY ANODIZED STEEL|-|45|-|4 +Brand#35|-|ECONOMY BRUSHED BRASS|-|3|-|4 +Brand#35|-|ECONOMY BRUSHED NICKEL|-|49|-|4 +Brand#35|-|ECONOMY BRUSHED STEEL|-|23|-|4 +Brand#35|-|ECONOMY BRUSHED STEEL|-|45|-|4 +Brand#35|-|ECONOMY BRUSHED TIN|-|14|-|4 +Brand#35|-|ECONOMY BRUSHED TIN|-|23|-|4 +Brand#35|-|ECONOMY BURNISHED NICKEL|-|19|-|4 +Brand#35|-|ECONOMY BURNISHED STEEL|-|36|-|4 +Brand#35|-|ECONOMY BURNISHED TIN|-|9|-|4 +Brand#35|-|ECONOMY BURNISHED TIN|-|19|-|4 +Brand#35|-|ECONOMY BURNISHED TIN|-|49|-|4 +Brand#35|-|ECONOMY POLISHED COPPER|-|9|-|4 +Brand#35|-|ECONOMY POLISHED TIN|-|19|-|4 +Brand#35|-|LARGE ANODIZED BRASS|-|3|-|4 +Brand#35|-|LARGE ANODIZED BRASS|-|23|-|4 +Brand#35|-|LARGE ANODIZED COPPER|-|49|-|4 +Brand#35|-|LARGE ANODIZED STEEL|-|36|-|4 +Brand#35|-|LARGE ANODIZED TIN|-|9|-|4 +Brand#35|-|LARGE BRUSHED COPPER|-|9|-|4 +Brand#35|-|LARGE BRUSHED COPPER|-|23|-|4 +Brand#35|-|LARGE BRUSHED STEEL|-|3|-|4 +Brand#35|-|LARGE BRUSHED STEEL|-|9|-|4 +Brand#35|-|LARGE BURNISHED BRASS|-|36|-|4 +Brand#35|-|LARGE BURNISHED BRASS|-|45|-|4 +Brand#35|-|LARGE BURNISHED COPPER|-|23|-|4 +Brand#35|-|LARGE BURNISHED NICKEL|-|23|-|4 +Brand#35|-|LARGE PLATED BRASS|-|9|-|4 +Brand#35|-|LARGE PLATED COPPER|-|36|-|4 +Brand#35|-|LARGE POLISHED BRASS|-|49|-|4 +Brand#35|-|LARGE POLISHED STEEL|-|9|-|4 +Brand#35|-|LARGE POLISHED TIN|-|14|-|4 +Brand#35|-|MEDIUM ANODIZED BRASS|-|9|-|4 +Brand#35|-|MEDIUM ANODIZED BRASS|-|36|-|4 +Brand#35|-|MEDIUM ANODIZED COPPER|-|9|-|4 +Brand#35|-|MEDIUM BRUSHED BRASS|-|14|-|4 +Brand#35|-|MEDIUM BRUSHED COPPER|-|9|-|4 +Brand#35|-|MEDIUM BRUSHED COPPER|-|36|-|4 +Brand#35|-|MEDIUM BURNISHED BRASS|-|49|-|4 +Brand#35|-|MEDIUM BURNISHED NICKEL|-|45|-|4 +Brand#35|-|MEDIUM BURNISHED TIN|-|36|-|4 +Brand#35|-|MEDIUM PLATED BRASS|-|23|-|4 +Brand#35|-|MEDIUM PLATED COPPER|-|9|-|4 +Brand#35|-|MEDIUM PLATED NICKEL|-|45|-|4 +Brand#35|-|MEDIUM PLATED NICKEL|-|49|-|4 +Brand#35|-|MEDIUM PLATED STEEL|-|49|-|4 +Brand#35|-|PROMO ANODIZED COPPER|-|49|-|4 +Brand#35|-|PROMO ANODIZED NICKEL|-|19|-|4 +Brand#35|-|PROMO ANODIZED NICKEL|-|23|-|4 +Brand#35|-|PROMO ANODIZED TIN|-|3|-|4 +Brand#35|-|PROMO ANODIZED TIN|-|14|-|4 +Brand#35|-|PROMO BRUSHED BRASS|-|49|-|4 +Brand#35|-|PROMO BRUSHED NICKEL|-|14|-|4 +Brand#35|-|PROMO BRUSHED NICKEL|-|19|-|4 +Brand#35|-|PROMO BURNISHED BRASS|-|3|-|4 +Brand#35|-|PROMO BURNISHED STEEL|-|3|-|4 +Brand#35|-|PROMO PLATED BRASS|-|19|-|4 +Brand#35|-|PROMO PLATED COPPER|-|14|-|4 +Brand#35|-|PROMO PLATED STEEL|-|23|-|4 +Brand#35|-|PROMO PLATED STEEL|-|36|-|4 +Brand#35|-|PROMO PLATED TIN|-|19|-|4 +Brand#35|-|PROMO POLISHED BRASS|-|9|-|4 +Brand#35|-|PROMO POLISHED BRASS|-|36|-|4 +Brand#35|-|PROMO POLISHED NICKEL|-|36|-|4 +Brand#35|-|PROMO POLISHED STEEL|-|23|-|4 +Brand#35|-|PROMO POLISHED TIN|-|36|-|4 +Brand#35|-|PROMO POLISHED TIN|-|45|-|4 +Brand#35|-|SMALL ANODIZED COPPER|-|9|-|4 +Brand#35|-|SMALL ANODIZED STEEL|-|19|-|4 +Brand#35|-|SMALL ANODIZED TIN|-|19|-|4 +Brand#35|-|SMALL BRUSHED BRASS|-|36|-|4 +Brand#35|-|SMALL BRUSHED STEEL|-|49|-|4 +Brand#35|-|SMALL BRUSHED TIN|-|3|-|4 +Brand#35|-|SMALL BRUSHED TIN|-|19|-|4 +Brand#35|-|SMALL BRUSHED TIN|-|23|-|4 +Brand#35|-|SMALL BURNISHED BRASS|-|23|-|4 +Brand#35|-|SMALL BURNISHED STEEL|-|36|-|4 +Brand#35|-|SMALL BURNISHED TIN|-|3|-|4 +Brand#35|-|SMALL BURNISHED TIN|-|36|-|4 +Brand#35|-|SMALL BURNISHED TIN|-|49|-|4 +Brand#35|-|SMALL PLATED BRASS|-|23|-|4 +Brand#35|-|SMALL PLATED STEEL|-|14|-|4 +Brand#35|-|SMALL POLISHED BRASS|-|36|-|4 +Brand#35|-|SMALL POLISHED STEEL|-|3|-|4 +Brand#35|-|SMALL POLISHED STEEL|-|49|-|4 +Brand#35|-|SMALL POLISHED TIN|-|23|-|4 +Brand#35|-|SMALL POLISHED TIN|-|45|-|4 +Brand#35|-|STANDARD ANODIZED NICKEL|-|14|-|4 +Brand#35|-|STANDARD ANODIZED STEEL|-|23|-|4 +Brand#35|-|STANDARD ANODIZED STEEL|-|45|-|4 +Brand#35|-|STANDARD ANODIZED TIN|-|9|-|4 +Brand#35|-|STANDARD ANODIZED TIN|-|19|-|4 +Brand#35|-|STANDARD BRUSHED BRASS|-|3|-|4 +Brand#35|-|STANDARD BRUSHED BRASS|-|23|-|4 +Brand#35|-|STANDARD BRUSHED BRASS|-|36|-|4 +Brand#35|-|STANDARD BRUSHED COPPER|-|36|-|4 +Brand#35|-|STANDARD BRUSHED NICKEL|-|36|-|4 +Brand#35|-|STANDARD BRUSHED NICKEL|-|49|-|4 +Brand#35|-|STANDARD BRUSHED TIN|-|9|-|4 +Brand#35|-|STANDARD BURNISHED BRASS|-|9|-|4 +Brand#35|-|STANDARD BURNISHED BRASS|-|19|-|4 +Brand#35|-|STANDARD BURNISHED BRASS|-|23|-|4 +Brand#35|-|STANDARD BURNISHED COPPER|-|36|-|4 +Brand#35|-|STANDARD BURNISHED STEEL|-|14|-|4 +Brand#35|-|STANDARD PLATED COPPER|-|19|-|4 +Brand#35|-|STANDARD PLATED NICKEL|-|23|-|4 +Brand#35|-|STANDARD PLATED STEEL|-|14|-|4 +Brand#35|-|STANDARD PLATED STEEL|-|23|-|4 +Brand#35|-|STANDARD PLATED TIN|-|49|-|4 +Brand#35|-|STANDARD POLISHED NICKEL|-|23|-|4 +Brand#35|-|STANDARD POLISHED TIN|-|23|-|4 +Brand#35|-|STANDARD POLISHED TIN|-|45|-|4 +Brand#41|-|ECONOMY ANODIZED STEEL|-|49|-|4 +Brand#41|-|ECONOMY BRUSHED BRASS|-|3|-|4 +Brand#41|-|ECONOMY BRUSHED COPPER|-|36|-|4 +Brand#41|-|ECONOMY BRUSHED NICKEL|-|23|-|4 +Brand#41|-|ECONOMY BRUSHED STEEL|-|36|-|4 +Brand#41|-|ECONOMY BRUSHED STEEL|-|45|-|4 +Brand#41|-|ECONOMY BRUSHED TIN|-|14|-|4 +Brand#41|-|ECONOMY PLATED COPPER|-|3|-|4 +Brand#41|-|ECONOMY PLATED STEEL|-|3|-|4 +Brand#41|-|ECONOMY PLATED TIN|-|23|-|4 +Brand#41|-|ECONOMY POLISHED COPPER|-|19|-|4 +Brand#41|-|ECONOMY POLISHED NICKEL|-|9|-|4 +Brand#41|-|ECONOMY POLISHED NICKEL|-|14|-|4 +Brand#41|-|ECONOMY POLISHED NICKEL|-|23|-|4 +Brand#41|-|ECONOMY POLISHED NICKEL|-|49|-|4 +Brand#41|-|ECONOMY POLISHED STEEL|-|9|-|4 +Brand#41|-|ECONOMY POLISHED STEEL|-|19|-|4 +Brand#41|-|ECONOMY POLISHED STEEL|-|45|-|4 +Brand#41|-|ECONOMY POLISHED TIN|-|19|-|4 +Brand#41|-|LARGE ANODIZED BRASS|-|14|-|4 +Brand#41|-|LARGE ANODIZED BRASS|-|23|-|4 +Brand#41|-|LARGE ANODIZED COPPER|-|49|-|4 +Brand#41|-|LARGE ANODIZED STEEL|-|3|-|4 +Brand#41|-|LARGE ANODIZED STEEL|-|23|-|4 +Brand#41|-|LARGE BRUSHED COPPER|-|23|-|4 +Brand#41|-|LARGE BRUSHED COPPER|-|49|-|4 +Brand#41|-|LARGE BRUSHED STEEL|-|19|-|4 +Brand#41|-|LARGE BURNISHED BRASS|-|45|-|4 +Brand#41|-|LARGE BURNISHED COPPER|-|3|-|4 +Brand#41|-|LARGE BURNISHED NICKEL|-|23|-|4 +Brand#41|-|LARGE BURNISHED TIN|-|9|-|4 +Brand#41|-|LARGE PLATED NICKEL|-|3|-|4 +Brand#41|-|LARGE PLATED NICKEL|-|23|-|4 +Brand#41|-|LARGE PLATED STEEL|-|9|-|4 +Brand#41|-|LARGE PLATED STEEL|-|36|-|4 +Brand#41|-|LARGE PLATED TIN|-|9|-|4 +Brand#41|-|LARGE POLISHED BRASS|-|36|-|4 +Brand#41|-|LARGE POLISHED COPPER|-|19|-|4 +Brand#41|-|LARGE POLISHED COPPER|-|49|-|4 +Brand#41|-|LARGE POLISHED NICKEL|-|36|-|4 +Brand#41|-|LARGE POLISHED STEEL|-|14|-|4 +Brand#41|-|MEDIUM ANODIZED BRASS|-|9|-|4 +Brand#41|-|MEDIUM ANODIZED COPPER|-|14|-|4 +Brand#41|-|MEDIUM ANODIZED NICKEL|-|3|-|4 +Brand#41|-|MEDIUM ANODIZED NICKEL|-|9|-|4 +Brand#41|-|MEDIUM ANODIZED STEEL|-|14|-|4 +Brand#41|-|MEDIUM BRUSHED COPPER|-|3|-|4 +Brand#41|-|MEDIUM BRUSHED TIN|-|9|-|4 +Brand#41|-|MEDIUM BURNISHED COPPER|-|23|-|4 +Brand#41|-|MEDIUM BURNISHED STEEL|-|9|-|4 +Brand#41|-|MEDIUM BURNISHED STEEL|-|45|-|4 +Brand#41|-|MEDIUM BURNISHED TIN|-|3|-|4 +Brand#41|-|MEDIUM PLATED BRASS|-|19|-|4 +Brand#41|-|MEDIUM PLATED BRASS|-|45|-|4 +Brand#41|-|MEDIUM PLATED COPPER|-|19|-|4 +Brand#41|-|MEDIUM PLATED STEEL|-|19|-|4 +Brand#41|-|MEDIUM PLATED STEEL|-|23|-|4 +Brand#41|-|PROMO ANODIZED BRASS|-|19|-|4 +Brand#41|-|PROMO ANODIZED COPPER|-|9|-|4 +Brand#41|-|PROMO ANODIZED NICKEL|-|9|-|4 +Brand#41|-|PROMO BRUSHED BRASS|-|14|-|4 +Brand#41|-|PROMO BRUSHED COPPER|-|36|-|4 +Brand#41|-|PROMO BRUSHED NICKEL|-|14|-|4 +Brand#41|-|PROMO BURNISHED BRASS|-|49|-|4 +Brand#41|-|PROMO BURNISHED NICKEL|-|36|-|4 +Brand#41|-|PROMO BURNISHED TIN|-|3|-|4 +Brand#41|-|PROMO PLATED NICKEL|-|14|-|4 +Brand#41|-|PROMO PLATED NICKEL|-|45|-|4 +Brand#41|-|PROMO PLATED STEEL|-|3|-|4 +Brand#41|-|PROMO PLATED TIN|-|3|-|4 +Brand#41|-|PROMO POLISHED COPPER|-|23|-|4 +Brand#41|-|SMALL ANODIZED BRASS|-|3|-|4 +Brand#41|-|SMALL ANODIZED BRASS|-|14|-|4 +Brand#41|-|SMALL ANODIZED STEEL|-|45|-|4 +Brand#41|-|SMALL ANODIZED TIN|-|9|-|4 +Brand#41|-|SMALL BRUSHED TIN|-|19|-|4 +Brand#41|-|SMALL BURNISHED COPPER|-|9|-|4 +Brand#41|-|SMALL BURNISHED NICKEL|-|3|-|4 +Brand#41|-|SMALL BURNISHED TIN|-|45|-|4 +Brand#41|-|SMALL PLATED COPPER|-|14|-|4 +Brand#41|-|SMALL PLATED COPPER|-|36|-|4 +Brand#41|-|SMALL PLATED COPPER|-|49|-|4 +Brand#41|-|SMALL PLATED TIN|-|19|-|4 +Brand#41|-|SMALL POLISHED COPPER|-|14|-|4 +Brand#41|-|SMALL POLISHED COPPER|-|19|-|4 +Brand#41|-|SMALL POLISHED COPPER|-|36|-|4 +Brand#41|-|SMALL POLISHED TIN|-|45|-|4 +Brand#41|-|STANDARD ANODIZED COPPER|-|19|-|4 +Brand#41|-|STANDARD ANODIZED NICKEL|-|9|-|4 +Brand#41|-|STANDARD ANODIZED STEEL|-|49|-|4 +Brand#41|-|STANDARD ANODIZED TIN|-|9|-|4 +Brand#41|-|STANDARD ANODIZED TIN|-|36|-|4 +Brand#41|-|STANDARD ANODIZED TIN|-|49|-|4 +Brand#41|-|STANDARD BRUSHED BRASS|-|19|-|4 +Brand#41|-|STANDARD BRUSHED NICKEL|-|3|-|4 +Brand#41|-|STANDARD BRUSHED NICKEL|-|9|-|4 +Brand#41|-|STANDARD BRUSHED STEEL|-|45|-|4 +Brand#41|-|STANDARD BRUSHED TIN|-|45|-|4 +Brand#41|-|STANDARD BURNISHED BRASS|-|23|-|4 +Brand#41|-|STANDARD BURNISHED BRASS|-|36|-|4 +Brand#41|-|STANDARD BURNISHED COPPER|-|49|-|4 +Brand#41|-|STANDARD BURNISHED STEEL|-|45|-|4 +Brand#41|-|STANDARD PLATED BRASS|-|45|-|4 +Brand#41|-|STANDARD PLATED NICKEL|-|14|-|4 +Brand#41|-|STANDARD PLATED STEEL|-|45|-|4 +Brand#41|-|STANDARD PLATED TIN|-|49|-|4 +Brand#41|-|STANDARD POLISHED STEEL|-|9|-|4 +Brand#41|-|STANDARD POLISHED STEEL|-|19|-|4 +Brand#41|-|STANDARD POLISHED TIN|-|45|-|4 +Brand#42|-|ECONOMY ANODIZED NICKEL|-|19|-|4 +Brand#42|-|ECONOMY BRUSHED BRASS|-|14|-|4 +Brand#42|-|ECONOMY BRUSHED COPPER|-|3|-|4 +Brand#42|-|ECONOMY BRUSHED COPPER|-|14|-|4 +Brand#42|-|ECONOMY BRUSHED NICKEL|-|14|-|4 +Brand#42|-|ECONOMY BRUSHED STEEL|-|14|-|4 +Brand#42|-|ECONOMY BRUSHED TIN|-|19|-|4 +Brand#42|-|ECONOMY BRUSHED TIN|-|49|-|4 +Brand#42|-|ECONOMY BURNISHED BRASS|-|19|-|4 +Brand#42|-|ECONOMY BURNISHED COPPER|-|23|-|4 +Brand#42|-|ECONOMY BURNISHED NICKEL|-|14|-|4 +Brand#42|-|ECONOMY BURNISHED TIN|-|14|-|4 +Brand#42|-|ECONOMY PLATED COPPER|-|23|-|4 +Brand#42|-|ECONOMY POLISHED BRASS|-|3|-|4 +Brand#42|-|ECONOMY POLISHED COPPER|-|9|-|4 +Brand#42|-|ECONOMY POLISHED STEEL|-|9|-|4 +Brand#42|-|ECONOMY POLISHED STEEL|-|36|-|4 +Brand#42|-|ECONOMY POLISHED TIN|-|14|-|4 +Brand#42|-|LARGE ANODIZED BRASS|-|49|-|4 +Brand#42|-|LARGE ANODIZED COPPER|-|14|-|4 +Brand#42|-|LARGE ANODIZED COPPER|-|49|-|4 +Brand#42|-|LARGE ANODIZED NICKEL|-|45|-|4 +Brand#42|-|LARGE ANODIZED NICKEL|-|49|-|4 +Brand#42|-|LARGE ANODIZED TIN|-|45|-|4 +Brand#42|-|LARGE BRUSHED BRASS|-|49|-|4 +Brand#42|-|LARGE BURNISHED BRASS|-|45|-|4 +Brand#42|-|LARGE BURNISHED BRASS|-|49|-|4 +Brand#42|-|LARGE BURNISHED COPPER|-|9|-|4 +Brand#42|-|LARGE BURNISHED TIN|-|9|-|4 +Brand#42|-|LARGE PLATED BRASS|-|45|-|4 +Brand#42|-|LARGE PLATED COPPER|-|9|-|4 +Brand#42|-|LARGE PLATED NICKEL|-|36|-|4 +Brand#42|-|LARGE PLATED TIN|-|23|-|4 +Brand#42|-|LARGE POLISHED BRASS|-|9|-|4 +Brand#42|-|LARGE POLISHED NICKEL|-|3|-|4 +Brand#42|-|LARGE POLISHED NICKEL|-|23|-|4 +Brand#42|-|LARGE POLISHED STEEL|-|9|-|4 +Brand#42|-|MEDIUM ANODIZED BRASS|-|23|-|4 +Brand#42|-|MEDIUM ANODIZED COPPER|-|19|-|4 +Brand#42|-|MEDIUM ANODIZED NICKEL|-|14|-|4 +Brand#42|-|MEDIUM ANODIZED NICKEL|-|19|-|4 +Brand#42|-|MEDIUM ANODIZED NICKEL|-|23|-|4 +Brand#42|-|MEDIUM ANODIZED STEEL|-|9|-|4 +Brand#42|-|MEDIUM ANODIZED STEEL|-|14|-|4 +Brand#42|-|MEDIUM ANODIZED STEEL|-|23|-|4 +Brand#42|-|MEDIUM ANODIZED TIN|-|14|-|4 +Brand#42|-|MEDIUM ANODIZED TIN|-|19|-|4 +Brand#42|-|MEDIUM BRUSHED COPPER|-|45|-|4 +Brand#42|-|MEDIUM BRUSHED COPPER|-|49|-|4 +Brand#42|-|MEDIUM BRUSHED STEEL|-|36|-|4 +Brand#42|-|MEDIUM BURNISHED COPPER|-|49|-|4 +Brand#42|-|MEDIUM BURNISHED TIN|-|3|-|4 +Brand#42|-|MEDIUM BURNISHED TIN|-|49|-|4 +Brand#42|-|MEDIUM PLATED NICKEL|-|45|-|4 +Brand#42|-|MEDIUM PLATED STEEL|-|3|-|4 +Brand#42|-|MEDIUM PLATED STEEL|-|23|-|4 +Brand#42|-|MEDIUM PLATED STEEL|-|45|-|4 +Brand#42|-|PROMO ANODIZED NICKEL|-|3|-|4 +Brand#42|-|PROMO ANODIZED NICKEL|-|19|-|4 +Brand#42|-|PROMO ANODIZED STEEL|-|49|-|4 +Brand#42|-|PROMO BRUSHED COPPER|-|45|-|4 +Brand#42|-|PROMO BRUSHED STEEL|-|19|-|4 +Brand#42|-|PROMO BRUSHED TIN|-|45|-|4 +Brand#42|-|PROMO BURNISHED COPPER|-|45|-|4 +Brand#42|-|PROMO BURNISHED NICKEL|-|3|-|4 +Brand#42|-|PROMO BURNISHED STEEL|-|9|-|4 +Brand#42|-|PROMO BURNISHED TIN|-|49|-|4 +Brand#42|-|PROMO PLATED BRASS|-|45|-|4 +Brand#42|-|PROMO PLATED NICKEL|-|23|-|4 +Brand#42|-|PROMO PLATED STEEL|-|19|-|4 +Brand#42|-|PROMO PLATED STEEL|-|45|-|4 +Brand#42|-|PROMO POLISHED COPPER|-|36|-|4 +Brand#42|-|PROMO POLISHED NICKEL|-|3|-|4 +Brand#42|-|SMALL ANODIZED BRASS|-|23|-|4 +Brand#42|-|SMALL ANODIZED COPPER|-|14|-|4 +Brand#42|-|SMALL ANODIZED COPPER|-|19|-|4 +Brand#42|-|SMALL ANODIZED NICKEL|-|23|-|4 +Brand#42|-|SMALL BRUSHED TIN|-|49|-|4 +Brand#42|-|SMALL BURNISHED BRASS|-|3|-|4 +Brand#42|-|SMALL BURNISHED BRASS|-|36|-|4 +Brand#42|-|SMALL BURNISHED COPPER|-|9|-|4 +Brand#42|-|SMALL BURNISHED NICKEL|-|9|-|4 +Brand#42|-|SMALL BURNISHED TIN|-|9|-|4 +Brand#42|-|SMALL PLATED NICKEL|-|9|-|4 +Brand#42|-|SMALL PLATED TIN|-|36|-|4 +Brand#42|-|SMALL POLISHED BRASS|-|3|-|4 +Brand#42|-|SMALL POLISHED COPPER|-|36|-|4 +Brand#42|-|SMALL POLISHED NICKEL|-|23|-|4 +Brand#42|-|SMALL POLISHED STEEL|-|49|-|4 +Brand#42|-|SMALL POLISHED TIN|-|3|-|4 +Brand#42|-|STANDARD ANODIZED BRASS|-|49|-|4 +Brand#42|-|STANDARD ANODIZED COPPER|-|49|-|4 +Brand#42|-|STANDARD ANODIZED NICKEL|-|36|-|4 +Brand#42|-|STANDARD ANODIZED NICKEL|-|45|-|4 +Brand#42|-|STANDARD BRUSHED NICKEL|-|23|-|4 +Brand#42|-|STANDARD BURNISHED NICKEL|-|49|-|4 +Brand#42|-|STANDARD BURNISHED STEEL|-|3|-|4 +Brand#42|-|STANDARD BURNISHED TIN|-|19|-|4 +Brand#42|-|STANDARD PLATED BRASS|-|19|-|4 +Brand#42|-|STANDARD PLATED COPPER|-|9|-|4 +Brand#42|-|STANDARD PLATED NICKEL|-|45|-|4 +Brand#42|-|STANDARD PLATED STEEL|-|3|-|4 +Brand#42|-|STANDARD POLISHED BRASS|-|36|-|4 +Brand#42|-|STANDARD POLISHED BRASS|-|45|-|4 +Brand#42|-|STANDARD POLISHED COPPER|-|14|-|4 +Brand#42|-|STANDARD POLISHED NICKEL|-|45|-|4 +Brand#42|-|STANDARD POLISHED TIN|-|9|-|4 +Brand#42|-|STANDARD POLISHED TIN|-|19|-|4 +Brand#42|-|STANDARD POLISHED TIN|-|23|-|4 +Brand#42|-|STANDARD POLISHED TIN|-|36|-|4 +Brand#43|-|ECONOMY ANODIZED COPPER|-|19|-|4 +Brand#43|-|ECONOMY ANODIZED COPPER|-|45|-|4 +Brand#43|-|ECONOMY ANODIZED NICKEL|-|3|-|4 +Brand#43|-|ECONOMY ANODIZED NICKEL|-|49|-|4 +Brand#43|-|ECONOMY ANODIZED STEEL|-|23|-|4 +Brand#43|-|ECONOMY ANODIZED TIN|-|49|-|4 +Brand#43|-|ECONOMY BRUSHED BRASS|-|49|-|4 +Brand#43|-|ECONOMY BRUSHED COPPER|-|45|-|4 +Brand#43|-|ECONOMY BRUSHED NICKEL|-|9|-|4 +Brand#43|-|ECONOMY BURNISHED NICKEL|-|9|-|4 +Brand#43|-|ECONOMY BURNISHED TIN|-|19|-|4 +Brand#43|-|ECONOMY PLATED COPPER|-|36|-|4 +Brand#43|-|ECONOMY PLATED STEEL|-|9|-|4 +Brand#43|-|ECONOMY PLATED TIN|-|14|-|4 +Brand#43|-|ECONOMY PLATED TIN|-|19|-|4 +Brand#43|-|ECONOMY PLATED TIN|-|49|-|4 +Brand#43|-|ECONOMY POLISHED COPPER|-|19|-|4 +Brand#43|-|ECONOMY POLISHED NICKEL|-|36|-|4 +Brand#43|-|ECONOMY POLISHED TIN|-|14|-|4 +Brand#43|-|ECONOMY POLISHED TIN|-|45|-|4 +Brand#43|-|LARGE ANODIZED BRASS|-|14|-|4 +Brand#43|-|LARGE ANODIZED BRASS|-|36|-|4 +Brand#43|-|LARGE ANODIZED COPPER|-|45|-|4 +Brand#43|-|LARGE BRUSHED COPPER|-|3|-|4 +Brand#43|-|LARGE BRUSHED NICKEL|-|14|-|4 +Brand#43|-|LARGE BRUSHED NICKEL|-|19|-|4 +Brand#43|-|LARGE BRUSHED NICKEL|-|45|-|4 +Brand#43|-|LARGE BRUSHED NICKEL|-|49|-|4 +Brand#43|-|LARGE BURNISHED COPPER|-|3|-|4 +Brand#43|-|LARGE BURNISHED TIN|-|23|-|4 +Brand#43|-|LARGE BURNISHED TIN|-|45|-|4 +Brand#43|-|LARGE PLATED BRASS|-|45|-|4 +Brand#43|-|LARGE PLATED STEEL|-|14|-|4 +Brand#43|-|LARGE PLATED TIN|-|36|-|4 +Brand#43|-|LARGE PLATED TIN|-|45|-|4 +Brand#43|-|LARGE POLISHED BRASS|-|9|-|4 +Brand#43|-|LARGE POLISHED COPPER|-|9|-|4 +Brand#43|-|LARGE POLISHED COPPER|-|19|-|4 +Brand#43|-|LARGE POLISHED STEEL|-|14|-|4 +Brand#43|-|LARGE POLISHED TIN|-|45|-|4 +Brand#43|-|MEDIUM ANODIZED BRASS|-|14|-|4 +Brand#43|-|MEDIUM ANODIZED COPPER|-|36|-|4 +Brand#43|-|MEDIUM ANODIZED COPPER|-|49|-|4 +Brand#43|-|MEDIUM ANODIZED STEEL|-|19|-|4 +Brand#43|-|MEDIUM ANODIZED STEEL|-|36|-|4 +Brand#43|-|MEDIUM BRUSHED BRASS|-|9|-|4 +Brand#43|-|MEDIUM BRUSHED BRASS|-|49|-|4 +Brand#43|-|MEDIUM BRUSHED COPPER|-|3|-|4 +Brand#43|-|MEDIUM BRUSHED NICKEL|-|9|-|4 +Brand#43|-|MEDIUM BRUSHED STEEL|-|23|-|4 +Brand#43|-|MEDIUM BURNISHED COPPER|-|14|-|4 +Brand#43|-|MEDIUM BURNISHED COPPER|-|45|-|4 +Brand#43|-|MEDIUM BURNISHED TIN|-|23|-|4 +Brand#43|-|MEDIUM PLATED BRASS|-|3|-|4 +Brand#43|-|MEDIUM PLATED COPPER|-|14|-|4 +Brand#43|-|MEDIUM PLATED NICKEL|-|36|-|4 +Brand#43|-|MEDIUM PLATED NICKEL|-|45|-|4 +Brand#43|-|MEDIUM PLATED TIN|-|49|-|4 +Brand#43|-|PROMO ANODIZED NICKEL|-|45|-|4 +Brand#43|-|PROMO ANODIZED TIN|-|14|-|4 +Brand#43|-|PROMO BRUSHED NICKEL|-|14|-|4 +Brand#43|-|PROMO BRUSHED STEEL|-|14|-|4 +Brand#43|-|PROMO BRUSHED TIN|-|45|-|4 +Brand#43|-|PROMO BURNISHED BRASS|-|49|-|4 +Brand#43|-|PROMO BURNISHED NICKEL|-|9|-|4 +Brand#43|-|PROMO BURNISHED STEEL|-|3|-|4 +Brand#43|-|PROMO BURNISHED STEEL|-|36|-|4 +Brand#43|-|PROMO BURNISHED TIN|-|36|-|4 +Brand#43|-|PROMO PLATED BRASS|-|19|-|4 +Brand#43|-|PROMO PLATED COPPER|-|45|-|4 +Brand#43|-|PROMO PLATED COPPER|-|49|-|4 +Brand#43|-|PROMO PLATED TIN|-|3|-|4 +Brand#43|-|PROMO POLISHED BRASS|-|19|-|4 +Brand#43|-|PROMO POLISHED BRASS|-|23|-|4 +Brand#43|-|PROMO POLISHED NICKEL|-|49|-|4 +Brand#43|-|PROMO POLISHED STEEL|-|14|-|4 +Brand#43|-|PROMO POLISHED STEEL|-|19|-|4 +Brand#43|-|PROMO POLISHED STEEL|-|23|-|4 +Brand#43|-|PROMO POLISHED STEEL|-|36|-|4 +Brand#43|-|SMALL ANODIZED BRASS|-|19|-|4 +Brand#43|-|SMALL ANODIZED NICKEL|-|9|-|4 +Brand#43|-|SMALL BRUSHED NICKEL|-|3|-|4 +Brand#43|-|SMALL BRUSHED NICKEL|-|9|-|4 +Brand#43|-|SMALL BURNISHED BRASS|-|49|-|4 +Brand#43|-|SMALL BURNISHED STEEL|-|23|-|4 +Brand#43|-|SMALL PLATED BRASS|-|14|-|4 +Brand#43|-|SMALL PLATED BRASS|-|36|-|4 +Brand#43|-|SMALL PLATED COPPER|-|23|-|4 +Brand#43|-|SMALL PLATED COPPER|-|49|-|4 +Brand#43|-|SMALL PLATED NICKEL|-|36|-|4 +Brand#43|-|SMALL PLATED NICKEL|-|49|-|4 +Brand#43|-|SMALL PLATED STEEL|-|14|-|4 +Brand#43|-|SMALL PLATED TIN|-|49|-|4 +Brand#43|-|SMALL POLISHED STEEL|-|19|-|4 +Brand#43|-|STANDARD ANODIZED BRASS|-|3|-|4 +Brand#43|-|STANDARD ANODIZED COPPER|-|49|-|4 +Brand#43|-|STANDARD ANODIZED NICKEL|-|14|-|4 +Brand#43|-|STANDARD BRUSHED TIN|-|14|-|4 +Brand#43|-|STANDARD BURNISHED BRASS|-|23|-|4 +Brand#43|-|STANDARD BURNISHED STEEL|-|19|-|4 +Brand#43|-|STANDARD BURNISHED STEEL|-|23|-|4 +Brand#43|-|STANDARD PLATED BRASS|-|9|-|4 +Brand#43|-|STANDARD PLATED BRASS|-|19|-|4 +Brand#43|-|STANDARD PLATED BRASS|-|49|-|4 +Brand#43|-|STANDARD PLATED COPPER|-|36|-|4 +Brand#43|-|STANDARD PLATED NICKEL|-|14|-|4 +Brand#43|-|STANDARD PLATED NICKEL|-|19|-|4 +Brand#43|-|STANDARD PLATED TIN|-|14|-|4 +Brand#43|-|STANDARD POLISHED BRASS|-|23|-|4 +Brand#43|-|STANDARD POLISHED TIN|-|9|-|4 +Brand#44|-|ECONOMY ANODIZED BRASS|-|3|-|4 +Brand#44|-|ECONOMY ANODIZED BRASS|-|45|-|4 +Brand#44|-|ECONOMY ANODIZED NICKEL|-|36|-|4 +Brand#44|-|ECONOMY ANODIZED STEEL|-|19|-|4 +Brand#44|-|ECONOMY BRUSHED COPPER|-|23|-|4 +Brand#44|-|ECONOMY BRUSHED TIN|-|49|-|4 +Brand#44|-|ECONOMY BURNISHED COPPER|-|19|-|4 +Brand#44|-|ECONOMY BURNISHED STEEL|-|45|-|4 +Brand#44|-|ECONOMY PLATED STEEL|-|19|-|4 +Brand#44|-|ECONOMY PLATED STEEL|-|23|-|4 +Brand#44|-|ECONOMY PLATED TIN|-|23|-|4 +Brand#44|-|ECONOMY POLISHED BRASS|-|23|-|4 +Brand#44|-|ECONOMY POLISHED COPPER|-|9|-|4 +Brand#44|-|ECONOMY POLISHED COPPER|-|45|-|4 +Brand#44|-|ECONOMY POLISHED NICKEL|-|14|-|4 +Brand#44|-|ECONOMY POLISHED NICKEL|-|23|-|4 +Brand#44|-|ECONOMY POLISHED STEEL|-|49|-|4 +Brand#44|-|ECONOMY POLISHED TIN|-|23|-|4 +Brand#44|-|ECONOMY POLISHED TIN|-|36|-|4 +Brand#44|-|LARGE ANODIZED BRASS|-|19|-|4 +Brand#44|-|LARGE ANODIZED TIN|-|3|-|4 +Brand#44|-|LARGE ANODIZED TIN|-|14|-|4 +Brand#44|-|LARGE BRUSHED TIN|-|3|-|4 +Brand#44|-|LARGE BRUSHED TIN|-|23|-|4 +Brand#44|-|LARGE BURNISHED BRASS|-|23|-|4 +Brand#44|-|LARGE BURNISHED BRASS|-|49|-|4 +Brand#44|-|LARGE BURNISHED COPPER|-|3|-|4 +Brand#44|-|LARGE BURNISHED COPPER|-|19|-|4 +Brand#44|-|LARGE BURNISHED COPPER|-|36|-|4 +Brand#44|-|LARGE BURNISHED TIN|-|14|-|4 +Brand#44|-|LARGE PLATED BRASS|-|9|-|4 +Brand#44|-|LARGE PLATED BRASS|-|49|-|4 +Brand#44|-|LARGE PLATED NICKEL|-|14|-|4 +Brand#44|-|LARGE PLATED STEEL|-|14|-|4 +Brand#44|-|LARGE PLATED TIN|-|19|-|4 +Brand#44|-|LARGE PLATED TIN|-|23|-|4 +Brand#44|-|LARGE POLISHED STEEL|-|23|-|4 +Brand#44|-|LARGE POLISHED STEEL|-|49|-|4 +Brand#44|-|MEDIUM ANODIZED COPPER|-|45|-|4 +Brand#44|-|MEDIUM ANODIZED NICKEL|-|45|-|4 +Brand#44|-|MEDIUM BRUSHED BRASS|-|49|-|4 +Brand#44|-|MEDIUM BRUSHED COPPER|-|3|-|4 +Brand#44|-|MEDIUM BRUSHED COPPER|-|45|-|4 +Brand#44|-|MEDIUM BRUSHED STEEL|-|19|-|4 +Brand#44|-|MEDIUM BRUSHED TIN|-|49|-|4 +Brand#44|-|MEDIUM BURNISHED COPPER|-|45|-|4 +Brand#44|-|MEDIUM BURNISHED NICKEL|-|23|-|4 +Brand#44|-|MEDIUM BURNISHED TIN|-|23|-|4 +Brand#44|-|MEDIUM PLATED COPPER|-|14|-|4 +Brand#44|-|PROMO ANODIZED COPPER|-|23|-|4 +Brand#44|-|PROMO ANODIZED STEEL|-|36|-|4 +Brand#44|-|PROMO BRUSHED COPPER|-|23|-|4 +Brand#44|-|PROMO BRUSHED COPPER|-|36|-|4 +Brand#44|-|PROMO BRUSHED TIN|-|19|-|4 +Brand#44|-|PROMO PLATED BRASS|-|3|-|4 +Brand#44|-|PROMO PLATED COPPER|-|36|-|4 +Brand#44|-|PROMO PLATED STEEL|-|3|-|4 +Brand#44|-|PROMO PLATED STEEL|-|36|-|4 +Brand#44|-|PROMO PLATED STEEL|-|49|-|4 +Brand#44|-|PROMO POLISHED BRASS|-|3|-|4 +Brand#44|-|PROMO POLISHED BRASS|-|19|-|4 +Brand#44|-|PROMO POLISHED COPPER|-|45|-|4 +Brand#44|-|PROMO POLISHED STEEL|-|36|-|4 +Brand#44|-|PROMO POLISHED TIN|-|9|-|4 +Brand#44|-|SMALL ANODIZED COPPER|-|23|-|4 +Brand#44|-|SMALL ANODIZED STEEL|-|23|-|4 +Brand#44|-|SMALL ANODIZED TIN|-|45|-|4 +Brand#44|-|SMALL BRUSHED COPPER|-|14|-|4 +Brand#44|-|SMALL BRUSHED STEEL|-|45|-|4 +Brand#44|-|SMALL BURNISHED COPPER|-|14|-|4 +Brand#44|-|SMALL BURNISHED COPPER|-|49|-|4 +Brand#44|-|SMALL BURNISHED NICKEL|-|14|-|4 +Brand#44|-|SMALL BURNISHED STEEL|-|23|-|4 +Brand#44|-|SMALL BURNISHED TIN|-|49|-|4 +Brand#44|-|SMALL PLATED BRASS|-|36|-|4 +Brand#44|-|SMALL PLATED COPPER|-|19|-|4 +Brand#44|-|SMALL PLATED NICKEL|-|3|-|4 +Brand#44|-|SMALL POLISHED COPPER|-|3|-|4 +Brand#44|-|SMALL POLISHED COPPER|-|49|-|4 +Brand#44|-|SMALL POLISHED STEEL|-|3|-|4 +Brand#44|-|STANDARD ANODIZED BRASS|-|3|-|4 +Brand#44|-|STANDARD ANODIZED COPPER|-|3|-|4 +Brand#44|-|STANDARD ANODIZED NICKEL|-|3|-|4 +Brand#44|-|STANDARD ANODIZED NICKEL|-|36|-|4 +Brand#44|-|STANDARD ANODIZED STEEL|-|14|-|4 +Brand#44|-|STANDARD ANODIZED TIN|-|3|-|4 +Brand#44|-|STANDARD ANODIZED TIN|-|9|-|4 +Brand#44|-|STANDARD ANODIZED TIN|-|36|-|4 +Brand#44|-|STANDARD BRUSHED COPPER|-|36|-|4 +Brand#44|-|STANDARD BRUSHED COPPER|-|45|-|4 +Brand#44|-|STANDARD BRUSHED TIN|-|9|-|4 +Brand#44|-|STANDARD BRUSHED TIN|-|49|-|4 +Brand#44|-|STANDARD BURNISHED COPPER|-|9|-|4 +Brand#44|-|STANDARD BURNISHED STEEL|-|23|-|4 +Brand#44|-|STANDARD PLATED BRASS|-|14|-|4 +Brand#44|-|STANDARD PLATED BRASS|-|23|-|4 +Brand#44|-|STANDARD PLATED BRASS|-|49|-|4 +Brand#44|-|STANDARD PLATED COPPER|-|14|-|4 +Brand#44|-|STANDARD POLISHED NICKEL|-|19|-|4 +Brand#44|-|STANDARD POLISHED TIN|-|9|-|4 +Brand#51|-|ECONOMY ANODIZED BRASS|-|9|-|4 +Brand#51|-|ECONOMY ANODIZED BRASS|-|23|-|4 +Brand#51|-|ECONOMY ANODIZED NICKEL|-|3|-|4 +Brand#51|-|ECONOMY ANODIZED NICKEL|-|23|-|4 +Brand#51|-|ECONOMY ANODIZED STEEL|-|19|-|4 +Brand#51|-|ECONOMY ANODIZED STEEL|-|23|-|4 +Brand#51|-|ECONOMY ANODIZED STEEL|-|49|-|4 +Brand#51|-|ECONOMY BRUSHED BRASS|-|3|-|4 +Brand#51|-|ECONOMY BRUSHED BRASS|-|49|-|4 +Brand#51|-|ECONOMY BRUSHED NICKEL|-|14|-|4 +Brand#51|-|ECONOMY BRUSHED STEEL|-|45|-|4 +Brand#51|-|ECONOMY BRUSHED TIN|-|36|-|4 +Brand#51|-|ECONOMY BURNISHED BRASS|-|14|-|4 +Brand#51|-|ECONOMY BURNISHED COPPER|-|45|-|4 +Brand#51|-|ECONOMY PLATED NICKEL|-|49|-|4 +Brand#51|-|ECONOMY PLATED TIN|-|36|-|4 +Brand#51|-|ECONOMY POLISHED COPPER|-|9|-|4 +Brand#51|-|ECONOMY POLISHED STEEL|-|14|-|4 +Brand#51|-|ECONOMY POLISHED STEEL|-|49|-|4 +Brand#51|-|LARGE ANODIZED COPPER|-|9|-|4 +Brand#51|-|LARGE ANODIZED COPPER|-|49|-|4 +Brand#51|-|LARGE ANODIZED NICKEL|-|14|-|4 +Brand#51|-|LARGE ANODIZED STEEL|-|36|-|4 +Brand#51|-|LARGE BRUSHED NICKEL|-|3|-|4 +Brand#51|-|LARGE BRUSHED NICKEL|-|9|-|4 +Brand#51|-|LARGE BURNISHED BRASS|-|19|-|4 +Brand#51|-|LARGE BURNISHED BRASS|-|36|-|4 +Brand#51|-|LARGE BURNISHED COPPER|-|14|-|4 +Brand#51|-|LARGE BURNISHED NICKEL|-|14|-|4 +Brand#51|-|LARGE PLATED BRASS|-|36|-|4 +Brand#51|-|LARGE POLISHED COPPER|-|14|-|4 +Brand#51|-|LARGE POLISHED NICKEL|-|23|-|4 +Brand#51|-|LARGE POLISHED NICKEL|-|36|-|4 +Brand#51|-|LARGE POLISHED STEEL|-|19|-|4 +Brand#51|-|MEDIUM ANODIZED COPPER|-|9|-|4 +Brand#51|-|MEDIUM ANODIZED STEEL|-|3|-|4 +Brand#51|-|MEDIUM BRUSHED BRASS|-|36|-|4 +Brand#51|-|MEDIUM BRUSHED BRASS|-|45|-|4 +Brand#51|-|MEDIUM BRUSHED STEEL|-|3|-|4 +Brand#51|-|MEDIUM BRUSHED TIN|-|36|-|4 +Brand#51|-|MEDIUM BURNISHED NICKEL|-|3|-|4 +Brand#51|-|MEDIUM BURNISHED NICKEL|-|36|-|4 +Brand#51|-|MEDIUM BURNISHED STEEL|-|14|-|4 +Brand#51|-|MEDIUM BURNISHED TIN|-|9|-|4 +Brand#51|-|MEDIUM PLATED STEEL|-|19|-|4 +Brand#51|-|MEDIUM PLATED TIN|-|3|-|4 +Brand#51|-|PROMO ANODIZED NICKEL|-|14|-|4 +Brand#51|-|PROMO ANODIZED STEEL|-|23|-|4 +Brand#51|-|PROMO ANODIZED TIN|-|19|-|4 +Brand#51|-|PROMO BRUSHED BRASS|-|23|-|4 +Brand#51|-|PROMO BRUSHED COPPER|-|45|-|4 +Brand#51|-|PROMO BRUSHED STEEL|-|45|-|4 +Brand#51|-|PROMO BRUSHED TIN|-|9|-|4 +Brand#51|-|PROMO BURNISHED BRASS|-|19|-|4 +Brand#51|-|PROMO BURNISHED BRASS|-|23|-|4 +Brand#51|-|PROMO BURNISHED NICKEL|-|14|-|4 +Brand#51|-|PROMO PLATED BRASS|-|3|-|4 +Brand#51|-|PROMO PLATED BRASS|-|23|-|4 +Brand#51|-|PROMO PLATED TIN|-|19|-|4 +Brand#51|-|PROMO PLATED TIN|-|23|-|4 +Brand#51|-|PROMO POLISHED BRASS|-|23|-|4 +Brand#51|-|PROMO POLISHED COPPER|-|9|-|4 +Brand#51|-|PROMO POLISHED NICKEL|-|9|-|4 +Brand#51|-|PROMO POLISHED STEEL|-|49|-|4 +Brand#51|-|SMALL ANODIZED STEEL|-|14|-|4 +Brand#51|-|SMALL BRUSHED BRASS|-|23|-|4 +Brand#51|-|SMALL BRUSHED TIN|-|19|-|4 +Brand#51|-|SMALL BURNISHED NICKEL|-|23|-|4 +Brand#51|-|SMALL PLATED COPPER|-|49|-|4 +Brand#51|-|SMALL PLATED NICKEL|-|3|-|4 +Brand#51|-|SMALL PLATED NICKEL|-|14|-|4 +Brand#51|-|SMALL PLATED STEEL|-|45|-|4 +Brand#51|-|SMALL POLISHED NICKEL|-|14|-|4 +Brand#51|-|SMALL POLISHED NICKEL|-|23|-|4 +Brand#51|-|SMALL POLISHED STEEL|-|3|-|4 +Brand#51|-|SMALL POLISHED STEEL|-|19|-|4 +Brand#51|-|SMALL POLISHED STEEL|-|49|-|4 +Brand#51|-|STANDARD ANODIZED NICKEL|-|3|-|4 +Brand#51|-|STANDARD ANODIZED NICKEL|-|49|-|4 +Brand#51|-|STANDARD BRUSHED BRASS|-|3|-|4 +Brand#51|-|STANDARD BRUSHED COPPER|-|3|-|4 +Brand#51|-|STANDARD BRUSHED NICKEL|-|19|-|4 +Brand#51|-|STANDARD BRUSHED STEEL|-|36|-|4 +Brand#51|-|STANDARD BURNISHED COPPER|-|19|-|4 +Brand#51|-|STANDARD BURNISHED NICKEL|-|49|-|4 +Brand#51|-|STANDARD BURNISHED STEEL|-|23|-|4 +Brand#51|-|STANDARD BURNISHED STEEL|-|36|-|4 +Brand#51|-|STANDARD BURNISHED TIN|-|45|-|4 +Brand#51|-|STANDARD PLATED BRASS|-|36|-|4 +Brand#51|-|STANDARD PLATED BRASS|-|49|-|4 +Brand#51|-|STANDARD PLATED COPPER|-|14|-|4 +Brand#51|-|STANDARD PLATED COPPER|-|23|-|4 +Brand#51|-|STANDARD POLISHED BRASS|-|14|-|4 +Brand#51|-|STANDARD POLISHED BRASS|-|45|-|4 +Brand#51|-|STANDARD POLISHED STEEL|-|36|-|4 +Brand#51|-|STANDARD POLISHED STEEL|-|49|-|4 +Brand#51|-|STANDARD POLISHED TIN|-|45|-|4 +Brand#52|-|ECONOMY ANODIZED BRASS|-|14|-|4 +Brand#52|-|ECONOMY ANODIZED BRASS|-|23|-|4 +Brand#52|-|ECONOMY ANODIZED COPPER|-|36|-|4 +Brand#52|-|ECONOMY ANODIZED NICKEL|-|49|-|4 +Brand#52|-|ECONOMY ANODIZED STEEL|-|19|-|4 +Brand#52|-|ECONOMY BRUSHED COPPER|-|49|-|4 +Brand#52|-|ECONOMY BURNISHED BRASS|-|36|-|4 +Brand#52|-|ECONOMY BURNISHED COPPER|-|19|-|4 +Brand#52|-|ECONOMY BURNISHED COPPER|-|45|-|4 +Brand#52|-|ECONOMY BURNISHED NICKEL|-|19|-|4 +Brand#52|-|ECONOMY BURNISHED STEEL|-|36|-|4 +Brand#52|-|ECONOMY PLATED TIN|-|14|-|4 +Brand#52|-|ECONOMY PLATED TIN|-|23|-|4 +Brand#52|-|ECONOMY POLISHED BRASS|-|23|-|4 +Brand#52|-|ECONOMY POLISHED BRASS|-|45|-|4 +Brand#52|-|ECONOMY POLISHED NICKEL|-|36|-|4 +Brand#52|-|ECONOMY POLISHED STEEL|-|49|-|4 +Brand#52|-|LARGE ANODIZED COPPER|-|14|-|4 +Brand#52|-|LARGE ANODIZED NICKEL|-|3|-|4 +Brand#52|-|LARGE ANODIZED NICKEL|-|45|-|4 +Brand#52|-|LARGE ANODIZED TIN|-|45|-|4 +Brand#52|-|LARGE BRUSHED COPPER|-|19|-|4 +Brand#52|-|LARGE BRUSHED NICKEL|-|3|-|4 +Brand#52|-|LARGE BRUSHED NICKEL|-|19|-|4 +Brand#52|-|LARGE BRUSHED NICKEL|-|23|-|4 +Brand#52|-|LARGE BRUSHED STEEL|-|49|-|4 +Brand#52|-|LARGE BRUSHED TIN|-|14|-|4 +Brand#52|-|LARGE BURNISHED NICKEL|-|9|-|4 +Brand#52|-|LARGE BURNISHED TIN|-|23|-|4 +Brand#52|-|LARGE BURNISHED TIN|-|45|-|4 +Brand#52|-|LARGE PLATED BRASS|-|14|-|4 +Brand#52|-|LARGE PLATED COPPER|-|14|-|4 +Brand#52|-|LARGE PLATED COPPER|-|19|-|4 +Brand#52|-|LARGE PLATED NICKEL|-|45|-|4 +Brand#52|-|LARGE PLATED STEEL|-|9|-|4 +Brand#52|-|LARGE PLATED TIN|-|9|-|4 +Brand#52|-|LARGE POLISHED NICKEL|-|19|-|4 +Brand#52|-|LARGE POLISHED NICKEL|-|23|-|4 +Brand#52|-|LARGE POLISHED NICKEL|-|36|-|4 +Brand#52|-|LARGE POLISHED TIN|-|9|-|4 +Brand#52|-|MEDIUM ANODIZED COPPER|-|36|-|4 +Brand#52|-|MEDIUM ANODIZED STEEL|-|14|-|4 +Brand#52|-|MEDIUM ANODIZED TIN|-|3|-|4 +Brand#52|-|MEDIUM ANODIZED TIN|-|49|-|4 +Brand#52|-|MEDIUM BRUSHED COPPER|-|9|-|4 +Brand#52|-|MEDIUM BRUSHED NICKEL|-|9|-|4 +Brand#52|-|MEDIUM BRUSHED STEEL|-|23|-|4 +Brand#52|-|MEDIUM BRUSHED STEEL|-|49|-|4 +Brand#52|-|MEDIUM BURNISHED STEEL|-|23|-|4 +Brand#52|-|MEDIUM BURNISHED TIN|-|45|-|4 +Brand#52|-|MEDIUM BURNISHED TIN|-|49|-|4 +Brand#52|-|MEDIUM PLATED BRASS|-|36|-|4 +Brand#52|-|MEDIUM PLATED STEEL|-|9|-|4 +Brand#52|-|MEDIUM PLATED STEEL|-|49|-|4 +Brand#52|-|MEDIUM PLATED TIN|-|9|-|4 +Brand#52|-|MEDIUM PLATED TIN|-|49|-|4 +Brand#52|-|PROMO ANODIZED BRASS|-|9|-|4 +Brand#52|-|PROMO ANODIZED BRASS|-|23|-|4 +Brand#52|-|PROMO ANODIZED BRASS|-|36|-|4 +Brand#52|-|PROMO ANODIZED NICKEL|-|45|-|4 +Brand#52|-|PROMO ANODIZED STEEL|-|36|-|4 +Brand#52|-|PROMO BRUSHED COPPER|-|3|-|4 +Brand#52|-|PROMO BRUSHED NICKEL|-|3|-|4 +Brand#52|-|PROMO BRUSHED NICKEL|-|49|-|4 +Brand#52|-|PROMO BRUSHED STEEL|-|14|-|4 +Brand#52|-|PROMO BRUSHED TIN|-|3|-|4 +Brand#52|-|PROMO BRUSHED TIN|-|19|-|4 +Brand#52|-|PROMO BRUSHED TIN|-|36|-|4 +Brand#52|-|PROMO BURNISHED COPPER|-|49|-|4 +Brand#52|-|PROMO BURNISHED NICKEL|-|9|-|4 +Brand#52|-|PROMO BURNISHED STEEL|-|9|-|4 +Brand#52|-|PROMO BURNISHED STEEL|-|23|-|4 +Brand#52|-|PROMO BURNISHED TIN|-|19|-|4 +Brand#52|-|PROMO BURNISHED TIN|-|36|-|4 +Brand#52|-|PROMO PLATED BRASS|-|19|-|4 +Brand#52|-|PROMO PLATED BRASS|-|45|-|4 +Brand#52|-|PROMO PLATED BRASS|-|49|-|4 +Brand#52|-|PROMO PLATED COPPER|-|9|-|4 +Brand#52|-|PROMO PLATED NICKEL|-|3|-|4 +Brand#52|-|PROMO PLATED NICKEL|-|23|-|4 +Brand#52|-|PROMO POLISHED NICKEL|-|14|-|4 +Brand#52|-|PROMO POLISHED NICKEL|-|49|-|4 +Brand#52|-|PROMO POLISHED TIN|-|36|-|4 +Brand#52|-|SMALL ANODIZED BRASS|-|3|-|4 +Brand#52|-|SMALL ANODIZED BRASS|-|14|-|4 +Brand#52|-|SMALL ANODIZED COPPER|-|3|-|4 +Brand#52|-|SMALL ANODIZED NICKEL|-|36|-|4 +Brand#52|-|SMALL ANODIZED STEEL|-|9|-|4 +Brand#52|-|SMALL ANODIZED STEEL|-|19|-|4 +Brand#52|-|SMALL BRUSHED NICKEL|-|19|-|4 +Brand#52|-|SMALL BRUSHED STEEL|-|23|-|4 +Brand#52|-|SMALL BRUSHED TIN|-|14|-|4 +Brand#52|-|SMALL BRUSHED TIN|-|19|-|4 +Brand#52|-|SMALL BURNISHED NICKEL|-|14|-|4 +Brand#52|-|SMALL BURNISHED NICKEL|-|49|-|4 +Brand#52|-|SMALL BURNISHED TIN|-|9|-|4 +Brand#52|-|SMALL POLISHED BRASS|-|36|-|4 +Brand#52|-|SMALL POLISHED BRASS|-|49|-|4 +Brand#52|-|SMALL POLISHED TIN|-|45|-|4 +Brand#52|-|STANDARD ANODIZED BRASS|-|45|-|4 +Brand#52|-|STANDARD BRUSHED BRASS|-|23|-|4 +Brand#52|-|STANDARD BRUSHED COPPER|-|14|-|4 +Brand#52|-|STANDARD BRUSHED TIN|-|36|-|4 +Brand#52|-|STANDARD BURNISHED BRASS|-|49|-|4 +Brand#52|-|STANDARD BURNISHED STEEL|-|19|-|4 +Brand#52|-|STANDARD BURNISHED TIN|-|9|-|4 +Brand#52|-|STANDARD BURNISHED TIN|-|19|-|4 +Brand#52|-|STANDARD PLATED NICKEL|-|36|-|4 +Brand#52|-|STANDARD PLATED STEEL|-|36|-|4 +Brand#52|-|STANDARD POLISHED BRASS|-|36|-|4 +Brand#52|-|STANDARD POLISHED COPPER|-|45|-|4 +Brand#52|-|STANDARD POLISHED STEEL|-|19|-|4 +Brand#52|-|STANDARD POLISHED TIN|-|19|-|4 +Brand#53|-|ECONOMY ANODIZED BRASS|-|45|-|4 +Brand#53|-|ECONOMY ANODIZED COPPER|-|9|-|4 +Brand#53|-|ECONOMY ANODIZED NICKEL|-|3|-|4 +Brand#53|-|ECONOMY ANODIZED NICKEL|-|19|-|4 +Brand#53|-|ECONOMY ANODIZED STEEL|-|45|-|4 +Brand#53|-|ECONOMY ANODIZED TIN|-|14|-|4 +Brand#53|-|ECONOMY ANODIZED TIN|-|36|-|4 +Brand#53|-|ECONOMY BRUSHED TIN|-|45|-|4 +Brand#53|-|ECONOMY BURNISHED BRASS|-|14|-|4 +Brand#53|-|ECONOMY BURNISHED COPPER|-|45|-|4 +Brand#53|-|ECONOMY BURNISHED NICKEL|-|3|-|4 +Brand#53|-|ECONOMY BURNISHED NICKEL|-|49|-|4 +Brand#53|-|ECONOMY BURNISHED TIN|-|45|-|4 +Brand#53|-|ECONOMY PLATED BRASS|-|3|-|4 +Brand#53|-|ECONOMY PLATED NICKEL|-|14|-|4 +Brand#53|-|ECONOMY PLATED STEEL|-|23|-|4 +Brand#53|-|ECONOMY PLATED STEEL|-|36|-|4 +Brand#53|-|ECONOMY POLISHED TIN|-|36|-|4 +Brand#53|-|LARGE ANODIZED NICKEL|-|49|-|4 +Brand#53|-|LARGE ANODIZED STEEL|-|19|-|4 +Brand#53|-|LARGE BRUSHED COPPER|-|3|-|4 +Brand#53|-|LARGE BRUSHED COPPER|-|14|-|4 +Brand#53|-|LARGE BRUSHED NICKEL|-|23|-|4 +Brand#53|-|LARGE BRUSHED NICKEL|-|36|-|4 +Brand#53|-|LARGE BRUSHED TIN|-|36|-|4 +Brand#53|-|LARGE BURNISHED BRASS|-|45|-|4 +Brand#53|-|LARGE BURNISHED COPPER|-|19|-|4 +Brand#53|-|LARGE BURNISHED COPPER|-|36|-|4 +Brand#53|-|LARGE BURNISHED NICKEL|-|23|-|4 +Brand#53|-|LARGE BURNISHED STEEL|-|19|-|4 +Brand#53|-|LARGE BURNISHED STEEL|-|23|-|4 +Brand#53|-|LARGE PLATED BRASS|-|9|-|4 +Brand#53|-|LARGE PLATED BRASS|-|45|-|4 +Brand#53|-|LARGE PLATED BRASS|-|49|-|4 +Brand#53|-|LARGE PLATED COPPER|-|23|-|4 +Brand#53|-|LARGE PLATED NICKEL|-|23|-|4 +Brand#53|-|LARGE PLATED NICKEL|-|49|-|4 +Brand#53|-|LARGE PLATED STEEL|-|49|-|4 +Brand#53|-|LARGE PLATED TIN|-|14|-|4 +Brand#53|-|LARGE POLISHED COPPER|-|49|-|4 +Brand#53|-|LARGE POLISHED STEEL|-|36|-|4 +Brand#53|-|LARGE POLISHED TIN|-|9|-|4 +Brand#53|-|MEDIUM ANODIZED BRASS|-|23|-|4 +Brand#53|-|MEDIUM ANODIZED STEEL|-|14|-|4 +Brand#53|-|MEDIUM ANODIZED STEEL|-|36|-|4 +Brand#53|-|MEDIUM ANODIZED TIN|-|3|-|4 +Brand#53|-|MEDIUM ANODIZED TIN|-|9|-|4 +Brand#53|-|MEDIUM BRUSHED BRASS|-|3|-|4 +Brand#53|-|MEDIUM BRUSHED COPPER|-|3|-|4 +Brand#53|-|MEDIUM BRUSHED NICKEL|-|14|-|4 +Brand#53|-|MEDIUM BRUSHED NICKEL|-|36|-|4 +Brand#53|-|MEDIUM BRUSHED NICKEL|-|49|-|4 +Brand#53|-|MEDIUM BRUSHED STEEL|-|45|-|4 +Brand#53|-|MEDIUM BURNISHED BRASS|-|3|-|4 +Brand#53|-|MEDIUM BURNISHED BRASS|-|36|-|4 +Brand#53|-|MEDIUM BURNISHED TIN|-|9|-|4 +Brand#53|-|MEDIUM BURNISHED TIN|-|14|-|4 +Brand#53|-|MEDIUM BURNISHED TIN|-|36|-|4 +Brand#53|-|MEDIUM PLATED BRASS|-|23|-|4 +Brand#53|-|MEDIUM PLATED COPPER|-|14|-|4 +Brand#53|-|MEDIUM PLATED NICKEL|-|45|-|4 +Brand#53|-|MEDIUM PLATED TIN|-|19|-|4 +Brand#53|-|MEDIUM PLATED TIN|-|45|-|4 +Brand#53|-|PROMO ANODIZED BRASS|-|36|-|4 +Brand#53|-|PROMO ANODIZED NICKEL|-|3|-|4 +Brand#53|-|PROMO ANODIZED NICKEL|-|19|-|4 +Brand#53|-|PROMO BRUSHED BRASS|-|45|-|4 +Brand#53|-|PROMO BRUSHED COPPER|-|3|-|4 +Brand#53|-|PROMO BRUSHED COPPER|-|23|-|4 +Brand#53|-|PROMO BRUSHED COPPER|-|45|-|4 +Brand#53|-|PROMO BURNISHED BRASS|-|23|-|4 +Brand#53|-|PROMO BURNISHED BRASS|-|36|-|4 +Brand#53|-|PROMO BURNISHED NICKEL|-|23|-|4 +Brand#53|-|PROMO BURNISHED STEEL|-|23|-|4 +Brand#53|-|PROMO BURNISHED STEEL|-|49|-|4 +Brand#53|-|PROMO PLATED TIN|-|19|-|4 +Brand#53|-|PROMO PLATED TIN|-|23|-|4 +Brand#53|-|PROMO PLATED TIN|-|36|-|4 +Brand#53|-|PROMO POLISHED STEEL|-|23|-|4 +Brand#53|-|PROMO POLISHED TIN|-|3|-|4 +Brand#53|-|SMALL ANODIZED COPPER|-|23|-|4 +Brand#53|-|SMALL ANODIZED COPPER|-|36|-|4 +Brand#53|-|SMALL ANODIZED COPPER|-|49|-|4 +Brand#53|-|SMALL ANODIZED NICKEL|-|36|-|4 +Brand#53|-|SMALL BRUSHED BRASS|-|36|-|4 +Brand#53|-|SMALL BRUSHED COPPER|-|3|-|4 +Brand#53|-|SMALL BRUSHED TIN|-|3|-|4 +Brand#53|-|SMALL BRUSHED TIN|-|36|-|4 +Brand#53|-|SMALL BURNISHED BRASS|-|9|-|4 +Brand#53|-|SMALL BURNISHED BRASS|-|49|-|4 +Brand#53|-|SMALL BURNISHED COPPER|-|19|-|4 +Brand#53|-|SMALL BURNISHED COPPER|-|45|-|4 +Brand#53|-|SMALL PLATED BRASS|-|9|-|4 +Brand#53|-|SMALL PLATED COPPER|-|3|-|4 +Brand#53|-|SMALL PLATED NICKEL|-|14|-|4 +Brand#53|-|SMALL POLISHED NICKEL|-|19|-|4 +Brand#53|-|SMALL POLISHED STEEL|-|36|-|4 +Brand#53|-|SMALL POLISHED TIN|-|23|-|4 +Brand#53|-|STANDARD ANODIZED BRASS|-|14|-|4 +Brand#53|-|STANDARD ANODIZED NICKEL|-|9|-|4 +Brand#53|-|STANDARD ANODIZED NICKEL|-|23|-|4 +Brand#53|-|STANDARD ANODIZED NICKEL|-|45|-|4 +Brand#53|-|STANDARD ANODIZED STEEL|-|45|-|4 +Brand#53|-|STANDARD BRUSHED COPPER|-|3|-|4 +Brand#53|-|STANDARD BRUSHED NICKEL|-|23|-|4 +Brand#53|-|STANDARD BRUSHED TIN|-|14|-|4 +Brand#53|-|STANDARD BURNISHED NICKEL|-|49|-|4 +Brand#53|-|STANDARD BURNISHED STEEL|-|9|-|4 +Brand#53|-|STANDARD PLATED BRASS|-|36|-|4 +Brand#53|-|STANDARD PLATED COPPER|-|45|-|4 +Brand#53|-|STANDARD PLATED NICKEL|-|36|-|4 +Brand#53|-|STANDARD PLATED STEEL|-|3|-|4 +Brand#53|-|STANDARD PLATED STEEL|-|49|-|4 +Brand#53|-|STANDARD PLATED TIN|-|23|-|4 +Brand#53|-|STANDARD POLISHED STEEL|-|3|-|4 +Brand#54|-|ECONOMY ANODIZED BRASS|-|9|-|4 +Brand#54|-|ECONOMY ANODIZED BRASS|-|45|-|4 +Brand#54|-|ECONOMY ANODIZED COPPER|-|9|-|4 +Brand#54|-|ECONOMY ANODIZED STEEL|-|19|-|4 +Brand#54|-|ECONOMY BRUSHED BRASS|-|45|-|4 +Brand#54|-|ECONOMY BRUSHED NICKEL|-|19|-|4 +Brand#54|-|ECONOMY BRUSHED STEEL|-|3|-|4 +Brand#54|-|ECONOMY BRUSHED TIN|-|19|-|4 +Brand#54|-|ECONOMY BURNISHED BRASS|-|45|-|4 +Brand#54|-|ECONOMY BURNISHED COPPER|-|14|-|4 +Brand#54|-|ECONOMY BURNISHED NICKEL|-|9|-|4 +Brand#54|-|ECONOMY BURNISHED NICKEL|-|36|-|4 +Brand#54|-|ECONOMY BURNISHED STEEL|-|36|-|4 +Brand#54|-|ECONOMY BURNISHED TIN|-|9|-|4 +Brand#54|-|ECONOMY BURNISHED TIN|-|14|-|4 +Brand#54|-|ECONOMY BURNISHED TIN|-|23|-|4 +Brand#54|-|ECONOMY PLATED TIN|-|23|-|4 +Brand#54|-|ECONOMY POLISHED BRASS|-|9|-|4 +Brand#54|-|ECONOMY POLISHED BRASS|-|19|-|4 +Brand#54|-|ECONOMY POLISHED COPPER|-|23|-|4 +Brand#54|-|ECONOMY POLISHED STEEL|-|23|-|4 +Brand#54|-|ECONOMY POLISHED TIN|-|3|-|4 +Brand#54|-|LARGE ANODIZED BRASS|-|14|-|4 +Brand#54|-|LARGE ANODIZED BRASS|-|49|-|4 +Brand#54|-|LARGE ANODIZED TIN|-|9|-|4 +Brand#54|-|LARGE BRUSHED BRASS|-|14|-|4 +Brand#54|-|LARGE BRUSHED STEEL|-|9|-|4 +Brand#54|-|LARGE BRUSHED STEEL|-|23|-|4 +Brand#54|-|LARGE BRUSHED TIN|-|14|-|4 +Brand#54|-|LARGE BURNISHED BRASS|-|49|-|4 +Brand#54|-|LARGE BURNISHED COPPER|-|19|-|4 +Brand#54|-|LARGE BURNISHED NICKEL|-|14|-|4 +Brand#54|-|LARGE BURNISHED TIN|-|14|-|4 +Brand#54|-|LARGE PLATED BRASS|-|19|-|4 +Brand#54|-|LARGE PLATED BRASS|-|23|-|4 +Brand#54|-|LARGE POLISHED BRASS|-|19|-|4 +Brand#54|-|LARGE POLISHED BRASS|-|23|-|4 +Brand#54|-|LARGE POLISHED NICKEL|-|3|-|4 +Brand#54|-|LARGE POLISHED NICKEL|-|14|-|4 +Brand#54|-|LARGE POLISHED STEEL|-|19|-|4 +Brand#54|-|LARGE POLISHED TIN|-|3|-|4 +Brand#54|-|LARGE POLISHED TIN|-|9|-|4 +Brand#54|-|LARGE POLISHED TIN|-|36|-|4 +Brand#54|-|MEDIUM ANODIZED NICKEL|-|9|-|4 +Brand#54|-|MEDIUM ANODIZED NICKEL|-|14|-|4 +Brand#54|-|MEDIUM ANODIZED NICKEL|-|36|-|4 +Brand#54|-|MEDIUM BRUSHED NICKEL|-|9|-|4 +Brand#54|-|MEDIUM BRUSHED NICKEL|-|19|-|4 +Brand#54|-|MEDIUM BURNISHED STEEL|-|3|-|4 +Brand#54|-|MEDIUM BURNISHED STEEL|-|19|-|4 +Brand#54|-|MEDIUM BURNISHED STEEL|-|23|-|4 +Brand#54|-|MEDIUM PLATED BRASS|-|3|-|4 +Brand#54|-|MEDIUM PLATED NICKEL|-|45|-|4 +Brand#54|-|PROMO ANODIZED NICKEL|-|45|-|4 +Brand#54|-|PROMO BRUSHED BRASS|-|3|-|4 +Brand#54|-|PROMO BRUSHED STEEL|-|23|-|4 +Brand#54|-|PROMO BRUSHED TIN|-|14|-|4 +Brand#54|-|PROMO BURNISHED COPPER|-|49|-|4 +Brand#54|-|PROMO BURNISHED TIN|-|9|-|4 +Brand#54|-|PROMO PLATED BRASS|-|14|-|4 +Brand#54|-|PROMO PLATED NICKEL|-|3|-|4 +Brand#54|-|PROMO PLATED STEEL|-|19|-|4 +Brand#54|-|PROMO PLATED TIN|-|23|-|4 +Brand#54|-|PROMO PLATED TIN|-|49|-|4 +Brand#54|-|PROMO POLISHED BRASS|-|3|-|4 +Brand#54|-|PROMO POLISHED NICKEL|-|9|-|4 +Brand#54|-|PROMO POLISHED TIN|-|49|-|4 +Brand#54|-|SMALL ANODIZED COPPER|-|49|-|4 +Brand#54|-|SMALL ANODIZED NICKEL|-|9|-|4 +Brand#54|-|SMALL ANODIZED NICKEL|-|36|-|4 +Brand#54|-|SMALL ANODIZED TIN|-|19|-|4 +Brand#54|-|SMALL BRUSHED BRASS|-|14|-|4 +Brand#54|-|SMALL BRUSHED BRASS|-|19|-|4 +Brand#54|-|SMALL BRUSHED BRASS|-|36|-|4 +Brand#54|-|SMALL BRUSHED COPPER|-|3|-|4 +Brand#54|-|SMALL BRUSHED COPPER|-|9|-|4 +Brand#54|-|SMALL BRUSHED COPPER|-|19|-|4 +Brand#54|-|SMALL BRUSHED TIN|-|9|-|4 +Brand#54|-|SMALL BRUSHED TIN|-|36|-|4 +Brand#54|-|SMALL BURNISHED COPPER|-|9|-|4 +Brand#54|-|SMALL BURNISHED COPPER|-|36|-|4 +Brand#54|-|SMALL BURNISHED STEEL|-|14|-|4 +Brand#54|-|SMALL BURNISHED STEEL|-|19|-|4 +Brand#54|-|SMALL BURNISHED TIN|-|9|-|4 +Brand#54|-|SMALL BURNISHED TIN|-|36|-|4 +Brand#54|-|SMALL PLATED BRASS|-|23|-|4 +Brand#54|-|SMALL PLATED COPPER|-|9|-|4 +Brand#54|-|SMALL PLATED COPPER|-|36|-|4 +Brand#54|-|SMALL PLATED COPPER|-|49|-|4 +Brand#54|-|SMALL PLATED NICKEL|-|9|-|4 +Brand#54|-|SMALL PLATED TIN|-|23|-|4 +Brand#54|-|SMALL PLATED TIN|-|36|-|4 +Brand#54|-|SMALL POLISHED BRASS|-|9|-|4 +Brand#54|-|SMALL POLISHED COPPER|-|9|-|4 +Brand#54|-|SMALL POLISHED TIN|-|9|-|4 +Brand#54|-|STANDARD ANODIZED BRASS|-|3|-|4 +Brand#54|-|STANDARD ANODIZED BRASS|-|9|-|4 +Brand#54|-|STANDARD ANODIZED COPPER|-|3|-|4 +Brand#54|-|STANDARD ANODIZED TIN|-|3|-|4 +Brand#54|-|STANDARD BRUSHED COPPER|-|3|-|4 +Brand#54|-|STANDARD BRUSHED NICKEL|-|45|-|4 +Brand#54|-|STANDARD BRUSHED TIN|-|36|-|4 +Brand#54|-|STANDARD BURNISHED BRASS|-|23|-|4 +Brand#54|-|STANDARD BURNISHED BRASS|-|49|-|4 +Brand#54|-|STANDARD BURNISHED COPPER|-|19|-|4 +Brand#54|-|STANDARD BURNISHED NICKEL|-|23|-|4 +Brand#54|-|STANDARD BURNISHED STEEL|-|45|-|4 +Brand#54|-|STANDARD PLATED BRASS|-|3|-|4 +Brand#54|-|STANDARD PLATED BRASS|-|45|-|4 +Brand#54|-|STANDARD PLATED BRASS|-|49|-|4 +Brand#54|-|STANDARD PLATED STEEL|-|3|-|4 +Brand#54|-|STANDARD POLISHED BRASS|-|36|-|4 +Brand#54|-|STANDARD POLISHED STEEL|-|3|-|4 +Brand#54|-|STANDARD POLISHED STEEL|-|14|-|4 +Brand#54|-|STANDARD POLISHED STEEL|-|45|-|4 +Brand#55|-|ECONOMY ANODIZED BRASS|-|3|-|4 +Brand#55|-|ECONOMY BRUSHED BRASS|-|19|-|4 +Brand#55|-|ECONOMY BRUSHED COPPER|-|9|-|4 +Brand#55|-|ECONOMY BRUSHED COPPER|-|23|-|4 +Brand#55|-|ECONOMY BRUSHED COPPER|-|45|-|4 +Brand#55|-|ECONOMY BRUSHED STEEL|-|23|-|4 +Brand#55|-|ECONOMY BURNISHED NICKEL|-|36|-|4 +Brand#55|-|ECONOMY BURNISHED NICKEL|-|45|-|4 +Brand#55|-|ECONOMY BURNISHED TIN|-|45|-|4 +Brand#55|-|ECONOMY PLATED NICKEL|-|19|-|4 +Brand#55|-|ECONOMY POLISHED NICKEL|-|9|-|4 +Brand#55|-|LARGE BRUSHED BRASS|-|23|-|4 +Brand#55|-|LARGE BRUSHED BRASS|-|45|-|4 +Brand#55|-|LARGE BRUSHED COPPER|-|49|-|4 +Brand#55|-|LARGE BRUSHED NICKEL|-|9|-|4 +Brand#55|-|LARGE BRUSHED NICKEL|-|14|-|4 +Brand#55|-|LARGE BURNISHED BRASS|-|3|-|4 +Brand#55|-|LARGE BURNISHED COPPER|-|14|-|4 +Brand#55|-|LARGE BURNISHED COPPER|-|36|-|4 +Brand#55|-|LARGE PLATED BRASS|-|45|-|4 +Brand#55|-|LARGE PLATED COPPER|-|19|-|4 +Brand#55|-|LARGE PLATED NICKEL|-|9|-|4 +Brand#55|-|LARGE PLATED STEEL|-|9|-|4 +Brand#55|-|LARGE PLATED TIN|-|9|-|4 +Brand#55|-|LARGE PLATED TIN|-|14|-|4 +Brand#55|-|LARGE PLATED TIN|-|23|-|4 +Brand#55|-|LARGE POLISHED NICKEL|-|3|-|4 +Brand#55|-|LARGE POLISHED STEEL|-|36|-|4 +Brand#55|-|LARGE POLISHED STEEL|-|45|-|4 +Brand#55|-|MEDIUM ANODIZED COPPER|-|9|-|4 +Brand#55|-|MEDIUM BRUSHED BRASS|-|3|-|4 +Brand#55|-|MEDIUM BRUSHED NICKEL|-|23|-|4 +Brand#55|-|MEDIUM BRUSHED TIN|-|45|-|4 +Brand#55|-|MEDIUM BURNISHED BRASS|-|23|-|4 +Brand#55|-|MEDIUM BURNISHED COPPER|-|36|-|4 +Brand#55|-|MEDIUM BURNISHED NICKEL|-|3|-|4 +Brand#55|-|MEDIUM BURNISHED STEEL|-|14|-|4 +Brand#55|-|MEDIUM BURNISHED STEEL|-|36|-|4 +Brand#55|-|MEDIUM PLATED NICKEL|-|23|-|4 +Brand#55|-|PROMO ANODIZED COPPER|-|14|-|4 +Brand#55|-|PROMO ANODIZED COPPER|-|49|-|4 +Brand#55|-|PROMO ANODIZED STEEL|-|36|-|4 +Brand#55|-|PROMO ANODIZED TIN|-|23|-|4 +Brand#55|-|PROMO BRUSHED NICKEL|-|36|-|4 +Brand#55|-|PROMO BRUSHED STEEL|-|3|-|4 +Brand#55|-|PROMO BRUSHED STEEL|-|36|-|4 +Brand#55|-|PROMO BRUSHED TIN|-|9|-|4 +Brand#55|-|PROMO BURNISHED COPPER|-|3|-|4 +Brand#55|-|PROMO BURNISHED STEEL|-|14|-|4 +Brand#55|-|PROMO BURNISHED TIN|-|23|-|4 +Brand#55|-|PROMO BURNISHED TIN|-|49|-|4 +Brand#55|-|PROMO PLATED COPPER|-|3|-|4 +Brand#55|-|PROMO PLATED NICKEL|-|3|-|4 +Brand#55|-|PROMO PLATED NICKEL|-|14|-|4 +Brand#55|-|PROMO PLATED NICKEL|-|23|-|4 +Brand#55|-|PROMO PLATED TIN|-|3|-|4 +Brand#55|-|PROMO POLISHED COPPER|-|3|-|4 +Brand#55|-|SMALL ANODIZED BRASS|-|19|-|4 +Brand#55|-|SMALL ANODIZED NICKEL|-|45|-|4 +Brand#55|-|SMALL BRUSHED COPPER|-|14|-|4 +Brand#55|-|SMALL BRUSHED COPPER|-|45|-|4 +Brand#55|-|SMALL BURNISHED BRASS|-|14|-|4 +Brand#55|-|SMALL BURNISHED TIN|-|3|-|4 +Brand#55|-|SMALL BURNISHED TIN|-|49|-|4 +Brand#55|-|SMALL PLATED BRASS|-|45|-|4 +Brand#55|-|SMALL PLATED COPPER|-|23|-|4 +Brand#55|-|SMALL PLATED COPPER|-|36|-|4 +Brand#55|-|SMALL PLATED COPPER|-|45|-|4 +Brand#55|-|SMALL PLATED COPPER|-|49|-|4 +Brand#55|-|SMALL PLATED NICKEL|-|9|-|4 +Brand#55|-|SMALL PLATED STEEL|-|9|-|4 +Brand#55|-|SMALL PLATED TIN|-|14|-|4 +Brand#55|-|SMALL PLATED TIN|-|36|-|4 +Brand#55|-|SMALL POLISHED NICKEL|-|45|-|4 +Brand#55|-|SMALL POLISHED STEEL|-|19|-|4 +Brand#55|-|SMALL POLISHED TIN|-|19|-|4 +Brand#55|-|STANDARD ANODIZED BRASS|-|36|-|4 +Brand#55|-|STANDARD ANODIZED BRASS|-|49|-|4 +Brand#55|-|STANDARD ANODIZED STEEL|-|19|-|4 +Brand#55|-|STANDARD ANODIZED TIN|-|36|-|4 +Brand#55|-|STANDARD ANODIZED TIN|-|49|-|4 +Brand#55|-|STANDARD BRUSHED BRASS|-|36|-|4 +Brand#55|-|STANDARD BRUSHED COPPER|-|3|-|4 +Brand#55|-|STANDARD BRUSHED COPPER|-|9|-|4 +Brand#55|-|STANDARD BRUSHED COPPER|-|23|-|4 +Brand#55|-|STANDARD BRUSHED STEEL|-|19|-|4 +Brand#55|-|STANDARD BRUSHED TIN|-|23|-|4 +Brand#55|-|STANDARD BRUSHED TIN|-|45|-|4 +Brand#55|-|STANDARD BURNISHED BRASS|-|19|-|4 +Brand#55|-|STANDARD BURNISHED NICKEL|-|3|-|4 +Brand#55|-|STANDARD BURNISHED NICKEL|-|36|-|4 +Brand#55|-|STANDARD BURNISHED STEEL|-|19|-|4 +Brand#55|-|STANDARD PLATED BRASS|-|23|-|4 +Brand#55|-|STANDARD PLATED NICKEL|-|9|-|4 +Brand#55|-|STANDARD PLATED TIN|-|36|-|4 +Brand#55|-|STANDARD POLISHED BRASS|-|3|-|4 +Brand#55|-|STANDARD POLISHED BRASS|-|49|-|4 +Brand#55|-|STANDARD POLISHED COPPER|-|19|-|4 +Brand#55|-|STANDARD POLISHED COPPER|-|36|-|4 +Brand#55|-|STANDARD POLISHED NICKEL|-|14|-|4 +Brand#55|-|STANDARD POLISHED STEEL|-|9|-|4 +Brand#55|-|STANDARD POLISHED STEEL|-|36|-|4 +Brand#12|-|LARGE BURNISHED NICKEL|-|14|-|3 +Brand#12|-|PROMO POLISHED TIN|-|3|-|3 +Brand#21|-|MEDIUM ANODIZED TIN|-|9|-|3 +Brand#22|-|PROMO BRUSHED BRASS|-|19|-|3 +Brand#22|-|PROMO BURNISHED COPPER|-|14|-|3 +Brand#43|-|STANDARD BRUSHED BRASS|-|23|-|3 +Brand#44|-|MEDIUM ANODIZED NICKEL|-|9|-|3 +Brand#53|-|MEDIUM BURNISHED BRASS|-|49|-|3 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q17.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q17.out new file mode 100644 index 0000000000000..0f4a8f8f2c42d --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q17.out @@ -0,0 +1,2 @@ +1 +23512.75285714285 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q18.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q18.out new file mode 100644 index 0000000000000..9611834608d13 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q18.out @@ -0,0 +1,6 @@ +5 +Customer#000001639|-|1639|-|502886|-|1994-04-12|-|456423.88|-|312.0 +Customer#000006655|-|6655|-|29158|-|1995-10-21|-|452805.02|-|305.0 +Customer#000014110|-|14110|-|565574|-|1995-09-24|-|425099.85|-|301.0 +Customer#000001775|-|1775|-|6882|-|1997-04-09|-|408368.1|-|303.0 +Customer#000011459|-|11459|-|551136|-|1993-05-19|-|386812.74|-|308.0 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q19.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q19.out new file mode 100644 index 0000000000000..9945c5826c048 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q19.out @@ -0,0 +1,2 @@ +1 +168597.286 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q20.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q20.out new file mode 100644 index 0000000000000..98880fb6599b2 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q20.out @@ -0,0 +1,10 @@ +9 +Supplier#000000157|-|,mEGorBfVIm +Supplier#000000197|-|YC2Acon6kjY3zj3Fbxs2k4Vdf7X0cd2F +Supplier#000000287|-|7a9SP7qW5Yku5PvSg +Supplier#000000378|-|FfbhyCxWvcPrO8ltp9 +Supplier#000000530|-|0qwCMwobKY OcmLyfRXlagA8ukENJv, +Supplier#000000555|-|TfB,a5bfl3Ah 3Z 74GqnNs6zKVGM +Supplier#000000557|-|jj0wUYh9K3fG5Jhdhrkuy ,4 +Supplier#000000729|-|pqck2ppy758TQpZCUAjPvlU55K3QjfL7Bi +Supplier#000000935|-|ij98czM 2KzWe7dDTOxB8sq0UfCdvrX diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q21.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q21.out new file mode 100644 index 0000000000000..c4342764a7c07 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q21.out @@ -0,0 +1,48 @@ +47 +Supplier#000000445|-|16 +Supplier#000000825|-|16 +Supplier#000000709|-|15 +Supplier#000000762|-|15 +Supplier#000000357|-|14 +Supplier#000000399|-|14 +Supplier#000000496|-|14 +Supplier#000000977|-|13 +Supplier#000000144|-|12 +Supplier#000000188|-|12 +Supplier#000000415|-|12 +Supplier#000000472|-|12 +Supplier#000000633|-|12 +Supplier#000000708|-|12 +Supplier#000000889|-|12 +Supplier#000000380|-|11 +Supplier#000000602|-|11 +Supplier#000000659|-|11 +Supplier#000000821|-|11 +Supplier#000000929|-|11 +Supplier#000000262|-|10 +Supplier#000000460|-|10 +Supplier#000000486|-|10 +Supplier#000000669|-|10 +Supplier#000000718|-|10 +Supplier#000000778|-|10 +Supplier#000000167|-|9 +Supplier#000000578|-|9 +Supplier#000000673|-|9 +Supplier#000000687|-|9 +Supplier#000000074|-|8 +Supplier#000000565|-|8 +Supplier#000000648|-|8 +Supplier#000000918|-|8 +Supplier#000000427|-|7 +Supplier#000000503|-|7 +Supplier#000000610|-|7 +Supplier#000000670|-|7 +Supplier#000000811|-|7 +Supplier#000000114|-|6 +Supplier#000000379|-|6 +Supplier#000000436|-|6 +Supplier#000000500|-|6 +Supplier#000000660|-|6 +Supplier#000000788|-|6 +Supplier#000000846|-|6 +Supplier#000000920|-|4 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q22.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q22.out new file mode 100644 index 0000000000000..71db28424e23a --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q22.out @@ -0,0 +1,8 @@ +7 +13|-|94|-|714035.0499999999 +17|-|96|-|722560.15 +18|-|99|-|738012.5200000004 +23|-|93|-|708285.25 +29|-|85|-|632693.46 +30|-|87|-|646748.0199999998 +31|-|87|-|647372.5000000001 diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/checksums.txt new file mode 100644 index 0000000000000..c24cc7adbbe97 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/columns.txt new file mode 100644 index 0000000000000..2142690808982 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/columns.txt @@ -0,0 +1,10 @@ +columns format version: 1 +8 columns: +`c_custkey` Int64 +`c_name` String +`c_address` String +`c_nationkey` Int64 +`c_phone` String +`c_acctbal` Float64 +`c_mktsegment` String +`c_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/count.txt new file mode 100644 index 0000000000000..40ab85b35d5b1 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/count.txt @@ -0,0 +1 @@ +3682 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/data.bin new file mode 100644 index 0000000000000..9315c89fde562 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/data.cmrk3 new file mode 100644 index 0000000000000..f93421cb80c12 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/default_compression_codec.txt new file mode 100644 index 0000000000000..061d1280b897b --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/metadata_version.txt new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/primary.cidx new file mode 100644 index 0000000000000..9b9567e44cded Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/serialization.json new file mode 100644 index 0000000000000..be6ce5eae1f47 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"c_acctbal","num_defaults":0,"num_rows":3682},{"kind":"Default","name":"c_address","num_defaults":0,"num_rows":3682},{"kind":"Default","name":"c_comment","num_defaults":0,"num_rows":3682},{"kind":"Default","name":"c_custkey","num_defaults":0,"num_rows":3682},{"kind":"Default","name":"c_mktsegment","num_defaults":0,"num_rows":3682},{"kind":"Default","name":"c_name","num_defaults":0,"num_rows":3682},{"kind":"Default","name":"c_nationkey","num_defaults":100,"num_rows":3682},{"kind":"Default","name":"c_phone","num_defaults":0,"num_rows":3682}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/checksums.txt new file mode 100644 index 0000000000000..fab53631b09c5 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/columns.txt new file mode 100644 index 0000000000000..2142690808982 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/columns.txt @@ -0,0 +1,10 @@ +columns format version: 1 +8 columns: +`c_custkey` Int64 +`c_name` String +`c_address` String +`c_nationkey` Int64 +`c_phone` String +`c_acctbal` Float64 +`c_mktsegment` String +`c_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/count.txt new file mode 100644 index 0000000000000..7f6931a12e811 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/count.txt @@ -0,0 +1 @@ +3698 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/data.bin new file mode 100644 index 0000000000000..959aea1035108 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/data.cmrk3 new file mode 100644 index 0000000000000..2d08173f6da05 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/default_compression_codec.txt new file mode 100644 index 0000000000000..061d1280b897b --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/metadata_version.txt new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/primary.cidx new file mode 100644 index 0000000000000..32fdc6a6f39d5 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/serialization.json new file mode 100644 index 0000000000000..43e0a379e0662 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"c_acctbal","num_defaults":0,"num_rows":3698},{"kind":"Default","name":"c_address","num_defaults":0,"num_rows":3698},{"kind":"Default","name":"c_comment","num_defaults":0,"num_rows":3698},{"kind":"Default","name":"c_custkey","num_defaults":0,"num_rows":3698},{"kind":"Default","name":"c_mktsegment","num_defaults":0,"num_rows":3698},{"kind":"Default","name":"c_name","num_defaults":0,"num_rows":3698},{"kind":"Default","name":"c_nationkey","num_defaults":160,"num_rows":3698},{"kind":"Default","name":"c_phone","num_defaults":0,"num_rows":3698}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/checksums.txt new file mode 100644 index 0000000000000..3be296ce3788e Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/columns.txt new file mode 100644 index 0000000000000..2142690808982 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/columns.txt @@ -0,0 +1,10 @@ +columns format version: 1 +8 columns: +`c_custkey` Int64 +`c_name` String +`c_address` String +`c_nationkey` Int64 +`c_phone` String +`c_acctbal` Float64 +`c_mktsegment` String +`c_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/count.txt new file mode 100644 index 0000000000000..3b93cf65b897d --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/count.txt @@ -0,0 +1 @@ +3818 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/data.bin new file mode 100644 index 0000000000000..f97d3c269c908 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/data.cmrk3 new file mode 100644 index 0000000000000..b1139c1af625a Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/default_compression_codec.txt new file mode 100644 index 0000000000000..061d1280b897b --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/metadata_version.txt new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/primary.cidx new file mode 100644 index 0000000000000..b5fbc294b66a9 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/serialization.json new file mode 100644 index 0000000000000..a1e3accc5dd39 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"c_acctbal","num_defaults":0,"num_rows":3818},{"kind":"Default","name":"c_address","num_defaults":0,"num_rows":3818},{"kind":"Default","name":"c_comment","num_defaults":0,"num_rows":3818},{"kind":"Default","name":"c_custkey","num_defaults":0,"num_rows":3818},{"kind":"Default","name":"c_mktsegment","num_defaults":0,"num_rows":3818},{"kind":"Default","name":"c_name","num_defaults":0,"num_rows":3818},{"kind":"Default","name":"c_nationkey","num_defaults":110,"num_rows":3818},{"kind":"Default","name":"c_phone","num_defaults":0,"num_rows":3818}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/checksums.txt new file mode 100644 index 0000000000000..2255a1064afec Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/columns.txt new file mode 100644 index 0000000000000..2142690808982 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/columns.txt @@ -0,0 +1,10 @@ +columns format version: 1 +8 columns: +`c_custkey` Int64 +`c_name` String +`c_address` String +`c_nationkey` Int64 +`c_phone` String +`c_acctbal` Float64 +`c_mktsegment` String +`c_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/count.txt new file mode 100644 index 0000000000000..ca26457e9e825 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/count.txt @@ -0,0 +1 @@ +3802 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/data.bin new file mode 100644 index 0000000000000..07f2a8b2bf9a6 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/data.cmrk3 new file mode 100644 index 0000000000000..719484322a5c1 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/default_compression_codec.txt new file mode 100644 index 0000000000000..061d1280b897b --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/metadata_version.txt new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/primary.cidx new file mode 100644 index 0000000000000..0beb0a8885477 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/serialization.json new file mode 100644 index 0000000000000..793486590fbe9 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"c_acctbal","num_defaults":0,"num_rows":3802},{"kind":"Default","name":"c_address","num_defaults":0,"num_rows":3802},{"kind":"Default","name":"c_comment","num_defaults":0,"num_rows":3802},{"kind":"Default","name":"c_custkey","num_defaults":0,"num_rows":3802},{"kind":"Default","name":"c_mktsegment","num_defaults":0,"num_rows":3802},{"kind":"Default","name":"c_name","num_defaults":0,"num_rows":3802},{"kind":"Default","name":"c_nationkey","num_defaults":140,"num_rows":3802},{"kind":"Default","name":"c_phone","num_defaults":0,"num_rows":3802}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/checksums.txt new file mode 100644 index 0000000000000..e5419574780a6 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/columns.txt new file mode 100644 index 0000000000000..598d676660f86 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/columns.txt @@ -0,0 +1,18 @@ +columns format version: 1 +16 columns: +`l_orderkey` Int64 +`l_partkey` Int64 +`l_suppkey` Int64 +`l_linenumber` Int64 +`l_quantity` Float64 +`l_extendedprice` Float64 +`l_discount` Float64 +`l_tax` Float64 +`l_returnflag` String +`l_linestatus` String +`l_shipdate` Date +`l_commitdate` Date +`l_receiptdate` Date +`l_shipinstruct` String +`l_shipmode` String +`l_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/count.txt new file mode 100644 index 0000000000000..46989323742cc --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/count.txt @@ -0,0 +1 @@ +150183 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/default_compression_codec.txt new file mode 100644 index 0000000000000..061d1280b897b --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_comment.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_comment.bin new file mode 100644 index 0000000000000..f8d3e600c5b14 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_comment.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_comment.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_comment.cmrk2 new file mode 100644 index 0000000000000..41304769fc267 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_comment.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_commitdate.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_commitdate.bin new file mode 100644 index 0000000000000..649a72b078ef2 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_commitdate.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_commitdate.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_commitdate.cmrk2 new file mode 100644 index 0000000000000..63002694dcafe Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_commitdate.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_discount.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_discount.bin new file mode 100644 index 0000000000000..5c5665e66302d Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_discount.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_discount.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_discount.cmrk2 new file mode 100644 index 0000000000000..fac91d4bc95de Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_discount.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_extendedprice.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_extendedprice.bin new file mode 100644 index 0000000000000..cadd769922638 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_extendedprice.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_extendedprice.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_extendedprice.cmrk2 new file mode 100644 index 0000000000000..3a41b31a55fb7 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_extendedprice.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_linenumber.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_linenumber.bin new file mode 100644 index 0000000000000..2fd4cc9e30bb6 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_linenumber.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_linenumber.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_linenumber.cmrk2 new file mode 100644 index 0000000000000..cadff0655a59e Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_linenumber.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_linestatus.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_linestatus.bin new file mode 100644 index 0000000000000..9e14e21014984 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_linestatus.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_linestatus.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_linestatus.cmrk2 new file mode 100644 index 0000000000000..7d8576becc949 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_linestatus.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_orderkey.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_orderkey.bin new file mode 100644 index 0000000000000..e98174a2110da Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_orderkey.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_orderkey.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_orderkey.cmrk2 new file mode 100644 index 0000000000000..e2125dbac0845 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_orderkey.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_partkey.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_partkey.bin new file mode 100644 index 0000000000000..e843bafaa032b Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_partkey.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_partkey.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_partkey.cmrk2 new file mode 100644 index 0000000000000..50d265966bd9c Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_partkey.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_quantity.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_quantity.bin new file mode 100644 index 0000000000000..2454f1890eed4 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_quantity.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_quantity.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_quantity.cmrk2 new file mode 100644 index 0000000000000..1d6c26cf63e97 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_quantity.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_receiptdate.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_receiptdate.bin new file mode 100644 index 0000000000000..c70b1086226e1 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_receiptdate.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_receiptdate.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_receiptdate.cmrk2 new file mode 100644 index 0000000000000..04831b24563df Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_receiptdate.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_returnflag.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_returnflag.bin new file mode 100644 index 0000000000000..5e31a53b6c6b8 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_returnflag.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_returnflag.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_returnflag.cmrk2 new file mode 100644 index 0000000000000..e53049390cddc Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_returnflag.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipdate.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipdate.bin new file mode 100644 index 0000000000000..c68ba1fcf3f1c Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipdate.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipdate.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipdate.cmrk2 new file mode 100644 index 0000000000000..34ce9f84741cb Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipdate.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipinstruct.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipinstruct.bin new file mode 100644 index 0000000000000..8755eeace690d Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipinstruct.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipinstruct.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipinstruct.cmrk2 new file mode 100644 index 0000000000000..e1bd489948d71 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipinstruct.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipmode.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipmode.bin new file mode 100644 index 0000000000000..2331c3b411008 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipmode.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipmode.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipmode.cmrk2 new file mode 100644 index 0000000000000..31994007f9017 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipmode.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_suppkey.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_suppkey.bin new file mode 100644 index 0000000000000..65b450e124fee Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_suppkey.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_suppkey.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_suppkey.cmrk2 new file mode 100644 index 0000000000000..9316b287e25e2 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_suppkey.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_tax.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_tax.bin new file mode 100644 index 0000000000000..c1e26f0ad31b7 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_tax.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_tax.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_tax.cmrk2 new file mode 100644 index 0000000000000..40e53e166e62f Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_tax.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/metadata_version.txt new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/primary.cidx new file mode 100644 index 0000000000000..d76cc0286dfa9 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/serialization.json new file mode 100644 index 0000000000000..51d9649342121 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"l_comment","num_defaults":0,"num_rows":150183},{"kind":"Default","name":"l_commitdate","num_defaults":0,"num_rows":150183},{"kind":"Default","name":"l_discount","num_defaults":13540,"num_rows":150183},{"kind":"Default","name":"l_extendedprice","num_defaults":0,"num_rows":150183},{"kind":"Default","name":"l_linenumber","num_defaults":0,"num_rows":150183},{"kind":"Default","name":"l_linestatus","num_defaults":0,"num_rows":150183},{"kind":"Default","name":"l_orderkey","num_defaults":0,"num_rows":150183},{"kind":"Default","name":"l_partkey","num_defaults":0,"num_rows":150183},{"kind":"Default","name":"l_quantity","num_defaults":0,"num_rows":150183},{"kind":"Default","name":"l_receiptdate","num_defaults":0,"num_rows":150183},{"kind":"Default","name":"l_returnflag","num_defaults":0,"num_rows":150183},{"kind":"Default","name":"l_shipdate","num_defaults":0,"num_rows":150183},{"kind":"Default","name":"l_shipinstruct","num_defaults":0,"num_rows":150183},{"kind":"Default","name":"l_shipmode","num_defaults":0,"num_rows":150183},{"kind":"Default","name":"l_suppkey","num_defaults":0,"num_rows":150183},{"kind":"Default","name":"l_tax","num_defaults":16500,"num_rows":150183}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/checksums.txt new file mode 100644 index 0000000000000..fe6bdd4eb2b44 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/columns.txt new file mode 100644 index 0000000000000..598d676660f86 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/columns.txt @@ -0,0 +1,18 @@ +columns format version: 1 +16 columns: +`l_orderkey` Int64 +`l_partkey` Int64 +`l_suppkey` Int64 +`l_linenumber` Int64 +`l_quantity` Float64 +`l_extendedprice` Float64 +`l_discount` Float64 +`l_tax` Float64 +`l_returnflag` String +`l_linestatus` String +`l_shipdate` Date +`l_commitdate` Date +`l_receiptdate` Date +`l_shipinstruct` String +`l_shipmode` String +`l_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/count.txt new file mode 100644 index 0000000000000..e33b55453bec2 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/count.txt @@ -0,0 +1 @@ +150025 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/default_compression_codec.txt new file mode 100644 index 0000000000000..061d1280b897b --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_comment.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_comment.bin new file mode 100644 index 0000000000000..9466f115e62f8 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_comment.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_comment.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_comment.cmrk2 new file mode 100644 index 0000000000000..ffc7984d73602 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_comment.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_commitdate.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_commitdate.bin new file mode 100644 index 0000000000000..9168d376ff1d5 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_commitdate.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_commitdate.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_commitdate.cmrk2 new file mode 100644 index 0000000000000..bf4449a673b4f Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_commitdate.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_discount.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_discount.bin new file mode 100644 index 0000000000000..62db4b48f9352 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_discount.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_discount.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_discount.cmrk2 new file mode 100644 index 0000000000000..98d67588623fc Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_discount.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_extendedprice.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_extendedprice.bin new file mode 100644 index 0000000000000..5f601760effaf Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_extendedprice.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_extendedprice.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_extendedprice.cmrk2 new file mode 100644 index 0000000000000..370a22b155b00 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_extendedprice.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_linenumber.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_linenumber.bin new file mode 100644 index 0000000000000..436a540493109 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_linenumber.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_linenumber.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_linenumber.cmrk2 new file mode 100644 index 0000000000000..e96b24a0d6cad Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_linenumber.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_linestatus.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_linestatus.bin new file mode 100644 index 0000000000000..e3d926af2a9f3 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_linestatus.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_linestatus.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_linestatus.cmrk2 new file mode 100644 index 0000000000000..4a6b1f63a8ce6 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_linestatus.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_orderkey.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_orderkey.bin new file mode 100644 index 0000000000000..01c90d9fe3e38 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_orderkey.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_orderkey.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_orderkey.cmrk2 new file mode 100644 index 0000000000000..f40a82e3481eb Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_orderkey.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_partkey.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_partkey.bin new file mode 100644 index 0000000000000..7fb3c626269a5 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_partkey.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_partkey.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_partkey.cmrk2 new file mode 100644 index 0000000000000..ec0641f72db98 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_partkey.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_quantity.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_quantity.bin new file mode 100644 index 0000000000000..e29f5881ef5a8 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_quantity.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_quantity.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_quantity.cmrk2 new file mode 100644 index 0000000000000..741b0f65fda00 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_quantity.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_receiptdate.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_receiptdate.bin new file mode 100644 index 0000000000000..17eb397a05936 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_receiptdate.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_receiptdate.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_receiptdate.cmrk2 new file mode 100644 index 0000000000000..7ab1d7e75a0a4 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_receiptdate.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_returnflag.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_returnflag.bin new file mode 100644 index 0000000000000..ff83da4c983f8 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_returnflag.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_returnflag.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_returnflag.cmrk2 new file mode 100644 index 0000000000000..aeca561e30c46 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_returnflag.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipdate.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipdate.bin new file mode 100644 index 0000000000000..f4d821d808f20 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipdate.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipdate.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipdate.cmrk2 new file mode 100644 index 0000000000000..af5d5b54f42bc Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipdate.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipinstruct.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipinstruct.bin new file mode 100644 index 0000000000000..448c814df204f Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipinstruct.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipinstruct.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipinstruct.cmrk2 new file mode 100644 index 0000000000000..fd2843cf2faa0 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipinstruct.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipmode.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipmode.bin new file mode 100644 index 0000000000000..5d4b26a7bc82b Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipmode.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipmode.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipmode.cmrk2 new file mode 100644 index 0000000000000..f60f9da6f5928 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipmode.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_suppkey.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_suppkey.bin new file mode 100644 index 0000000000000..6532245dc1c97 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_suppkey.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_suppkey.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_suppkey.cmrk2 new file mode 100644 index 0000000000000..c494259dfe421 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_suppkey.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_tax.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_tax.bin new file mode 100644 index 0000000000000..6e835238a5455 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_tax.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_tax.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_tax.cmrk2 new file mode 100644 index 0000000000000..d7a2cf92aac68 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_tax.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/metadata_version.txt new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/primary.cidx new file mode 100644 index 0000000000000..f577026504b25 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/serialization.json new file mode 100644 index 0000000000000..4b8c99383d9f6 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"l_comment","num_defaults":0,"num_rows":150025},{"kind":"Default","name":"l_commitdate","num_defaults":0,"num_rows":150025},{"kind":"Default","name":"l_discount","num_defaults":13620,"num_rows":150025},{"kind":"Default","name":"l_extendedprice","num_defaults":0,"num_rows":150025},{"kind":"Default","name":"l_linenumber","num_defaults":0,"num_rows":150025},{"kind":"Default","name":"l_linestatus","num_defaults":0,"num_rows":150025},{"kind":"Default","name":"l_orderkey","num_defaults":0,"num_rows":150025},{"kind":"Default","name":"l_partkey","num_defaults":0,"num_rows":150025},{"kind":"Default","name":"l_quantity","num_defaults":0,"num_rows":150025},{"kind":"Default","name":"l_receiptdate","num_defaults":0,"num_rows":150025},{"kind":"Default","name":"l_returnflag","num_defaults":0,"num_rows":150025},{"kind":"Default","name":"l_shipdate","num_defaults":0,"num_rows":150025},{"kind":"Default","name":"l_shipinstruct","num_defaults":0,"num_rows":150025},{"kind":"Default","name":"l_shipmode","num_defaults":0,"num_rows":150025},{"kind":"Default","name":"l_suppkey","num_defaults":0,"num_rows":150025},{"kind":"Default","name":"l_tax","num_defaults":16260,"num_rows":150025}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/checksums.txt new file mode 100644 index 0000000000000..3887c120e65dc Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/columns.txt new file mode 100644 index 0000000000000..598d676660f86 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/columns.txt @@ -0,0 +1,18 @@ +columns format version: 1 +16 columns: +`l_orderkey` Int64 +`l_partkey` Int64 +`l_suppkey` Int64 +`l_linenumber` Int64 +`l_quantity` Float64 +`l_extendedprice` Float64 +`l_discount` Float64 +`l_tax` Float64 +`l_returnflag` String +`l_linestatus` String +`l_shipdate` Date +`l_commitdate` Date +`l_receiptdate` Date +`l_shipinstruct` String +`l_shipmode` String +`l_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/count.txt new file mode 100644 index 0000000000000..45c374d3b6351 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/count.txt @@ -0,0 +1 @@ +150103 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/default_compression_codec.txt new file mode 100644 index 0000000000000..061d1280b897b --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_comment.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_comment.bin new file mode 100644 index 0000000000000..408e74b95e9a3 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_comment.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_comment.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_comment.cmrk2 new file mode 100644 index 0000000000000..421d1eaf1e69b Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_comment.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_commitdate.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_commitdate.bin new file mode 100644 index 0000000000000..2fbd419494916 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_commitdate.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_commitdate.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_commitdate.cmrk2 new file mode 100644 index 0000000000000..8829d7381ed7c Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_commitdate.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_discount.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_discount.bin new file mode 100644 index 0000000000000..75f3fbc03cace Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_discount.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_discount.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_discount.cmrk2 new file mode 100644 index 0000000000000..e37555a4008e3 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_discount.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_extendedprice.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_extendedprice.bin new file mode 100644 index 0000000000000..2780d5f256c07 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_extendedprice.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_extendedprice.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_extendedprice.cmrk2 new file mode 100644 index 0000000000000..03af8943f3cfb Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_extendedprice.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_linenumber.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_linenumber.bin new file mode 100644 index 0000000000000..059048d52bbaf Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_linenumber.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_linenumber.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_linenumber.cmrk2 new file mode 100644 index 0000000000000..fe3742e5d9ca7 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_linenumber.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_linestatus.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_linestatus.bin new file mode 100644 index 0000000000000..f79b740e14255 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_linestatus.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_linestatus.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_linestatus.cmrk2 new file mode 100644 index 0000000000000..98281152d342e Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_linestatus.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_orderkey.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_orderkey.bin new file mode 100644 index 0000000000000..db4603c92bc90 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_orderkey.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_orderkey.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_orderkey.cmrk2 new file mode 100644 index 0000000000000..175401d84327a Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_orderkey.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_partkey.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_partkey.bin new file mode 100644 index 0000000000000..63ab678d94fdb Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_partkey.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_partkey.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_partkey.cmrk2 new file mode 100644 index 0000000000000..703cd3a9106ad Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_partkey.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_quantity.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_quantity.bin new file mode 100644 index 0000000000000..b1a8a224d92ef Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_quantity.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_quantity.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_quantity.cmrk2 new file mode 100644 index 0000000000000..055ca8f0a9baa Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_quantity.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_receiptdate.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_receiptdate.bin new file mode 100644 index 0000000000000..b46766a56b653 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_receiptdate.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_receiptdate.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_receiptdate.cmrk2 new file mode 100644 index 0000000000000..4a065f4bd8a2d Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_receiptdate.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_returnflag.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_returnflag.bin new file mode 100644 index 0000000000000..6db9a44fac28d Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_returnflag.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_returnflag.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_returnflag.cmrk2 new file mode 100644 index 0000000000000..0c1af0fc822b6 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_returnflag.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipdate.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipdate.bin new file mode 100644 index 0000000000000..edf48da70d82f Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipdate.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipdate.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipdate.cmrk2 new file mode 100644 index 0000000000000..9a625ad4ab2e5 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipdate.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipinstruct.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipinstruct.bin new file mode 100644 index 0000000000000..65cac6d9aaa23 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipinstruct.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipinstruct.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipinstruct.cmrk2 new file mode 100644 index 0000000000000..2de907621bb9d Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipinstruct.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipmode.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipmode.bin new file mode 100644 index 0000000000000..85d79cd828a58 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipmode.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipmode.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipmode.cmrk2 new file mode 100644 index 0000000000000..dbd1a9fa07752 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipmode.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_suppkey.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_suppkey.bin new file mode 100644 index 0000000000000..3bd908ef18b8c Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_suppkey.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_suppkey.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_suppkey.cmrk2 new file mode 100644 index 0000000000000..9a9ead3f9362c Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_suppkey.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_tax.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_tax.bin new file mode 100644 index 0000000000000..5ae494b93b721 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_tax.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_tax.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_tax.cmrk2 new file mode 100644 index 0000000000000..024467b188d9e Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_tax.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/metadata_version.txt new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/primary.cidx new file mode 100644 index 0000000000000..45ac79b664354 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/serialization.json new file mode 100644 index 0000000000000..6dcc4458e81d2 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"l_comment","num_defaults":0,"num_rows":150103},{"kind":"Default","name":"l_commitdate","num_defaults":0,"num_rows":150103},{"kind":"Default","name":"l_discount","num_defaults":13330,"num_rows":150103},{"kind":"Default","name":"l_extendedprice","num_defaults":0,"num_rows":150103},{"kind":"Default","name":"l_linenumber","num_defaults":0,"num_rows":150103},{"kind":"Default","name":"l_linestatus","num_defaults":0,"num_rows":150103},{"kind":"Default","name":"l_orderkey","num_defaults":0,"num_rows":150103},{"kind":"Default","name":"l_partkey","num_defaults":0,"num_rows":150103},{"kind":"Default","name":"l_quantity","num_defaults":0,"num_rows":150103},{"kind":"Default","name":"l_receiptdate","num_defaults":0,"num_rows":150103},{"kind":"Default","name":"l_returnflag","num_defaults":0,"num_rows":150103},{"kind":"Default","name":"l_shipdate","num_defaults":0,"num_rows":150103},{"kind":"Default","name":"l_shipinstruct","num_defaults":0,"num_rows":150103},{"kind":"Default","name":"l_shipmode","num_defaults":0,"num_rows":150103},{"kind":"Default","name":"l_suppkey","num_defaults":0,"num_rows":150103},{"kind":"Default","name":"l_tax","num_defaults":16800,"num_rows":150103}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/checksums.txt new file mode 100644 index 0000000000000..c3eafce89b34b Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/columns.txt new file mode 100644 index 0000000000000..598d676660f86 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/columns.txt @@ -0,0 +1,18 @@ +columns format version: 1 +16 columns: +`l_orderkey` Int64 +`l_partkey` Int64 +`l_suppkey` Int64 +`l_linenumber` Int64 +`l_quantity` Float64 +`l_extendedprice` Float64 +`l_discount` Float64 +`l_tax` Float64 +`l_returnflag` String +`l_linestatus` String +`l_shipdate` Date +`l_commitdate` Date +`l_receiptdate` Date +`l_shipinstruct` String +`l_shipmode` String +`l_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/count.txt new file mode 100644 index 0000000000000..86f6ce8ede196 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/count.txt @@ -0,0 +1 @@ +150261 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/default_compression_codec.txt new file mode 100644 index 0000000000000..061d1280b897b --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_comment.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_comment.bin new file mode 100644 index 0000000000000..95f69fd6418e1 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_comment.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_comment.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_comment.cmrk2 new file mode 100644 index 0000000000000..5f4cc6326a91b Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_comment.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_commitdate.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_commitdate.bin new file mode 100644 index 0000000000000..1b1f8fbed1333 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_commitdate.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_commitdate.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_commitdate.cmrk2 new file mode 100644 index 0000000000000..a054b7cc41855 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_commitdate.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_discount.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_discount.bin new file mode 100644 index 0000000000000..485983463b67d Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_discount.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_discount.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_discount.cmrk2 new file mode 100644 index 0000000000000..dd85512cc40ab Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_discount.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_extendedprice.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_extendedprice.bin new file mode 100644 index 0000000000000..a9b92b2d81127 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_extendedprice.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_extendedprice.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_extendedprice.cmrk2 new file mode 100644 index 0000000000000..7ef7b0963d2da Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_extendedprice.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_linenumber.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_linenumber.bin new file mode 100644 index 0000000000000..1f5763119603c Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_linenumber.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_linenumber.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_linenumber.cmrk2 new file mode 100644 index 0000000000000..bc8886112c84b Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_linenumber.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_linestatus.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_linestatus.bin new file mode 100644 index 0000000000000..adb867de7c8f8 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_linestatus.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_linestatus.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_linestatus.cmrk2 new file mode 100644 index 0000000000000..6b21a1ca465a4 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_linestatus.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_orderkey.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_orderkey.bin new file mode 100644 index 0000000000000..f4840d2ca245f Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_orderkey.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_orderkey.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_orderkey.cmrk2 new file mode 100644 index 0000000000000..0067fc50da8cc Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_orderkey.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_partkey.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_partkey.bin new file mode 100644 index 0000000000000..e82138df2fb41 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_partkey.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_partkey.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_partkey.cmrk2 new file mode 100644 index 0000000000000..1819a433430d7 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_partkey.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_quantity.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_quantity.bin new file mode 100644 index 0000000000000..21a469330612a Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_quantity.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_quantity.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_quantity.cmrk2 new file mode 100644 index 0000000000000..afed0101b3c4b Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_quantity.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_receiptdate.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_receiptdate.bin new file mode 100644 index 0000000000000..7a6c267cb9cc4 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_receiptdate.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_receiptdate.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_receiptdate.cmrk2 new file mode 100644 index 0000000000000..e4adf9e2dd70d Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_receiptdate.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_returnflag.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_returnflag.bin new file mode 100644 index 0000000000000..b0d48b41c5ec3 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_returnflag.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_returnflag.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_returnflag.cmrk2 new file mode 100644 index 0000000000000..29bd96ae152aa Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_returnflag.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipdate.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipdate.bin new file mode 100644 index 0000000000000..dd8141efe33c1 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipdate.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipdate.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipdate.cmrk2 new file mode 100644 index 0000000000000..4185e02b69b22 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipdate.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipinstruct.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipinstruct.bin new file mode 100644 index 0000000000000..53838fb3559fc Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipinstruct.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipinstruct.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipinstruct.cmrk2 new file mode 100644 index 0000000000000..108307a66b6e4 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipinstruct.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipmode.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipmode.bin new file mode 100644 index 0000000000000..bc3dd7a22ad08 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipmode.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipmode.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipmode.cmrk2 new file mode 100644 index 0000000000000..d906135ee5872 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipmode.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_suppkey.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_suppkey.bin new file mode 100644 index 0000000000000..3df367f9e7408 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_suppkey.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_suppkey.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_suppkey.cmrk2 new file mode 100644 index 0000000000000..4144cdf14a382 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_suppkey.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_tax.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_tax.bin new file mode 100644 index 0000000000000..95d648c9397c3 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_tax.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_tax.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_tax.cmrk2 new file mode 100644 index 0000000000000..47a6e77fc5763 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_tax.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/metadata_version.txt new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/primary.cidx new file mode 100644 index 0000000000000..32ae4d72f47a4 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/serialization.json new file mode 100644 index 0000000000000..91fd12eedfb2f --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"l_comment","num_defaults":0,"num_rows":150261},{"kind":"Default","name":"l_commitdate","num_defaults":0,"num_rows":150261},{"kind":"Default","name":"l_discount","num_defaults":13460,"num_rows":150261},{"kind":"Default","name":"l_extendedprice","num_defaults":0,"num_rows":150261},{"kind":"Default","name":"l_linenumber","num_defaults":0,"num_rows":150261},{"kind":"Default","name":"l_linestatus","num_defaults":0,"num_rows":150261},{"kind":"Default","name":"l_orderkey","num_defaults":0,"num_rows":150261},{"kind":"Default","name":"l_partkey","num_defaults":0,"num_rows":150261},{"kind":"Default","name":"l_quantity","num_defaults":0,"num_rows":150261},{"kind":"Default","name":"l_receiptdate","num_defaults":0,"num_rows":150261},{"kind":"Default","name":"l_returnflag","num_defaults":0,"num_rows":150261},{"kind":"Default","name":"l_shipdate","num_defaults":0,"num_rows":150261},{"kind":"Default","name":"l_shipinstruct","num_defaults":0,"num_rows":150261},{"kind":"Default","name":"l_shipmode","num_defaults":0,"num_rows":150261},{"kind":"Default","name":"l_suppkey","num_defaults":0,"num_rows":150261},{"kind":"Default","name":"l_tax","num_defaults":16870,"num_rows":150261}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/checksums.txt new file mode 100644 index 0000000000000..dbdb1d37e197c Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/columns.txt new file mode 100644 index 0000000000000..1b0cd5e295ec2 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/columns.txt @@ -0,0 +1,6 @@ +columns format version: 1 +4 columns: +`n_nationkey` Int64 +`n_name` String +`n_regionkey` Int64 +`n_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/count.txt new file mode 100644 index 0000000000000..410b14d2ce6f9 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/count.txt @@ -0,0 +1 @@ +25 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/data.bin new file mode 100644 index 0000000000000..ffa999a7f9d2b Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/data.cmrk3 new file mode 100644 index 0000000000000..a3ed1f6539970 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/default_compression_codec.txt new file mode 100644 index 0000000000000..061d1280b897b --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/metadata_version.txt new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/primary.cidx new file mode 100644 index 0000000000000..f323dcd7db25a Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/serialization.json new file mode 100644 index 0000000000000..6caa0c7892a8b --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"n_comment","num_defaults":0,"num_rows":25},{"kind":"Default","name":"n_name","num_defaults":0,"num_rows":25},{"kind":"Default","name":"n_nationkey","num_defaults":1,"num_rows":25},{"kind":"Default","name":"n_regionkey","num_defaults":5,"num_rows":25}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/checksums.txt new file mode 100644 index 0000000000000..85ffbd61c76b6 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/columns.txt new file mode 100644 index 0000000000000..6dfe31e78fefa --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/columns.txt @@ -0,0 +1,11 @@ +columns format version: 1 +9 columns: +`o_orderkey` Int64 +`o_custkey` Int64 +`o_orderstatus` String +`o_totalprice` Float64 +`o_orderdate` Date +`o_orderpriority` String +`o_clerk` String +`o_shippriority` Int64 +`o_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/count.txt new file mode 100644 index 0000000000000..ae1d2dbe54725 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/count.txt @@ -0,0 +1 @@ +37359 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/data.bin new file mode 100644 index 0000000000000..e63f8b3e5a9bb Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/data.cmrk3 new file mode 100644 index 0000000000000..af4873bbbdc34 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/default_compression_codec.txt new file mode 100644 index 0000000000000..061d1280b897b --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/metadata_version.txt new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/primary.cidx new file mode 100644 index 0000000000000..abf42eed670ad Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/serialization.json new file mode 100644 index 0000000000000..11c8f61315bbb --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"o_clerk","num_defaults":0,"num_rows":37359},{"kind":"Default","name":"o_comment","num_defaults":0,"num_rows":37359},{"kind":"Default","name":"o_custkey","num_defaults":0,"num_rows":37359},{"kind":"Default","name":"o_orderdate","num_defaults":0,"num_rows":37359},{"kind":"Default","name":"o_orderkey","num_defaults":0,"num_rows":37359},{"kind":"Default","name":"o_orderpriority","num_defaults":0,"num_rows":37359},{"kind":"Default","name":"o_orderstatus","num_defaults":0,"num_rows":37359},{"kind":"Sparse","name":"o_shippriority","num_defaults":37359,"num_rows":37359},{"kind":"Default","name":"o_totalprice","num_defaults":0,"num_rows":37359}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/checksums.txt new file mode 100644 index 0000000000000..ba6219414354d Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/columns.txt new file mode 100644 index 0000000000000..6dfe31e78fefa --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/columns.txt @@ -0,0 +1,11 @@ +columns format version: 1 +9 columns: +`o_orderkey` Int64 +`o_custkey` Int64 +`o_orderstatus` String +`o_totalprice` Float64 +`o_orderdate` Date +`o_orderpriority` String +`o_clerk` String +`o_shippriority` Int64 +`o_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/count.txt new file mode 100644 index 0000000000000..7b8330dab9c40 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/count.txt @@ -0,0 +1 @@ +37652 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/data.bin new file mode 100644 index 0000000000000..2f3e820560a05 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/data.cmrk3 new file mode 100644 index 0000000000000..71770cd09f355 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/default_compression_codec.txt new file mode 100644 index 0000000000000..061d1280b897b --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/metadata_version.txt new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/primary.cidx new file mode 100644 index 0000000000000..76052c3f6b33f Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/serialization.json new file mode 100644 index 0000000000000..3c3a6d6c6988e --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"o_clerk","num_defaults":0,"num_rows":37652},{"kind":"Default","name":"o_comment","num_defaults":0,"num_rows":37652},{"kind":"Default","name":"o_custkey","num_defaults":0,"num_rows":37652},{"kind":"Default","name":"o_orderdate","num_defaults":0,"num_rows":37652},{"kind":"Default","name":"o_orderkey","num_defaults":0,"num_rows":37652},{"kind":"Default","name":"o_orderpriority","num_defaults":0,"num_rows":37652},{"kind":"Default","name":"o_orderstatus","num_defaults":0,"num_rows":37652},{"kind":"Sparse","name":"o_shippriority","num_defaults":37652,"num_rows":37652},{"kind":"Default","name":"o_totalprice","num_defaults":0,"num_rows":37652}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/checksums.txt new file mode 100644 index 0000000000000..cfd8888b37df9 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/columns.txt new file mode 100644 index 0000000000000..6dfe31e78fefa --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/columns.txt @@ -0,0 +1,11 @@ +columns format version: 1 +9 columns: +`o_orderkey` Int64 +`o_custkey` Int64 +`o_orderstatus` String +`o_totalprice` Float64 +`o_orderdate` Date +`o_orderpriority` String +`o_clerk` String +`o_shippriority` Int64 +`o_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/count.txt new file mode 100644 index 0000000000000..c585e5375cc45 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/count.txt @@ -0,0 +1 @@ +37641 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/data.bin new file mode 100644 index 0000000000000..64346528d5f46 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/data.cmrk3 new file mode 100644 index 0000000000000..014387b46a38f Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/default_compression_codec.txt new file mode 100644 index 0000000000000..061d1280b897b --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/metadata_version.txt new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/primary.cidx new file mode 100644 index 0000000000000..8983040b4f749 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/serialization.json new file mode 100644 index 0000000000000..155cd493ede5a --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"o_clerk","num_defaults":0,"num_rows":37641},{"kind":"Default","name":"o_comment","num_defaults":0,"num_rows":37641},{"kind":"Default","name":"o_custkey","num_defaults":0,"num_rows":37641},{"kind":"Default","name":"o_orderdate","num_defaults":0,"num_rows":37641},{"kind":"Default","name":"o_orderkey","num_defaults":0,"num_rows":37641},{"kind":"Default","name":"o_orderpriority","num_defaults":0,"num_rows":37641},{"kind":"Default","name":"o_orderstatus","num_defaults":0,"num_rows":37641},{"kind":"Sparse","name":"o_shippriority","num_defaults":37641,"num_rows":37641},{"kind":"Default","name":"o_totalprice","num_defaults":0,"num_rows":37641}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/checksums.txt new file mode 100644 index 0000000000000..8319c32515213 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/columns.txt new file mode 100644 index 0000000000000..6dfe31e78fefa --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/columns.txt @@ -0,0 +1,11 @@ +columns format version: 1 +9 columns: +`o_orderkey` Int64 +`o_custkey` Int64 +`o_orderstatus` String +`o_totalprice` Float64 +`o_orderdate` Date +`o_orderpriority` String +`o_clerk` String +`o_shippriority` Int64 +`o_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/count.txt new file mode 100644 index 0000000000000..fec61cad7645d --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/count.txt @@ -0,0 +1 @@ +37348 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/data.bin new file mode 100644 index 0000000000000..e72837d98dd1c Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/data.cmrk3 new file mode 100644 index 0000000000000..5a885f6cefcfb Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/default_compression_codec.txt new file mode 100644 index 0000000000000..061d1280b897b --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/metadata_version.txt new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/primary.cidx new file mode 100644 index 0000000000000..743b7c6dbd602 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/serialization.json new file mode 100644 index 0000000000000..239572db24702 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"o_clerk","num_defaults":0,"num_rows":37348},{"kind":"Default","name":"o_comment","num_defaults":0,"num_rows":37348},{"kind":"Default","name":"o_custkey","num_defaults":0,"num_rows":37348},{"kind":"Default","name":"o_orderdate","num_defaults":0,"num_rows":37348},{"kind":"Default","name":"o_orderkey","num_defaults":0,"num_rows":37348},{"kind":"Default","name":"o_orderpriority","num_defaults":0,"num_rows":37348},{"kind":"Default","name":"o_orderstatus","num_defaults":0,"num_rows":37348},{"kind":"Sparse","name":"o_shippriority","num_defaults":37348,"num_rows":37348},{"kind":"Default","name":"o_totalprice","num_defaults":0,"num_rows":37348}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/checksums.txt new file mode 100644 index 0000000000000..d75cf1d42a8ff Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/columns.txt new file mode 100644 index 0000000000000..03af5b613ab86 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/columns.txt @@ -0,0 +1,11 @@ +columns format version: 1 +9 columns: +`p_partkey` Int64 +`p_name` String +`p_mfgr` String +`p_brand` String +`p_type` String +`p_size` Int64 +`p_container` String +`p_retailprice` Float64 +`p_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/count.txt new file mode 100644 index 0000000000000..9f945bb2547e0 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/count.txt @@ -0,0 +1 @@ +4956 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/data.bin new file mode 100644 index 0000000000000..eee5662f5e10f Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/data.cmrk3 new file mode 100644 index 0000000000000..63e3f1e3eaa94 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/default_compression_codec.txt new file mode 100644 index 0000000000000..061d1280b897b --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/metadata_version.txt new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/primary.cidx new file mode 100644 index 0000000000000..1111223bf4100 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/serialization.json new file mode 100644 index 0000000000000..2dba9a4fb0630 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"p_brand","num_defaults":0,"num_rows":4956},{"kind":"Default","name":"p_comment","num_defaults":0,"num_rows":4956},{"kind":"Default","name":"p_container","num_defaults":0,"num_rows":4956},{"kind":"Default","name":"p_mfgr","num_defaults":0,"num_rows":4956},{"kind":"Default","name":"p_name","num_defaults":0,"num_rows":4956},{"kind":"Default","name":"p_partkey","num_defaults":0,"num_rows":4956},{"kind":"Default","name":"p_retailprice","num_defaults":0,"num_rows":4956},{"kind":"Default","name":"p_size","num_defaults":0,"num_rows":4956},{"kind":"Default","name":"p_type","num_defaults":0,"num_rows":4956}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/checksums.txt new file mode 100644 index 0000000000000..5776a23ba4dd2 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/columns.txt new file mode 100644 index 0000000000000..03af5b613ab86 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/columns.txt @@ -0,0 +1,11 @@ +columns format version: 1 +9 columns: +`p_partkey` Int64 +`p_name` String +`p_mfgr` String +`p_brand` String +`p_type` String +`p_size` Int64 +`p_container` String +`p_retailprice` Float64 +`p_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/count.txt new file mode 100644 index 0000000000000..c65ed0e61653b --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/count.txt @@ -0,0 +1 @@ +4917 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/data.bin new file mode 100644 index 0000000000000..4db23bebe40bd Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/data.cmrk3 new file mode 100644 index 0000000000000..8adb51f83063f Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/default_compression_codec.txt new file mode 100644 index 0000000000000..061d1280b897b --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/metadata_version.txt new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/primary.cidx new file mode 100644 index 0000000000000..9d0261e14c039 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/serialization.json new file mode 100644 index 0000000000000..56da900c9fa45 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"p_brand","num_defaults":0,"num_rows":4917},{"kind":"Default","name":"p_comment","num_defaults":0,"num_rows":4917},{"kind":"Default","name":"p_container","num_defaults":0,"num_rows":4917},{"kind":"Default","name":"p_mfgr","num_defaults":0,"num_rows":4917},{"kind":"Default","name":"p_name","num_defaults":0,"num_rows":4917},{"kind":"Default","name":"p_partkey","num_defaults":0,"num_rows":4917},{"kind":"Default","name":"p_retailprice","num_defaults":0,"num_rows":4917},{"kind":"Default","name":"p_size","num_defaults":0,"num_rows":4917},{"kind":"Default","name":"p_type","num_defaults":0,"num_rows":4917}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/checksums.txt new file mode 100644 index 0000000000000..80db7774f31d1 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/columns.txt new file mode 100644 index 0000000000000..03af5b613ab86 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/columns.txt @@ -0,0 +1,11 @@ +columns format version: 1 +9 columns: +`p_partkey` Int64 +`p_name` String +`p_mfgr` String +`p_brand` String +`p_type` String +`p_size` Int64 +`p_container` String +`p_retailprice` Float64 +`p_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/count.txt new file mode 100644 index 0000000000000..edbcad025b230 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/count.txt @@ -0,0 +1 @@ +5044 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/data.bin new file mode 100644 index 0000000000000..7981580a9e021 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/data.cmrk3 new file mode 100644 index 0000000000000..a157da55307d1 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/default_compression_codec.txt new file mode 100644 index 0000000000000..061d1280b897b --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/metadata_version.txt new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/primary.cidx new file mode 100644 index 0000000000000..28a1b5f367108 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/serialization.json new file mode 100644 index 0000000000000..a6e9dc7b61b4b --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"p_brand","num_defaults":0,"num_rows":5044},{"kind":"Default","name":"p_comment","num_defaults":0,"num_rows":5044},{"kind":"Default","name":"p_container","num_defaults":0,"num_rows":5044},{"kind":"Default","name":"p_mfgr","num_defaults":0,"num_rows":5044},{"kind":"Default","name":"p_name","num_defaults":0,"num_rows":5044},{"kind":"Default","name":"p_partkey","num_defaults":0,"num_rows":5044},{"kind":"Default","name":"p_retailprice","num_defaults":0,"num_rows":5044},{"kind":"Default","name":"p_size","num_defaults":0,"num_rows":5044},{"kind":"Default","name":"p_type","num_defaults":0,"num_rows":5044}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/checksums.txt new file mode 100644 index 0000000000000..3fad79d20e970 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/columns.txt new file mode 100644 index 0000000000000..03af5b613ab86 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/columns.txt @@ -0,0 +1,11 @@ +columns format version: 1 +9 columns: +`p_partkey` Int64 +`p_name` String +`p_mfgr` String +`p_brand` String +`p_type` String +`p_size` Int64 +`p_container` String +`p_retailprice` Float64 +`p_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/count.txt new file mode 100644 index 0000000000000..0caca7bf9e157 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/count.txt @@ -0,0 +1 @@ +5083 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/data.bin new file mode 100644 index 0000000000000..c33d22e5b517b Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/data.cmrk3 new file mode 100644 index 0000000000000..114907af69329 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/default_compression_codec.txt new file mode 100644 index 0000000000000..061d1280b897b --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/metadata_version.txt new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/primary.cidx new file mode 100644 index 0000000000000..4cf8a8543b720 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/serialization.json new file mode 100644 index 0000000000000..98a4a89817b41 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"p_brand","num_defaults":0,"num_rows":5083},{"kind":"Default","name":"p_comment","num_defaults":0,"num_rows":5083},{"kind":"Default","name":"p_container","num_defaults":0,"num_rows":5083},{"kind":"Default","name":"p_mfgr","num_defaults":0,"num_rows":5083},{"kind":"Default","name":"p_name","num_defaults":0,"num_rows":5083},{"kind":"Default","name":"p_partkey","num_defaults":0,"num_rows":5083},{"kind":"Default","name":"p_retailprice","num_defaults":0,"num_rows":5083},{"kind":"Default","name":"p_size","num_defaults":0,"num_rows":5083},{"kind":"Default","name":"p_type","num_defaults":0,"num_rows":5083}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/checksums.txt new file mode 100644 index 0000000000000..925f0214c1f9b Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/columns.txt new file mode 100644 index 0000000000000..b238d80aa5326 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/columns.txt @@ -0,0 +1,7 @@ +columns format version: 1 +5 columns: +`ps_partkey` Int64 +`ps_suppkey` Int64 +`ps_availqty` Int64 +`ps_supplycost` Float64 +`ps_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/count.txt new file mode 100644 index 0000000000000..e76cc2d994644 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/count.txt @@ -0,0 +1 @@ +19692 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/data.bin new file mode 100644 index 0000000000000..4596e07e8024d Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/data.cmrk3 new file mode 100644 index 0000000000000..7d205cc0ba922 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/default_compression_codec.txt new file mode 100644 index 0000000000000..061d1280b897b --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/metadata_version.txt new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/primary.cidx new file mode 100644 index 0000000000000..1a6aa1025d1a3 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/serialization.json new file mode 100644 index 0000000000000..e9f2684142a36 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"ps_availqty","num_defaults":0,"num_rows":19692},{"kind":"Default","name":"ps_comment","num_defaults":0,"num_rows":19692},{"kind":"Default","name":"ps_partkey","num_defaults":0,"num_rows":19692},{"kind":"Default","name":"ps_suppkey","num_defaults":0,"num_rows":19692},{"kind":"Default","name":"ps_supplycost","num_defaults":0,"num_rows":19692}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/checksums.txt new file mode 100644 index 0000000000000..7c4fbc1075e97 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/columns.txt new file mode 100644 index 0000000000000..b238d80aa5326 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/columns.txt @@ -0,0 +1,7 @@ +columns format version: 1 +5 columns: +`ps_partkey` Int64 +`ps_suppkey` Int64 +`ps_availqty` Int64 +`ps_supplycost` Float64 +`ps_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/count.txt new file mode 100644 index 0000000000000..f8ff62878b469 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/count.txt @@ -0,0 +1 @@ +19800 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/data.bin new file mode 100644 index 0000000000000..64ba656fe0eaa Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/data.cmrk3 new file mode 100644 index 0000000000000..125b815c52b48 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/default_compression_codec.txt new file mode 100644 index 0000000000000..061d1280b897b --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/metadata_version.txt new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/primary.cidx new file mode 100644 index 0000000000000..97e1cdc21faf9 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/serialization.json new file mode 100644 index 0000000000000..ab84853e95b25 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"ps_availqty","num_defaults":0,"num_rows":19800},{"kind":"Default","name":"ps_comment","num_defaults":0,"num_rows":19800},{"kind":"Default","name":"ps_partkey","num_defaults":0,"num_rows":19800},{"kind":"Default","name":"ps_suppkey","num_defaults":0,"num_rows":19800},{"kind":"Default","name":"ps_supplycost","num_defaults":0,"num_rows":19800}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/checksums.txt new file mode 100644 index 0000000000000..87aeeeb3ca082 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/columns.txt new file mode 100644 index 0000000000000..b238d80aa5326 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/columns.txt @@ -0,0 +1,7 @@ +columns format version: 1 +5 columns: +`ps_partkey` Int64 +`ps_suppkey` Int64 +`ps_availqty` Int64 +`ps_supplycost` Float64 +`ps_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/count.txt new file mode 100644 index 0000000000000..7657197913e27 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/count.txt @@ -0,0 +1 @@ +20308 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/data.bin new file mode 100644 index 0000000000000..76fc644c02023 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/data.cmrk3 new file mode 100644 index 0000000000000..67cb661bc7e61 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/default_compression_codec.txt new file mode 100644 index 0000000000000..061d1280b897b --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/metadata_version.txt new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/primary.cidx new file mode 100644 index 0000000000000..d9174b671f283 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/serialization.json new file mode 100644 index 0000000000000..5a366eba2d5c6 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"ps_availqty","num_defaults":0,"num_rows":20308},{"kind":"Default","name":"ps_comment","num_defaults":0,"num_rows":20308},{"kind":"Default","name":"ps_partkey","num_defaults":0,"num_rows":20308},{"kind":"Default","name":"ps_suppkey","num_defaults":0,"num_rows":20308},{"kind":"Default","name":"ps_supplycost","num_defaults":0,"num_rows":20308}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/checksums.txt new file mode 100644 index 0000000000000..f06dcd26da9f2 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/columns.txt new file mode 100644 index 0000000000000..b238d80aa5326 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/columns.txt @@ -0,0 +1,7 @@ +columns format version: 1 +5 columns: +`ps_partkey` Int64 +`ps_suppkey` Int64 +`ps_availqty` Int64 +`ps_supplycost` Float64 +`ps_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/count.txt new file mode 100644 index 0000000000000..2a32e263681f6 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/count.txt @@ -0,0 +1 @@ +20200 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/data.bin new file mode 100644 index 0000000000000..d1109634c1d9a Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/data.cmrk3 new file mode 100644 index 0000000000000..5f1afe89e2c63 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/default_compression_codec.txt new file mode 100644 index 0000000000000..061d1280b897b --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/metadata_version.txt new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/primary.cidx new file mode 100644 index 0000000000000..428d2a33ad597 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/serialization.json new file mode 100644 index 0000000000000..f80553ad2f377 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"ps_availqty","num_defaults":0,"num_rows":20200},{"kind":"Default","name":"ps_comment","num_defaults":0,"num_rows":20200},{"kind":"Default","name":"ps_partkey","num_defaults":0,"num_rows":20200},{"kind":"Default","name":"ps_suppkey","num_defaults":0,"num_rows":20200},{"kind":"Default","name":"ps_supplycost","num_defaults":0,"num_rows":20200}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/checksums.txt new file mode 100644 index 0000000000000..6310184e154bd Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/columns.txt new file mode 100644 index 0000000000000..c22e2b6776e69 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/columns.txt @@ -0,0 +1,5 @@ +columns format version: 1 +3 columns: +`r_regionkey` Int64 +`r_name` String +`r_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/count.txt new file mode 100644 index 0000000000000..7813681f5b41c --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/count.txt @@ -0,0 +1 @@ +5 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/data.bin new file mode 100644 index 0000000000000..6c763f4d18edd Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/data.cmrk3 new file mode 100644 index 0000000000000..b864a4758a35e Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/default_compression_codec.txt new file mode 100644 index 0000000000000..061d1280b897b --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/metadata_version.txt new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/primary.cidx new file mode 100644 index 0000000000000..ba2a8d00588c8 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/serialization.json new file mode 100644 index 0000000000000..96475e59e7714 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"r_comment","num_defaults":0,"num_rows":5},{"kind":"Default","name":"r_name","num_defaults":0,"num_rows":5},{"kind":"Default","name":"r_regionkey","num_defaults":1,"num_rows":5}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/checksums.txt new file mode 100644 index 0000000000000..18b4a3c5fdc92 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/columns.txt new file mode 100644 index 0000000000000..c1b3eb58785b5 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/columns.txt @@ -0,0 +1,9 @@ +columns format version: 1 +7 columns: +`s_suppkey` Int64 +`s_name` String +`s_address` String +`s_nationkey` Int64 +`s_phone` String +`s_acctbal` Float64 +`s_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/count.txt new file mode 100644 index 0000000000000..e37d32abba426 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/count.txt @@ -0,0 +1 @@ +1000 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/data.bin new file mode 100644 index 0000000000000..0b215ffe47c81 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/data.cmrk3 new file mode 100644 index 0000000000000..2164150655887 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/default_compression_codec.txt new file mode 100644 index 0000000000000..061d1280b897b --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/metadata_version.txt new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/primary.cidx new file mode 100644 index 0000000000000..2540ced74cd1b Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/serialization.json new file mode 100644 index 0000000000000..67c8884ba4fee --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"s_acctbal","num_defaults":0,"num_rows":1000},{"kind":"Default","name":"s_address","num_defaults":0,"num_rows":1000},{"kind":"Default","name":"s_comment","num_defaults":0,"num_rows":1000},{"kind":"Default","name":"s_name","num_defaults":0,"num_rows":1000},{"kind":"Default","name":"s_nationkey","num_defaults":36,"num_rows":1000},{"kind":"Default","name":"s_phone","num_defaults":0,"num_rows":1000},{"kind":"Default","name":"s_suppkey","num_defaults":0,"num_rows":1000}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/customer/part-00000-72fdd3c7-0c9d-453e-abca-c9caa7219875_00000.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/customer/part-00000-72fdd3c7-0c9d-453e-abca-c9caa7219875_00000.c000.snappy.parquet new file mode 100644 index 0000000000000..0fa4d5944e619 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/customer/part-00000-72fdd3c7-0c9d-453e-abca-c9caa7219875_00000.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/customer/part-00000-72fdd3c7-0c9d-453e-abca-c9caa7219875_00001.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/customer/part-00000-72fdd3c7-0c9d-453e-abca-c9caa7219875_00001.c000.snappy.parquet new file mode 100644 index 0000000000000..d1b02a02e2414 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/customer/part-00000-72fdd3c7-0c9d-453e-abca-c9caa7219875_00001.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/customer/part-00001-72fdd3c7-0c9d-453e-abca-c9caa7219875_00000.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/customer/part-00001-72fdd3c7-0c9d-453e-abca-c9caa7219875_00000.c000.snappy.parquet new file mode 100644 index 0000000000000..4a2e590c27806 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/customer/part-00001-72fdd3c7-0c9d-453e-abca-c9caa7219875_00000.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/customer/part-00001-72fdd3c7-0c9d-453e-abca-c9caa7219875_00001.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/customer/part-00001-72fdd3c7-0c9d-453e-abca-c9caa7219875_00001.c000.snappy.parquet new file mode 100644 index 0000000000000..ad43a2333d8e0 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/customer/part-00001-72fdd3c7-0c9d-453e-abca-c9caa7219875_00001.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/lineitem/part-00000-8b86c10d-22d3-4800-8d49-0de982c95d93_00000.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/lineitem/part-00000-8b86c10d-22d3-4800-8d49-0de982c95d93_00000.c000.snappy.parquet new file mode 100644 index 0000000000000..d69bc752d08ee Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/lineitem/part-00000-8b86c10d-22d3-4800-8d49-0de982c95d93_00000.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/lineitem/part-00000-8b86c10d-22d3-4800-8d49-0de982c95d93_00001.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/lineitem/part-00000-8b86c10d-22d3-4800-8d49-0de982c95d93_00001.c000.snappy.parquet new file mode 100644 index 0000000000000..69018b7f4570b Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/lineitem/part-00000-8b86c10d-22d3-4800-8d49-0de982c95d93_00001.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/lineitem/part-00001-8b86c10d-22d3-4800-8d49-0de982c95d93_00000.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/lineitem/part-00001-8b86c10d-22d3-4800-8d49-0de982c95d93_00000.c000.snappy.parquet new file mode 100644 index 0000000000000..86084b590de36 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/lineitem/part-00001-8b86c10d-22d3-4800-8d49-0de982c95d93_00000.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/lineitem/part-00001-8b86c10d-22d3-4800-8d49-0de982c95d93_00001.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/lineitem/part-00001-8b86c10d-22d3-4800-8d49-0de982c95d93_00001.c000.snappy.parquet new file mode 100644 index 0000000000000..1a3b0f91b0b15 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/lineitem/part-00001-8b86c10d-22d3-4800-8d49-0de982c95d93_00001.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/nation/part-00000-008a5ae9-3c9c-42f1-bef3-81cc60b3f4ab_00000.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/nation/part-00000-008a5ae9-3c9c-42f1-bef3-81cc60b3f4ab_00000.c000.snappy.parquet new file mode 100644 index 0000000000000..c10d7bb9b1518 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/nation/part-00000-008a5ae9-3c9c-42f1-bef3-81cc60b3f4ab_00000.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/orders/part-00000-a527ff3d-0d73-4f1a-a1a2-78c1beaf7ece_00000.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/orders/part-00000-a527ff3d-0d73-4f1a-a1a2-78c1beaf7ece_00000.c000.snappy.parquet new file mode 100644 index 0000000000000..938cc20de48ca Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/orders/part-00000-a527ff3d-0d73-4f1a-a1a2-78c1beaf7ece_00000.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/orders/part-00000-a527ff3d-0d73-4f1a-a1a2-78c1beaf7ece_00001.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/orders/part-00000-a527ff3d-0d73-4f1a-a1a2-78c1beaf7ece_00001.c000.snappy.parquet new file mode 100644 index 0000000000000..eb80f5f0ce36b Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/orders/part-00000-a527ff3d-0d73-4f1a-a1a2-78c1beaf7ece_00001.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/orders/part-00001-a527ff3d-0d73-4f1a-a1a2-78c1beaf7ece_00000.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/orders/part-00001-a527ff3d-0d73-4f1a-a1a2-78c1beaf7ece_00000.c000.snappy.parquet new file mode 100644 index 0000000000000..879e358c2d35a Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/orders/part-00001-a527ff3d-0d73-4f1a-a1a2-78c1beaf7ece_00000.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/orders/part-00001-a527ff3d-0d73-4f1a-a1a2-78c1beaf7ece_00001.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/orders/part-00001-a527ff3d-0d73-4f1a-a1a2-78c1beaf7ece_00001.c000.snappy.parquet new file mode 100644 index 0000000000000..eef0227873c6d Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/orders/part-00001-a527ff3d-0d73-4f1a-a1a2-78c1beaf7ece_00001.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/part/part-00000-0c39a257-1f7a-4c6e-85a2-7d21fefbef78_00000.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/part/part-00000-0c39a257-1f7a-4c6e-85a2-7d21fefbef78_00000.c000.snappy.parquet new file mode 100644 index 0000000000000..2fded2af5a0e5 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/part/part-00000-0c39a257-1f7a-4c6e-85a2-7d21fefbef78_00000.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/part/part-00000-0c39a257-1f7a-4c6e-85a2-7d21fefbef78_00001.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/part/part-00000-0c39a257-1f7a-4c6e-85a2-7d21fefbef78_00001.c000.snappy.parquet new file mode 100644 index 0000000000000..3d342e80a78f0 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/part/part-00000-0c39a257-1f7a-4c6e-85a2-7d21fefbef78_00001.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/part/part-00001-0c39a257-1f7a-4c6e-85a2-7d21fefbef78_00000.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/part/part-00001-0c39a257-1f7a-4c6e-85a2-7d21fefbef78_00000.c000.snappy.parquet new file mode 100644 index 0000000000000..3180a2724bb19 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/part/part-00001-0c39a257-1f7a-4c6e-85a2-7d21fefbef78_00000.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/part/part-00001-0c39a257-1f7a-4c6e-85a2-7d21fefbef78_00001.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/part/part-00001-0c39a257-1f7a-4c6e-85a2-7d21fefbef78_00001.c000.snappy.parquet new file mode 100644 index 0000000000000..37cc223cd6527 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/part/part-00001-0c39a257-1f7a-4c6e-85a2-7d21fefbef78_00001.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/partsupp/part-00000-a989f762-9541-4d30-a8cf-96d8e695e188_00000.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/partsupp/part-00000-a989f762-9541-4d30-a8cf-96d8e695e188_00000.c000.snappy.parquet new file mode 100644 index 0000000000000..c25e5565222b6 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/partsupp/part-00000-a989f762-9541-4d30-a8cf-96d8e695e188_00000.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/partsupp/part-00000-a989f762-9541-4d30-a8cf-96d8e695e188_00001.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/partsupp/part-00000-a989f762-9541-4d30-a8cf-96d8e695e188_00001.c000.snappy.parquet new file mode 100644 index 0000000000000..40735373a0573 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/partsupp/part-00000-a989f762-9541-4d30-a8cf-96d8e695e188_00001.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/partsupp/part-00001-a989f762-9541-4d30-a8cf-96d8e695e188_00000.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/partsupp/part-00001-a989f762-9541-4d30-a8cf-96d8e695e188_00000.c000.snappy.parquet new file mode 100644 index 0000000000000..e59dc1e8844ad Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/partsupp/part-00001-a989f762-9541-4d30-a8cf-96d8e695e188_00000.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/partsupp/part-00001-a989f762-9541-4d30-a8cf-96d8e695e188_00001.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/partsupp/part-00001-a989f762-9541-4d30-a8cf-96d8e695e188_00001.c000.snappy.parquet new file mode 100644 index 0000000000000..e53d6517ce6b2 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/partsupp/part-00001-a989f762-9541-4d30-a8cf-96d8e695e188_00001.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/region/part-00000-4f02eea9-c038-4c8b-ab09-5f36588cb8be_00000.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/region/part-00000-4f02eea9-c038-4c8b-ab09-5f36588cb8be_00000.c000.snappy.parquet new file mode 100644 index 0000000000000..74955d0483e0c Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/region/part-00000-4f02eea9-c038-4c8b-ab09-5f36588cb8be_00000.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/supplier/part-00000-2d14f2f1-565e-49d3-8f00-cf5d3a8466a3_00000.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/supplier/part-00000-2d14f2f1-565e-49d3-8f00-cf5d3a8466a3_00000.c000.snappy.parquet new file mode 100644 index 0000000000000..c9b45ea125590 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/supplier/part-00000-2d14f2f1-565e-49d3-8f00-cf5d3a8466a3_00000.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHBucketSuite.scala b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHBucketSuite.scala new file mode 100644 index 0000000000000..d031e904a7c0a --- /dev/null +++ b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHBucketSuite.scala @@ -0,0 +1,488 @@ +/* + * 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 io.glutenproject.execution + +import org.apache.spark.{SPARK_VERSION_SHORT, SparkConf} +import org.apache.spark.sql.execution.ColumnarInputAdapter +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper + +import org.apache.commons.io.FileUtils + +import java.io.File + +// Some sqls' line length exceeds 100 +// scalastyle:off line.size.limit + +class GlutenClickHouseTPCHBucketSuite + extends GlutenClickHouseTPCHAbstractSuite + with AdaptiveSparkPlanHelper { + + override protected val tablesPath: String = basePath + "/tpch-data-ch" + override protected val tpchQueries: String = + rootPath + "../../../../gluten-core/src/test/resources/tpch-queries" + override protected val queriesResults: String = rootPath + "bucket-queries-output" + + protected val bucketTableResourcePath: String = rootPath + "tpch-data-bucket/mergetree_bucket" + protected val bucketTableDataPath: String = basePath + "/tpch-mergetree-bucket" + protected lazy val sparkVersion: String = { + val version = SPARK_VERSION_SHORT.split("\\.") + version(0) + "." + version(1) + } + + 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", "-1") // for test bucket join + .set("spark.sql.adaptive.enabled", "true") + .set("spark.gluten.sql.columnar.backend.ch.shuffle.hash.algorithm", "sparkMurmurHash3_32") + } + + override protected val createNullableTables = true + + override def beforeAll(): Unit = { + super.beforeAll() + FileUtils.copyDirectory(new File(bucketTableResourcePath), new File(bucketTableDataPath)) + createTPCHMergeTreeBucketTables() + } + + protected def createTPCHMergeTreeBucketTables(): Unit = { + spark.sql(s""" + |CREATE DATABASE IF NOT EXISTS tpch_mergetree_bucket + |""".stripMargin) + spark.sql("use tpch_mergetree_bucket") + val customerData = bucketTableDataPath + "/customer" + spark.sql(s"DROP TABLE IF EXISTS customer") + // On Spark 3.2, bucket table does not support to create bucket column with sort columns for + // DS V2 + spark.sql(s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS customer ( + | c_custkey bigint, + | c_name string, + | c_address string, + | c_nationkey bigint, + | c_phone string, + | c_acctbal double, + | c_mktsegment string, + | c_comment string) + | USING clickhouse + | LOCATION '$customerData' + | CLUSTERED BY (c_custkey) + | ${if (sparkVersion.equals("3.2")) "" else "SORTED BY (c_custkey)"} INTO 2 BUCKETS; + |""".stripMargin) + + val lineitemData = bucketTableDataPath + "/lineitem" + spark.sql(s"DROP TABLE IF EXISTS lineitem") + spark.sql( + s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS lineitem ( + | 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 + | LOCATION '$lineitemData' + | CLUSTERED BY (l_orderkey) + | ${if (sparkVersion.equals("3.2")) "" else "SORTED BY (l_shipdate, l_orderkey)"} INTO 2 BUCKETS; + |""".stripMargin) + + val nationData = bucketTableDataPath + "/nation" + spark.sql(s"DROP TABLE IF EXISTS nation") + spark.sql( + s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS nation ( + | n_nationkey bigint, + | n_name string, + | n_regionkey bigint, + | n_comment string) + | USING clickhouse + | LOCATION '$nationData' + | CLUSTERED BY (n_nationkey) + | ${if (sparkVersion.equals("3.2")) "" else "SORTED BY (n_nationkey)"} INTO 1 BUCKETS; + |""".stripMargin) + + val regionData = bucketTableDataPath + "/region" + spark.sql(s"DROP TABLE IF EXISTS region") + spark.sql( + s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS region ( + | r_regionkey bigint, + | r_name string, + | r_comment string) + | USING clickhouse + | LOCATION '$regionData' + | CLUSTERED BY (r_regionkey) + | ${if (sparkVersion.equals("3.2")) "" else "SORTED BY (r_regionkey)"} INTO 1 BUCKETS; + |""".stripMargin) + + val ordersData = bucketTableDataPath + "/orders" + spark.sql(s"DROP TABLE IF EXISTS orders") + spark.sql( + s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS orders ( + | o_orderkey bigint, + | o_custkey bigint, + | o_orderstatus string, + | o_totalprice double, + | o_orderdate date, + | o_orderpriority string, + | o_clerk string, + | o_shippriority bigint, + | o_comment string) + | USING clickhouse + | LOCATION '$ordersData' + | CLUSTERED BY (o_orderkey) + | ${if (sparkVersion.equals("3.2")) "" else "SORTED BY (o_orderkey, o_orderdate)"} INTO 2 BUCKETS; + |""".stripMargin) + + val partData = bucketTableDataPath + "/part" + spark.sql(s"DROP TABLE IF EXISTS part") + spark.sql(s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS part ( + | p_partkey bigint, + | p_name string, + | p_mfgr string, + | p_brand string, + | p_type string, + | p_size bigint, + | p_container string, + | p_retailprice double, + | p_comment string) + | USING clickhouse + | LOCATION '$partData' + | CLUSTERED BY (p_partkey) + | ${if (sparkVersion.equals("3.2")) "" else "SORTED BY (p_partkey)"} INTO 2 BUCKETS; + |""".stripMargin) + + val partsuppData = bucketTableDataPath + "/partsupp" + spark.sql(s"DROP TABLE IF EXISTS partsupp") + spark.sql( + s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS partsupp ( + | ps_partkey bigint, + | ps_suppkey bigint, + | ps_availqty bigint, + | ps_supplycost double, + | ps_comment string) + | USING clickhouse + | LOCATION '$partsuppData' + | CLUSTERED BY (ps_partkey) + | ${if (sparkVersion.equals("3.2")) "" else "SORTED BY (ps_partkey)"} INTO 2 BUCKETS; + |""".stripMargin) + + val supplierData = bucketTableDataPath + "/supplier" + spark.sql(s"DROP TABLE IF EXISTS supplier") + spark.sql(s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS supplier ( + | s_suppkey bigint, + | s_name string, + | s_address string, + | s_nationkey bigint, + | s_phone string, + | s_acctbal double, + | s_comment string) + | USING clickhouse + | LOCATION '$supplierData' + | CLUSTERED BY (s_suppkey) + | ${if (sparkVersion.equals("3.2")) "" else "SORTED BY (s_suppkey)"} INTO 1 BUCKETS; + |""".stripMargin) + + val result = spark + .sql(s""" + | show tables; + |""".stripMargin) + .collect() + assert(result.length == 8) + } + + test("TPCH Q1") { + runTPCHQuery(1)( + df => { + 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("outputRows").value === 591673) + }) + } + + test("TPCH Q2") { + runTPCHQuery(2)( + df => { + val plans = collect(df.queryExecution.executedPlan) { + case scanExec: BasicScanExecTransformer => scanExec + case joinExec: HashJoinLikeExecTransformer => joinExec + } + assert( + plans(3) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ColumnarInputAdapter]) + assert( + plans(3) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ColumnarInputAdapter]) + + // Check the bucket join + assert( + plans(4) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ProjectExecTransformer]) + assert( + plans(4) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[FilterExecTransformerBase]) + + assert( + plans(9) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ColumnarInputAdapter]) + if (sparkVersion.equals("3.2")) { + assert( + plans(9) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ColumnarInputAdapter]) + } else { + assert( + plans(9) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[FilterExecTransformerBase]) + } + + if (sparkVersion.equals("3.2")) { + assert(!(plans(11).asInstanceOf[FileSourceScanExecTransformer].bucketedScan)) + } else { + assert(plans(11).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) + } + assert(plans(11).metrics("numFiles").value === 1) + assert(plans(11).metrics("outputRows").value === 1000) + }) + } + + test("TPCH Q3") { + runTPCHQuery(3)( + df => { + val plans = collect(df.queryExecution.executedPlan) { + case scanExec: BasicScanExecTransformer => scanExec + case joinExec: HashJoinLikeExecTransformer => joinExec + } + if (sparkVersion.equals("3.2")) { + assert( + plans(1) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ColumnarInputAdapter]) + } else { + assert( + plans(1) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ProjectExecTransformer]) + } + assert( + plans(1) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ColumnarInputAdapter]) + + if (sparkVersion.equals("3.2")) { + assert(!(plans(2).asInstanceOf[FileSourceScanExecTransformer].bucketedScan)) + } else { + assert(plans(2).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) + } + assert(plans(2).metrics("numFiles").value === 4) + assert(plans(2).metrics("outputRows").value === 3111) + + assert(!(plans(3).asInstanceOf[FileSourceScanExecTransformer].bucketedScan)) + assert(plans(3).metrics("numFiles").value === 4) + assert(plans(3).metrics("outputRows").value === 72678) + }) + } + + test("TPCH Q4") { + runTPCHQuery(4)( + df => { + val plans = collect(df.queryExecution.executedPlan) { + case scanExec: BasicScanExecTransformer => scanExec + case joinExec: HashJoinLikeExecTransformer => joinExec + } + // bucket join + assert( + plans(0) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ProjectExecTransformer]) + assert( + plans(0) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ProjectExecTransformer]) + + assert(plans(1).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) + assert(plans(1).metrics("numFiles").value === 4) + assert(plans(1).metrics("outputRows").value === 5552) + + assert(plans(2).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) + assert(plans(2).metrics("numFiles").value === 4) + assert(plans(2).metrics("outputRows").value === 379809) + }) + } + + test("TPCH Q6") { + runTPCHQuery(6)( + df => { + 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("outputRows").value === 11618) + }) + } + + test("TPCH Q12") { + runTPCHQuery(12)( + df => { + val plans = collect(df.queryExecution.executedPlan) { + case scanExec: BasicScanExecTransformer => scanExec + case joinExec: HashJoinLikeExecTransformer => joinExec + } + // bucket join + assert( + plans(0) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[FilterExecTransformerBase]) + assert( + plans(0) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ProjectExecTransformer]) + + assert(plans(1).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) + assert(plans(1).metrics("numFiles").value === 4) + assert(plans(1).metrics("outputRows").value === 150000) + + assert(plans(2).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) + assert(plans(2).metrics("numFiles").value === 4) + assert(plans(2).metrics("outputRows").value === 3155) + }) + } + + test("TPCH Q18") { + runTPCHQuery(18)( + df => { + val plans = collect(df.queryExecution.executedPlan) { + case joinExec: HashJoinLikeExecTransformer => joinExec + } + // bucket join + assert( + plans(2) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[FilterExecTransformerBase]) + assert( + plans(2) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ProjectExecTransformer]) + // bucket join + assert( + plans(3) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[FilterExecTransformerBase]) + assert( + plans(3) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ProjectExecTransformer]) + }) + } + + test("TPCH Q20") { + runTPCHQuery(20)( + df => { + val plans = collect(df.queryExecution.executedPlan) { + case joinExec: HashJoinLikeExecTransformer => joinExec + } + if (sparkVersion.equals("3.2")) { + assert( + plans(1) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ColumnarInputAdapter]) + } else { + assert( + plans(1) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[FilterExecTransformerBase]) + } + assert( + plans(1) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ColumnarInputAdapter]) + + assert( + plans(2) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ColumnarInputAdapter]) + assert( + plans(2) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ColumnarInputAdapter]) + + assert( + plans(3) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[FilterExecTransformerBase]) + assert( + plans(3) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ProjectExecTransformer]) + }) + } +} +// scalastyle:off line.size.limit diff --git a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala index 0aa622466864e..38192a32b4655 100644 --- a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala +++ b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala @@ -63,6 +63,7 @@ class GlutenClickHouseTPCHColumnarShuffleParquetAQESuite assert(plans(2).metrics("numFiles").value === 1) assert(plans(2).metrics("pruningTime").value === -1) assert(plans(2).metrics("filesSize").value === 17777735) + assert(plans(2).metrics("outputRows").value === 600572) assert(plans(1).metrics("inputRows").value === 591673) assert(plans(1).metrics("resizeInputRows").value === 4) diff --git a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHParquetBucketSuite.scala b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHParquetBucketSuite.scala new file mode 100644 index 0000000000000..94675988a15c6 --- /dev/null +++ b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHParquetBucketSuite.scala @@ -0,0 +1,502 @@ +/* + * 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 io.glutenproject.execution + +import org.apache.spark.{SPARK_VERSION_SHORT, SparkConf} +import org.apache.spark.sql.execution.ColumnarInputAdapter +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper + +import org.apache.commons.io.FileUtils + +import java.io.File + +// Some sqls' line length exceeds 100 +// scalastyle:off line.size.limit + +class GlutenClickHouseTPCHParquetBucketSuite + extends GlutenClickHouseTPCHAbstractSuite + with AdaptiveSparkPlanHelper { + + override protected val resourcePath: String = + "../../../../gluten-core/src/test/resources/tpch-data" + + override protected val tablesPath: String = basePath + "/tpch-data" + override protected val tpchQueries: String = + rootPath + "../../../../gluten-core/src/test/resources/tpch-queries" + override protected val queriesResults: String = rootPath + "queries-output" + + protected val bucketTableResourcePath: String = rootPath + "tpch-data-bucket/parquet_bucket" + protected val bucketTableDataPath: String = basePath + "/tpch-parquet-bucket" + protected lazy val sparkVersion: String = { + val version = SPARK_VERSION_SHORT.split("\\.") + version(0) + "." + version(1) + } + + 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", "-1") // for test bucket join + .set("spark.sql.adaptive.enabled", "true") + .set("spark.gluten.sql.columnar.backend.ch.shuffle.hash.algorithm", "sparkMurmurHash3_32") + } + + override protected val createNullableTables = true + + override def beforeAll(): Unit = { + super.beforeAll() + FileUtils.copyDirectory(new File(bucketTableResourcePath), new File(bucketTableDataPath)) + createTPCHParquetBucketTables() + } + + protected def createTPCHParquetBucketTables(): Unit = { + + spark.sql(s""" + |CREATE DATABASE IF NOT EXISTS tpch_parquet_bucket + |""".stripMargin) + spark.sql("use tpch_parquet_bucket") + val customerData = bucketTableDataPath + "/customer" + spark.sql(s"DROP TABLE IF EXISTS customer") + spark.sql(s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS customer ( + | c_custkey bigint, + | c_name string, + | c_address string, + | c_nationkey bigint, + | c_phone string, + | c_acctbal double, + | c_mktsegment string, + | c_comment string) + | USING PARQUET + | LOCATION '$customerData' + | CLUSTERED BY (c_custkey) SORTED BY (c_custkey) INTO 2 BUCKETS; + |""".stripMargin) + + val lineitemData = bucketTableDataPath + "/lineitem" + spark.sql(s"DROP TABLE IF EXISTS lineitem") + spark.sql(s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS lineitem ( + | 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 PARQUET + | LOCATION '$lineitemData' + | CLUSTERED BY (l_orderkey) SORTED BY (l_shipdate, l_orderkey) INTO 2 BUCKETS; + |""".stripMargin) + + val nationData = bucketTableDataPath + "/nation" + spark.sql(s"DROP TABLE IF EXISTS nation") + spark.sql(s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS nation ( + | n_nationkey bigint, + | n_name string, + | n_regionkey bigint, + | n_comment string) + | USING PARQUET + | LOCATION '$nationData' + | CLUSTERED BY (n_nationkey) SORTED BY (n_nationkey) INTO 1 BUCKETS; + |""".stripMargin) + + val regionData = bucketTableDataPath + "/region" + spark.sql(s"DROP TABLE IF EXISTS region") + spark.sql(s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS region ( + | r_regionkey bigint, + | r_name string, + | r_comment string) + | USING PARQUET + | LOCATION '$regionData' + | CLUSTERED BY (r_regionkey) SORTED BY (r_regionkey) INTO 1 BUCKETS; + |""".stripMargin) + + val ordersData = bucketTableDataPath + "/orders" + spark.sql(s"DROP TABLE IF EXISTS orders") + spark.sql(s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS orders ( + | o_orderkey bigint, + | o_custkey bigint, + | o_orderstatus string, + | o_totalprice double, + | o_orderdate date, + | o_orderpriority string, + | o_clerk string, + | o_shippriority bigint, + | o_comment string) + | USING PARQUET + | LOCATION '$ordersData' + | CLUSTERED BY (o_orderkey) SORTED BY (o_orderkey, o_orderdate) INTO 2 BUCKETS; + |""".stripMargin) + + val partData = bucketTableDataPath + "/part" + spark.sql(s"DROP TABLE IF EXISTS part") + spark.sql(s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS part ( + | p_partkey bigint, + | p_name string, + | p_mfgr string, + | p_brand string, + | p_type string, + | p_size bigint, + | p_container string, + | p_retailprice double, + | p_comment string) + | USING PARQUET + | LOCATION '$partData' + | CLUSTERED BY (p_partkey) SORTED BY (p_partkey) INTO 2 BUCKETS; + |""".stripMargin) + + val partsuppData = bucketTableDataPath + "/partsupp" + spark.sql(s"DROP TABLE IF EXISTS partsupp") + spark.sql(s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS partsupp ( + | ps_partkey bigint, + | ps_suppkey bigint, + | ps_availqty bigint, + | ps_supplycost double, + | ps_comment string) + | USING PARQUET + | LOCATION '$partsuppData' + | CLUSTERED BY (ps_partkey) SORTED BY (ps_partkey) INTO 2 BUCKETS; + |""".stripMargin) + + val supplierData = bucketTableDataPath + "/supplier" + spark.sql(s"DROP TABLE IF EXISTS supplier") + spark.sql(s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS supplier ( + | s_suppkey bigint, + | s_name string, + | s_address string, + | s_nationkey bigint, + | s_phone string, + | s_acctbal double, + | s_comment string) + | USING PARQUET + | LOCATION '$supplierData' + | CLUSTERED BY (s_suppkey) SORTED BY (s_suppkey) INTO 1 BUCKETS; + |""".stripMargin) + + val result = spark + .sql(s""" + | show tables; + |""".stripMargin) + .collect() + assert(result.length == 8) + } + + test("TPCH Q1") { + compareTPCHQueryAgainstVanillaSpark( + 1, + tpchQueries, + df => { + 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("outputRows").value === 600572) + } + ) + } + + test("TPCH Q2") { + compareTPCHQueryAgainstVanillaSpark( + 2, + tpchQueries, + df => { + val plans = collect(df.queryExecution.executedPlan) { + case scanExec: BasicScanExecTransformer => scanExec + case joinExec: HashJoinLikeExecTransformer => joinExec + } + assert( + plans(3) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ColumnarInputAdapter]) + assert( + plans(3) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ColumnarInputAdapter]) + + // Check the bucket join + assert( + plans(4) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ProjectExecTransformer]) + assert( + plans(4) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[FilterExecTransformerBase]) + + assert( + plans(9) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ColumnarInputAdapter]) + if (sparkVersion.equals("3.2")) { + assert( + plans(9) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ColumnarInputAdapter]) + } else { + assert( + plans(9) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[FilterExecTransformerBase]) + } + + if (sparkVersion.equals("3.2")) { + assert(!(plans(11).asInstanceOf[FileSourceScanExecTransformer].bucketedScan)) + } else { + assert(plans(11).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) + } + assert(plans(11).metrics("numFiles").value === 1) + assert(plans(11).metrics("outputRows").value === 1000) + } + ) + } + + test("TPCH Q3") { + compareTPCHQueryAgainstVanillaSpark( + 3, + tpchQueries, + df => { + val plans = collect(df.queryExecution.executedPlan) { + case scanExec: BasicScanExecTransformer => scanExec + case joinExec: HashJoinLikeExecTransformer => joinExec + } + if (sparkVersion.equals("3.2")) { + assert( + plans(1) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ColumnarInputAdapter]) + } else { + assert( + plans(1) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ProjectExecTransformer]) + } + + assert( + plans(1) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ColumnarInputAdapter]) + + if (sparkVersion.equals("3.2")) { + assert(!(plans(2).asInstanceOf[FileSourceScanExecTransformer].bucketedScan)) + } else { + assert(plans(2).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) + } + assert(plans(2).metrics("numFiles").value === 4) + assert(plans(2).metrics("outputRows").value === 15000) + + assert(!(plans(3).asInstanceOf[FileSourceScanExecTransformer].bucketedScan)) + assert(plans(3).metrics("numFiles").value === 4) + assert(plans(3).metrics("outputRows").value === 150000) + } + ) + } + + test("TPCH Q4") { + compareTPCHQueryAgainstVanillaSpark( + 4, + tpchQueries, + df => { + val plans = collect(df.queryExecution.executedPlan) { + case scanExec: BasicScanExecTransformer => scanExec + case joinExec: HashJoinLikeExecTransformer => joinExec + } + // bucket join + assert( + plans(0) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ProjectExecTransformer]) + assert( + plans(0) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ProjectExecTransformer]) + + assert(plans(1).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) + assert(plans(1).metrics("numFiles").value === 4) + assert(plans(1).metrics("outputRows").value === 150000) + + assert(plans(2).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) + assert(plans(2).metrics("numFiles").value === 4) + assert(plans(2).metrics("outputRows").value === 600572) + } + ) + } + + test("TPCH Q6") { + compareTPCHQueryAgainstVanillaSpark( + 6, + tpchQueries, + df => { + 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("outputRows").value === 600572) + } + ) + } + + test("TPCH Q12") { + compareTPCHQueryAgainstVanillaSpark( + 12, + tpchQueries, + df => { + val plans = collect(df.queryExecution.executedPlan) { + case scanExec: BasicScanExecTransformer => scanExec + case joinExec: HashJoinLikeExecTransformer => joinExec + } + // bucket join + assert( + plans(0) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[FilterExecTransformerBase]) + assert( + plans(0) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ProjectExecTransformer]) + + assert(plans(1).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) + assert(plans(1).metrics("numFiles").value === 4) + assert(plans(1).metrics("outputRows").value === 150000) + + assert(plans(2).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) + assert(plans(2).metrics("numFiles").value === 4) + assert(plans(2).metrics("outputRows").value === 600572) + } + ) + } + + test("TPCH Q18") { + compareTPCHQueryAgainstVanillaSpark( + 18, + tpchQueries, + df => { + val plans = collect(df.queryExecution.executedPlan) { + case joinExec: HashJoinLikeExecTransformer => joinExec + } + // bucket join + assert( + plans(2) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[FilterExecTransformerBase]) + assert( + plans(2) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ProjectExecTransformer]) + // bucket join + assert( + plans(3) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[FilterExecTransformerBase]) + assert( + plans(3) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ProjectExecTransformer]) + } + ) + } + + test("TPCH Q20") { + compareTPCHQueryAgainstVanillaSpark( + 20, + tpchQueries, + df => { + val plans = collect(df.queryExecution.executedPlan) { + case joinExec: HashJoinLikeExecTransformer => joinExec + } + if (sparkVersion.equals("3.2")) { + assert( + plans(1) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ColumnarInputAdapter]) + } else { + assert( + plans(1) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[FilterExecTransformerBase]) + } + assert( + plans(1) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ColumnarInputAdapter]) + + assert( + plans(2) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ColumnarInputAdapter]) + assert( + plans(2) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ColumnarInputAdapter]) + + assert( + plans(3) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[FilterExecTransformerBase]) + assert( + plans(3) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ProjectExecTransformer]) + } + ) + } +} +// scalastyle:on line.size.limit diff --git a/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/TransformerApiImpl.scala b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/TransformerApiImpl.scala index b2e4d9b6b7891..18bc2f61b1b6f 100644 --- a/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/TransformerApiImpl.scala +++ b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/TransformerApiImpl.scala @@ -51,6 +51,7 @@ class TransformerApiImpl extends TransformerApi with Logging { relation: HadoopFsRelation, selectedPartitions: Array[PartitionDirectory], output: Seq[Attribute], + bucketedScan: Boolean, optionalBucketSet: Option[BitSet], optionalNumCoalescedBuckets: Option[Int], disableBucketedScan: Boolean): Seq[InputPartition] = { @@ -58,6 +59,7 @@ class TransformerApiImpl extends TransformerApi with Logging { relation, selectedPartitions, output, + bucketedScan, optionalBucketSet, optionalNumCoalescedBuckets, disableBucketedScan) diff --git a/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/VeloxBackend.scala b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/VeloxBackend.scala index e94efb536c76d..40d42b90873ec 100644 --- a/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/VeloxBackend.scala +++ b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/VeloxBackend.scala @@ -306,8 +306,6 @@ object BackendSettings extends BackendSettingsApi { UDFResolver.resolveUdfConf(nativeConf) } - override def supportBucketScan(): Boolean = true - override def insertPostProjectForGenerate(): Boolean = true override def skipNativeCtas(ctas: CreateDataSourceTableAsSelectCommand): Boolean = true diff --git a/gluten-core/src/main/scala/io/glutenproject/backendsapi/BackendSettingsApi.scala b/gluten-core/src/main/scala/io/glutenproject/backendsapi/BackendSettingsApi.scala index 502ccc1669e31..fcd1bbfe84533 100644 --- a/gluten-core/src/main/scala/io/glutenproject/backendsapi/BackendSettingsApi.scala +++ b/gluten-core/src/main/scala/io/glutenproject/backendsapi/BackendSettingsApi.scala @@ -98,8 +98,6 @@ trait BackendSettingsApi { /** Apply necessary conversions before passing to native side */ def resolveNativeConf(nativeConf: java.util.Map[String, String]): Unit = {} - def supportBucketScan(): Boolean = false - def insertPostProjectForGenerate(): Boolean = false def skipNativeCtas(ctas: CreateDataSourceTableAsSelectCommand): Boolean = false diff --git a/gluten-core/src/main/scala/io/glutenproject/backendsapi/TransformerApi.scala b/gluten-core/src/main/scala/io/glutenproject/backendsapi/TransformerApi.scala index 2a42524910c85..58350b9f9b078 100644 --- a/gluten-core/src/main/scala/io/glutenproject/backendsapi/TransformerApi.scala +++ b/gluten-core/src/main/scala/io/glutenproject/backendsapi/TransformerApi.scala @@ -45,6 +45,7 @@ trait TransformerApi { relation: HadoopFsRelation, selectedPartitions: Array[PartitionDirectory], output: Seq[Attribute], + bucketedScan: Boolean, optionalBucketSet: Option[BitSet], optionalNumCoalescedBuckets: Option[Int], disableBucketedScan: Boolean): Seq[InputPartition] diff --git a/gluten-core/src/main/scala/io/glutenproject/execution/FileSourceScanExecTransformer.scala b/gluten-core/src/main/scala/io/glutenproject/execution/FileSourceScanExecTransformer.scala index d66ca52f24313..ad1252df65ea5 100644 --- a/gluten-core/src/main/scala/io/glutenproject/execution/FileSourceScanExecTransformer.scala +++ b/gluten-core/src/main/scala/io/glutenproject/execution/FileSourceScanExecTransformer.scala @@ -84,6 +84,7 @@ class FileSourceScanExecTransformer( relation, dynamicallySelectedPartitions, output, + bucketedScan, optionalBucketSet, optionalNumCoalescedBuckets, disableBucketedScan) @@ -107,12 +108,6 @@ class FileSourceScanExecTransformer( override def hashCode(): Int = super.hashCode() override protected def doValidateInternal(): ValidationResult = { - // Bucketing table has `bucketId` in filename, should apply this in backends - // TODO Support bucketed scan - if (bucketedScan && !BackendsApiManager.getSettings.supportBucketScan()) { - throw new UnsupportedOperationException("Bucketed scan is unsupported for now.") - } - if (hasMetadataColumns) { return ValidationResult.notOk(s"Unsupported metadataColumns scan in native.") } diff --git a/gluten-core/src/main/scala/io/glutenproject/execution/HashJoinExecTransformer.scala b/gluten-core/src/main/scala/io/glutenproject/execution/HashJoinExecTransformer.scala index b95aa765d3148..67bff42ac07d3 100644 --- a/gluten-core/src/main/scala/io/glutenproject/execution/HashJoinExecTransformer.scala +++ b/gluten-core/src/main/scala/io/glutenproject/execution/HashJoinExecTransformer.scala @@ -219,27 +219,29 @@ trait HashJoinLikeExecTransformer override def doTransform(substraitContext: SubstraitContext): TransformContext = { - def transformAndGetOutput(plan: SparkPlan): (RelNode, Seq[Attribute], Boolean) = { + def transformAndGetOutput(plan: SparkPlan): (RelNode, Seq[Attribute], Boolean, JLong) = { plan match { case p: TransformSupport => val transformContext = p.doTransform(substraitContext) - (transformContext.root, transformContext.outputAttributes, false) + (transformContext.root, transformContext.outputAttributes, false, -1L) case _ => val readRel = RelBuilder.makeReadRel( plan.output.asJava, substraitContext, -1 ) /* A special handling in Join to delay the rel registration. */ - (readRel, plan.output, true) + // Make sure create a new read relId for the stream side first + // before the one of the build side, when there is no shuffle on the build side + (readRel, plan.output, true, substraitContext.nextRelId()) } } val joinParams = new JoinParams - val (inputStreamedRelNode, inputStreamedOutput, isStreamedReadRel) = + val (inputStreamedRelNode, inputStreamedOutput, isStreamedReadRel, streamdReadRelId) = transformAndGetOutput(streamedPlan) joinParams.isStreamedReadRel = isStreamedReadRel - val (inputBuildRelNode, inputBuildOutput, isBuildReadRel) = + val (inputBuildRelNode, inputBuildOutput, isBuildReadRel, buildReadRelId) = transformAndGetOutput(buildPlan) joinParams.isBuildReadRel = isBuildReadRel @@ -248,10 +250,10 @@ trait HashJoinLikeExecTransformer // Register the ReadRel to correct operator Id. if (joinParams.isStreamedReadRel) { - substraitContext.registerRelToOperator(operatorId) + substraitContext.registerRelToOperator(operatorId, streamdReadRelId) } if (joinParams.isBuildReadRel) { - substraitContext.registerRelToOperator(operatorId) + substraitContext.registerRelToOperator(operatorId, buildReadRelId) } if (JoinUtils.preProjectionNeeded(streamedKeyExprs)) { diff --git a/gluten-core/src/main/scala/io/glutenproject/substrait/SubstraitContext.scala b/gluten-core/src/main/scala/io/glutenproject/substrait/SubstraitContext.scala index ce37cbfc96f96..f02a8338c7803 100644 --- a/gluten-core/src/main/scala/io/glutenproject/substrait/SubstraitContext.scala +++ b/gluten-core/src/main/scala/io/glutenproject/substrait/SubstraitContext.scala @@ -164,6 +164,25 @@ class SubstraitContext extends Serializable { relId += 1 } + /** Register a specified rel to certain operator id. */ + def registerRelToOperator(operatorId: JLong, specifiedRedId: JLong): Unit = { + if (operatorToRelsMap.containsKey(operatorId)) { + val rels = operatorToRelsMap.get(operatorId) + rels.add(specifiedRedId) + } else { + val rels = new JArrayList[JLong]() + rels.add(specifiedRedId) + operatorToRelsMap.put(operatorId, rels) + } + } + + /** Add the relId and register to operator later */ + def nextRelId(): JLong = { + val id = this.relId + this.relId += 1 + id + } + /** * Register empty rel list to certain operator id. Used when the computing of a Spark transformer * is omitted. diff --git a/gluten-core/src/main/scala/io/glutenproject/utils/InputPartitionsUtil.scala b/gluten-core/src/main/scala/io/glutenproject/utils/InputPartitionsUtil.scala index 7fbb6beb746d6..37ee1eaf1fc76 100644 --- a/gluten-core/src/main/scala/io/glutenproject/utils/InputPartitionsUtil.scala +++ b/gluten-core/src/main/scala/io/glutenproject/utils/InputPartitionsUtil.scala @@ -29,24 +29,12 @@ case class InputPartitionsUtil( relation: HadoopFsRelation, selectedPartitions: Array[PartitionDirectory], output: Seq[Attribute], + bucketedScan: Boolean, optionalBucketSet: Option[BitSet], optionalNumCoalescedBuckets: Option[Int], disableBucketedScan: Boolean) extends Logging { - private val bucketedScan: Boolean = { - if ( - relation.sparkSession.sessionState.conf.bucketingEnabled && relation.bucketSpec.isDefined - && !disableBucketedScan - ) { - val spec = relation.bucketSpec.get - val bucketColumns = spec.bucketColumnNames.flatMap(n => toAttribute(n)) - bucketColumns.size == spec.bucketColumnNames.size - } else { - false - } - } - def genInputPartitionSeq(): Seq[InputPartition] = { if (bucketedScan) { genBucketedInputPartitionSeq() diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala index dfbeb746b7930..2bd75448498ab 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala @@ -53,6 +53,7 @@ class TestFileSourceScanExecTransformer( relation, selectedPartitions, output, + bucketedScan, optionalBucketSet, optionalNumCoalescedBuckets, disableBucketedScan) diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala index 2b46705126933..36fe19bd70399 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala @@ -52,6 +52,7 @@ class TestFileSourceScanExecTransformer( relation, selectedPartitions, output, + bucketedScan, optionalBucketSet, optionalNumCoalescedBuckets, disableBucketedScan) diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala index 2b46705126933..36fe19bd70399 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala @@ -52,6 +52,7 @@ class TestFileSourceScanExecTransformer( relation, selectedPartitions, output, + bucketedScan, optionalBucketSet, optionalNumCoalescedBuckets, disableBucketedScan)