diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergFileWriterFactory.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergFileWriterFactory.java index 4fd1386c0095..423173a10fe5 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergFileWriterFactory.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergFileWriterFactory.java @@ -36,6 +36,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapred.JobConf; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetricsConfig; import org.apache.iceberg.Schema; import org.apache.iceberg.types.Types; import org.weakref.jmx.Managed; @@ -110,13 +111,15 @@ public IcebergFileWriter createFileWriter( JobConf jobConf, ConnectorSession session, HdfsContext hdfsContext, - FileFormat fileFormat) + FileFormat fileFormat, + MetricsConfig metricsConfig) { switch (fileFormat) { case PARQUET: + // TODO use metricsConfig return createParquetWriter(outputPath, icebergSchema, jobConf, session, hdfsContext); case ORC: - return createOrcWriter(outputPath, icebergSchema, jobConf, session); + return createOrcWriter(metricsConfig, outputPath, icebergSchema, jobConf, session); default: throw new TrinoException(NOT_SUPPORTED, "File format not supported for Iceberg: " + fileFormat); } @@ -170,6 +173,7 @@ private IcebergFileWriter createParquetWriter( } private IcebergFileWriter createOrcWriter( + MetricsConfig metricsConfig, Path outputPath, Schema icebergSchema, JobConf jobConf, @@ -210,6 +214,7 @@ private IcebergFileWriter createOrcWriter( } return new IcebergOrcFileWriter( + metricsConfig, icebergSchema, orcDataSink, rollbackAction, diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergOrcFileWriter.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergOrcFileWriter.java index 7ae8b3bc8ec9..5097f5af7a4c 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergOrcFileWriter.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergOrcFileWriter.java @@ -37,9 +37,15 @@ import io.trino.plugin.hive.orc.OrcFileWriter; import io.trino.spi.type.Type; import org.apache.iceberg.Metrics; +import org.apache.iceberg.MetricsConfig; +import org.apache.iceberg.MetricsModes; +import org.apache.iceberg.MetricsUtil; import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.Literal; import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.BinaryUtil; +import org.apache.iceberg.util.UnicodeUtil; import java.math.BigDecimal; import java.nio.ByteBuffer; @@ -65,8 +71,10 @@ public class IcebergOrcFileWriter { private final Schema icebergSchema; private final ColumnMetadata orcColumns; + private final MetricsConfig metricsConfig; public IcebergOrcFileWriter( + MetricsConfig metricsConfig, Schema icebergSchema, OrcDataSink orcDataSink, Callable rollbackAction, @@ -83,16 +91,17 @@ public IcebergOrcFileWriter( { super(orcDataSink, WriterKind.INSERT, NO_ACID_TRANSACTION, false, OptionalInt.empty(), rollbackAction, columnNames, fileColumnTypes, fileColumnOrcTypes, compression, options, fileInputColumnIndexes, metadata, validationInputFactory, validationMode, stats); this.icebergSchema = requireNonNull(icebergSchema, "icebergSchema is null"); + this.metricsConfig = requireNonNull(metricsConfig, "metricsConfig is null"); orcColumns = fileColumnOrcTypes; } @Override public Metrics getMetrics() { - return computeMetrics(icebergSchema, orcColumns, orcWriter.getFileRowCount(), orcWriter.getFileStats()); + return computeMetrics(metricsConfig, icebergSchema, orcColumns, orcWriter.getFileRowCount(), orcWriter.getFileStats()); } - private static Metrics computeMetrics(Schema icebergSchema, ColumnMetadata orcColumns, long fileRowCount, Optional> columnStatistics) + private static Metrics computeMetrics(MetricsConfig metricsConfig, Schema icebergSchema, ColumnMetadata orcColumns, long fileRowCount, Optional> columnStatistics) { if (columnStatistics.isEmpty()) { return new Metrics(fileRowCount, null, null, null, null, null); @@ -118,15 +127,22 @@ private static Metrics computeMetrics(Schema icebergSchema, ColumnMetadata { - lowerBoundsBuilder.put(icebergId, minMax.getMin()); - upperBoundsBuilder.put(icebergId, minMax.getMax()); - }); + + if (!metricsMode.equals(MetricsModes.Counts.get())) { + toIcebergMinMax(orcColumnStats, icebergField.type(), metricsMode).ifPresent(minMax -> { + lowerBoundsBuilder.put(icebergId, minMax.getMin()); + upperBoundsBuilder.put(icebergId, minMax.getMax()); + }); + } } Map valueCounts = valueCountsBuilder.buildOrThrow(); Map nullCounts = nullCountsBuilder.buildOrThrow(); @@ -178,13 +194,13 @@ private static int getIcebergId(OrcType orcColumn) return Integer.parseInt(icebergId); } - private static Optional toIcebergMinMax(ColumnStatistics orcColumnStats, org.apache.iceberg.types.Type icebergType) + private static Optional toIcebergMinMax(ColumnStatistics orcColumnStats, org.apache.iceberg.types.Type icebergType, MetricsModes.MetricsMode metricsModes) { BooleanStatistics booleanStatistics = orcColumnStats.getBooleanStatistics(); if (booleanStatistics != null) { boolean hasTrueValues = booleanStatistics.getTrueValueCount() != 0; boolean hasFalseValues = orcColumnStats.getNumberOfValues() != booleanStatistics.getTrueValueCount(); - return Optional.of(new IcebergMinMax(icebergType, !hasFalseValues, hasTrueValues)); + return Optional.of(new IcebergMinMax(icebergType, !hasFalseValues, hasTrueValues, metricsModes)); } IntegerStatistics integerStatistics = orcColumnStats.getIntegerStatistics(); @@ -198,7 +214,7 @@ private static Optional toIcebergMinMax(ColumnStatistics orcColum min = toIntExact((Long) min); max = toIntExact((Long) max); } - return Optional.of(new IcebergMinMax(icebergType, min, max)); + return Optional.of(new IcebergMinMax(icebergType, min, max, metricsModes)); } DoubleStatistics doubleStatistics = orcColumnStats.getDoubleStatistics(); if (doubleStatistics != null) { @@ -211,7 +227,7 @@ private static Optional toIcebergMinMax(ColumnStatistics orcColum min = ((Double) min).floatValue(); max = ((Double) max).floatValue(); } - return Optional.of(new IcebergMinMax(icebergType, min, max)); + return Optional.of(new IcebergMinMax(icebergType, min, max, metricsModes)); } StringStatistics stringStatistics = orcColumnStats.getStringStatistics(); if (stringStatistics != null) { @@ -220,7 +236,7 @@ private static Optional toIcebergMinMax(ColumnStatistics orcColum if (min == null || max == null) { return Optional.empty(); } - return Optional.of(new IcebergMinMax(icebergType, min.toStringUtf8(), max.toStringUtf8())); + return Optional.of(new IcebergMinMax(icebergType, min.toStringUtf8(), max.toStringUtf8(), metricsModes)); } DateStatistics dateStatistics = orcColumnStats.getDateStatistics(); if (dateStatistics != null) { @@ -229,7 +245,7 @@ private static Optional toIcebergMinMax(ColumnStatistics orcColum if (min == null || max == null) { return Optional.empty(); } - return Optional.of(new IcebergMinMax(icebergType, min, max)); + return Optional.of(new IcebergMinMax(icebergType, min, max, metricsModes)); } DecimalStatistics decimalStatistics = orcColumnStats.getDecimalStatistics(); if (decimalStatistics != null) { @@ -240,7 +256,7 @@ private static Optional toIcebergMinMax(ColumnStatistics orcColum } min = min.setScale(((Types.DecimalType) icebergType).scale()); max = max.setScale(((Types.DecimalType) icebergType).scale()); - return Optional.of(new IcebergMinMax(icebergType, min, max)); + return Optional.of(new IcebergMinMax(icebergType, min, max, metricsModes)); } TimestampStatistics timestampStatistics = orcColumnStats.getTimestampStatistics(); if (timestampStatistics != null) { @@ -251,7 +267,7 @@ private static Optional toIcebergMinMax(ColumnStatistics orcColum } // Since ORC timestamp statistics are truncated to millisecond precision, this can cause some column values to fall outside the stats range. // We are appending 999 microseconds to account for the fact that Trino ORC writer truncates timestamps. - return Optional.of(new IcebergMinMax(icebergType, min * MICROSECONDS_PER_MILLISECOND, (max * MICROSECONDS_PER_MILLISECOND) + (MICROSECONDS_PER_MILLISECOND - 1))); + return Optional.of(new IcebergMinMax(icebergType, min * MICROSECONDS_PER_MILLISECOND, (max * MICROSECONDS_PER_MILLISECOND) + (MICROSECONDS_PER_MILLISECOND - 1), metricsModes)); } return Optional.empty(); } @@ -261,10 +277,33 @@ private static class IcebergMinMax private ByteBuffer min; private ByteBuffer max; - private IcebergMinMax(org.apache.iceberg.types.Type type, Object min, Object max) + private IcebergMinMax(org.apache.iceberg.types.Type type, Object min, Object max, MetricsModes.MetricsMode metricsMode) { - this.min = Conversions.toByteBuffer(type, min); - this.max = Conversions.toByteBuffer(type, max); + if (metricsMode instanceof MetricsModes.Full) { + this.min = Conversions.toByteBuffer(type, min); + this.max = Conversions.toByteBuffer(type, max); + } + else if (metricsMode instanceof MetricsModes.Truncate) { + MetricsModes.Truncate truncateMode = (MetricsModes.Truncate) metricsMode; + int truncateLength = truncateMode.length(); + switch (type.typeId()) { + case STRING: + this.min = UnicodeUtil.truncateStringMin(Literal.of((CharSequence) min), truncateLength).toByteBuffer(); + this.max = UnicodeUtil.truncateStringMax(Literal.of((CharSequence) max), truncateLength).toByteBuffer(); + break; + case FIXED: + case BINARY: + this.min = BinaryUtil.truncateBinaryMin(Literal.of((ByteBuffer) min), truncateLength).toByteBuffer(); + this.max = BinaryUtil.truncateBinaryMax(Literal.of((ByteBuffer) max), truncateLength).toByteBuffer(); + break; + default: + this.min = Conversions.toByteBuffer(type, min); + this.max = Conversions.toByteBuffer(type, max); + } + } + else { + throw new UnsupportedOperationException("Unsupported metrics mode for Iceberg Max/Min Bound: " + metricsMode); + } } public ByteBuffer getMin() diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSink.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSink.java index 6d6cd4cc903a..57af22553085 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSink.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSink.java @@ -42,6 +42,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapred.JobConf; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetricsConfig; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -95,6 +96,7 @@ public class IcebergPageSink private final JsonCodec jsonCodec; private final ConnectorSession session; private final FileFormat fileFormat; + private final MetricsConfig metricsConfig; private final PagePartitioner pagePartitioner; private final List writers = new ArrayList<>(); @@ -115,6 +117,7 @@ public IcebergPageSink( JsonCodec jsonCodec, ConnectorSession session, FileFormat fileFormat, + Map storageProperties, int maxOpenWriters) { requireNonNull(inputColumns, "inputColumns is null"); @@ -128,6 +131,7 @@ public IcebergPageSink( this.jsonCodec = requireNonNull(jsonCodec, "jsonCodec is null"); this.session = requireNonNull(session, "session is null"); this.fileFormat = requireNonNull(fileFormat, "fileFormat is null"); + this.metricsConfig = MetricsConfig.fromProperties(requireNonNull(storageProperties, "storageProperties is null")); this.maxOpenWriters = maxOpenWriters; this.pagePartitioner = new PagePartitioner(pageIndexerFactory, toPartitionColumns(inputColumns, partitionSpec)); } @@ -312,7 +316,8 @@ private WriteContext createWriter(Optional partitionData) jobConf, session, hdfsContext, - fileFormat); + fileFormat, + metricsConfig); return new WriteContext(writer, outputPath, partitionData); } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSinkProvider.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSinkProvider.java index 35c5c6839cbb..a5a76c6a8f1d 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSinkProvider.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSinkProvider.java @@ -94,6 +94,7 @@ private ConnectorPageSink createPageSink(ConnectorSession session, IcebergWritab jsonCodec, session, tableHandle.getFileFormat(), + tableHandle.getStorageProperties(), maxOpenPartitions); } @@ -121,6 +122,7 @@ public ConnectorPageSink createPageSink(ConnectorTransactionHandle transactionHa jsonCodec, session, optimizeHandle.getFileFormat(), + optimizeHandle.getTableStorageProperties(), maxOpenPartitions); } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergOrcMetricsCollection.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergOrcMetricsCollection.java index 1717606b0728..c08849fad82e 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergOrcMetricsCollection.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergOrcMetricsCollection.java @@ -27,16 +27,22 @@ import io.trino.plugin.hive.metastore.MetastoreConfig; import io.trino.plugin.hive.metastore.file.FileHiveMetastore; import io.trino.plugin.hive.metastore.file.FileHiveMetastoreConfig; +import io.trino.plugin.iceberg.catalog.IcebergTableOperationsProvider; +import io.trino.plugin.iceberg.catalog.file.FileMetastoreTableOperationsProvider; import io.trino.plugin.tpch.TpchPlugin; +import io.trino.spi.connector.SchemaTableName; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.DistributedQueryRunner; import io.trino.testing.MaterializedResult; import io.trino.testing.MaterializedRow; import io.trino.testing.QueryRunner; +import io.trino.testing.TestingConnectorSession; import org.apache.iceberg.FileContent; +import org.apache.iceberg.Table; import org.testng.annotations.Test; import java.io.File; +import java.util.List; import java.util.Map; import java.util.Optional; @@ -51,6 +57,10 @@ public class TestIcebergOrcMetricsCollection extends AbstractTestQueryFramework { + private HiveMetastore metastore; + private HdfsEnvironment hdfsEnvironment; + private IcebergTableOperationsProvider tableOperationsProvider; + @Override protected QueryRunner createQueryRunner() throws Exception @@ -71,15 +81,16 @@ protected QueryRunner createQueryRunner() HdfsConfig hdfsConfig = new HdfsConfig(); HdfsConfiguration hdfsConfiguration = new HiveHdfsConfiguration(new HdfsConfigurationInitializer(hdfsConfig), ImmutableSet.of()); - HdfsEnvironment hdfsEnvironment = new HdfsEnvironment(hdfsConfiguration, hdfsConfig, new NoHdfsAuthentication()); + hdfsEnvironment = new HdfsEnvironment(hdfsConfiguration, hdfsConfig, new NoHdfsAuthentication()); - HiveMetastore metastore = new FileHiveMetastore( + metastore = new FileHiveMetastore( new NodeVersion("test_version"), hdfsEnvironment, new MetastoreConfig(), new FileHiveMetastoreConfig() .setCatalogDirectory(baseDir.toURI().toString()) .setMetastoreUser("test")); + tableOperationsProvider = new FileMetastoreTableOperationsProvider(new HdfsFileIoProvider(hdfsEnvironment)); queryRunner.installPlugin(new TestingIcebergPlugin(Optional.of(metastore), Optional.empty())); queryRunner.createCatalog("iceberg", "iceberg"); @@ -92,6 +103,115 @@ protected QueryRunner createQueryRunner() return queryRunner; } + @Test + public void testMetrics() + { + assertUpdate("create table no_metrics (c1 varchar, c2 varchar)"); + Table table = IcebergUtil.loadIcebergTable(metastore, tableOperationsProvider, TestingConnectorSession.SESSION, + new SchemaTableName("test_schema", "no_metrics")); + // skip metrics for all columns + table.updateProperties().set("write.metadata.metrics.default", "none").commit(); + // add one row + assertUpdate("insert into no_metrics values ('abcd', 'a')", 1); + List materializedRows = computeActual("select * from \"no_metrics$files\"").getMaterializedRows(); + DataFileRecord datafile = toDataFileRecord(materializedRows.get(0)); + assertEquals(datafile.getRecordCount(), 1); + assertNull(datafile.getValueCounts()); + assertNull(datafile.getNullValueCounts()); + assertNull(datafile.getUpperBounds()); + assertNull(datafile.getLowerBounds()); + assertNull(datafile.getColumnSizes()); + + // keep c1 metrics + assertUpdate("create table c1_metrics (c1 varchar, c2 varchar)"); + table = IcebergUtil.loadIcebergTable(metastore, tableOperationsProvider, TestingConnectorSession.SESSION, + new SchemaTableName("test_schema", "c1_metrics")); + table.updateProperties() + .set("write.metadata.metrics.default", "none") + .set("write.metadata.metrics.column.c1", "full") + .commit(); + + assertUpdate("insert into c1_metrics values ('b', 'a')", 1); + materializedRows = computeActual("select * from \"c1_metrics$files\"").getMaterializedRows(); + datafile = toDataFileRecord(materializedRows.get(0)); + assertEquals(datafile.getRecordCount(), 1); + assertEquals(datafile.getValueCounts().size(), 1); + assertEquals(datafile.getNullValueCounts().size(), 1); + assertEquals(datafile.getUpperBounds().size(), 1); + assertEquals(datafile.getLowerBounds().size(), 1); + + // set c1 metrics mode to count + assertUpdate("create table c1_metrics_count (c1 varchar, c2 varchar)"); + table = IcebergUtil.loadIcebergTable(metastore, tableOperationsProvider, TestingConnectorSession.SESSION, + new SchemaTableName("test_schema", "c1_metrics_count")); + table.updateProperties() + .set("write.metadata.metrics.default", "none") + .set("write.metadata.metrics.column.c1", "counts") + .commit(); + + assertUpdate("insert into c1_metrics_count values ('b', 'a')", 1); + materializedRows = computeActual("select * from \"c1_metrics_count$files\"").getMaterializedRows(); + datafile = toDataFileRecord(materializedRows.get(0)); + assertEquals(datafile.getRecordCount(), 1); + assertEquals(datafile.getValueCounts().size(), 1); + assertEquals(datafile.getNullValueCounts().size(), 1); + assertNull(datafile.getUpperBounds()); + assertNull(datafile.getLowerBounds()); + + // set c1 metrics mode to truncate(10) + assertUpdate("create table c1_metrics_truncate (c1 varchar, c2 varchar)"); + table = IcebergUtil.loadIcebergTable(metastore, tableOperationsProvider, TestingConnectorSession.SESSION, + new SchemaTableName("test_schema", "c1_metrics_truncate")); + table.updateProperties() + .set("write.metadata.metrics.default", "none") + .set("write.metadata.metrics.column.c1", "truncate(10)") + .commit(); + + assertUpdate("insert into c1_metrics_truncate values ('abcaabcaabcaabca', 'a')", 1); + materializedRows = computeActual("select * from \"c1_metrics_truncate$files\"").getMaterializedRows(); + datafile = toDataFileRecord(materializedRows.get(0)); + assertEquals(datafile.getRecordCount(), 1); + assertEquals(datafile.getValueCounts().size(), 1); + assertEquals(datafile.getNullValueCounts().size(), 1); + datafile.getUpperBounds().forEach((k, v) -> { + assertEquals(v.length(), 10); }); + datafile.getLowerBounds().forEach((k, v) -> { + assertEquals(v.length(), 10); }); + + // keep both c1 and c2 metrics + assertUpdate("create table c_metrics (c1 varchar, c2 varchar)"); + table = IcebergUtil.loadIcebergTable(metastore, tableOperationsProvider, TestingConnectorSession.SESSION, + new SchemaTableName("test_schema", "c_metrics")); + table.updateProperties() + .set("write.metadata.metrics.column.c1", "full") + .set("write.metadata.metrics.column.c2", "full") + .commit(); + assertUpdate("insert into c_metrics values ('b', 'a')", 1); + materializedRows = computeActual("select * from \"c_metrics$files\"").getMaterializedRows(); + datafile = toDataFileRecord(materializedRows.get(0)); + assertEquals(datafile.getRecordCount(), 1); + assertEquals(datafile.getValueCounts().size(), 2); + assertEquals(datafile.getNullValueCounts().size(), 2); + assertEquals(datafile.getUpperBounds().size(), 2); + assertEquals(datafile.getLowerBounds().size(), 2); + + // keep all metrics + assertUpdate("create table metrics (c1 varchar, c2 varchar)"); + table = IcebergUtil.loadIcebergTable(metastore, tableOperationsProvider, TestingConnectorSession.SESSION, + new SchemaTableName("test_schema", "metrics")); + table.updateProperties() + .set("write.metadata.metrics.default", "full") + .commit(); + assertUpdate("insert into metrics values ('b', 'a')", 1); + materializedRows = computeActual("select * from \"metrics$files\"").getMaterializedRows(); + datafile = toDataFileRecord(materializedRows.get(0)); + assertEquals(datafile.getRecordCount(), 1); + assertEquals(datafile.getValueCounts().size(), 2); + assertEquals(datafile.getNullValueCounts().size(), 2); + assertEquals(datafile.getUpperBounds().size(), 2); + assertEquals(datafile.getLowerBounds().size(), 2); + } + @Test public void testBasic() { @@ -121,27 +241,27 @@ public void testBasic() // Check per-column lower bound Map lowerBounds = datafile.getLowerBounds(); - assertQuery("SELECT min(orderkey) FROM tpch.tiny.orders", "VALUES " + lowerBounds.get(1)); - assertQuery("SELECT min(custkey) FROM tpch.tiny.orders", "VALUES " + lowerBounds.get(2)); - assertQuery("SELECT min(orderstatus) FROM tpch.tiny.orders", "VALUES '" + lowerBounds.get(3) + "'"); - assertQuery("SELECT min(totalprice) FROM tpch.tiny.orders", "VALUES " + lowerBounds.get(4)); - assertQuery("SELECT min(orderdate) FROM tpch.tiny.orders", "VALUES DATE '" + lowerBounds.get(5) + "'"); - assertQuery("SELECT min(orderpriority) FROM tpch.tiny.orders", "VALUES '" + lowerBounds.get(6) + "'"); - assertQuery("SELECT min(clerk) FROM tpch.tiny.orders", "VALUES '" + lowerBounds.get(7) + "'"); - assertQuery("SELECT min(shippriority) FROM tpch.tiny.orders", "VALUES " + lowerBounds.get(8)); - assertQuery("SELECT min(comment) FROM tpch.tiny.orders", "VALUES '" + lowerBounds.get(9) + "'"); + assertEquals(lowerBounds.get(1), "1"); + assertEquals(lowerBounds.get(2), "1"); + assertEquals(lowerBounds.get(3), "F"); + assertEquals(lowerBounds.get(4), "874.89"); + assertEquals(lowerBounds.get(5), "1992-01-01"); + assertEquals(lowerBounds.get(6), "1-URGENT"); + assertEquals(lowerBounds.get(7), "Clerk#000000001"); + assertEquals(lowerBounds.get(8), "0"); + assertEquals(lowerBounds.get(9), " about the accou"); // Check per-column upper bound Map upperBounds = datafile.getUpperBounds(); - assertQuery("SELECT max(orderkey) FROM tpch.tiny.orders", "VALUES " + upperBounds.get(1)); - assertQuery("SELECT max(custkey) FROM tpch.tiny.orders", "VALUES " + upperBounds.get(2)); - assertQuery("SELECT max(orderstatus) FROM tpch.tiny.orders", "VALUES '" + upperBounds.get(3) + "'"); - assertQuery("SELECT max(totalprice) FROM tpch.tiny.orders", "VALUES " + upperBounds.get(4)); - assertQuery("SELECT max(orderdate) FROM tpch.tiny.orders", "VALUES DATE '" + upperBounds.get(5) + "'"); - assertQuery("SELECT max(orderpriority) FROM tpch.tiny.orders", "VALUES '" + upperBounds.get(6) + "'"); - assertQuery("SELECT max(clerk) FROM tpch.tiny.orders", "VALUES '" + upperBounds.get(7) + "'"); - assertQuery("SELECT max(shippriority) FROM tpch.tiny.orders", "VALUES " + upperBounds.get(8)); - assertQuery("SELECT max(comment) FROM tpch.tiny.orders", "VALUES '" + upperBounds.get(9) + "'"); + assertEquals(upperBounds.get(1), "60000"); + assertEquals(upperBounds.get(2), "1499"); + assertEquals(upperBounds.get(3), "P"); + assertEquals(upperBounds.get(4), "466001.28"); + assertEquals(upperBounds.get(5), "1998-08-02"); + assertEquals(upperBounds.get(6), "5-LOW"); + assertEquals(upperBounds.get(7), "Clerk#000001000"); + assertEquals(upperBounds.get(8), "0"); + assertEquals(upperBounds.get(9), "zzle. carefully!"); assertUpdate("DROP TABLE orders"); }