From 7a1858d796a0119fae717e5025c2de4f755b34a3 Mon Sep 17 00:00:00 2001 From: pvary Date: Tue, 5 Jan 2021 18:22:43 +0100 Subject: [PATCH 01/12] Hive: Fix minor issues issues from #1917 (#2029) --- .../iceberg/mr/hive/HiveIcebergMetaHook.java | 26 ++++++++----------- 1 file changed, 11 insertions(+), 15 deletions(-) diff --git a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java index 011178a99903..6526d8820ab1 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java +++ b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java @@ -19,7 +19,6 @@ package org.apache.iceberg.mr.hive; -import java.util.ArrayList; import java.util.List; import java.util.Properties; import java.util.Set; @@ -45,6 +44,7 @@ import org.apache.iceberg.mr.InputFormatConfig; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -204,15 +204,13 @@ private static Properties getCatalogProperties(org.apache.hadoop.hive.metastore. private static Schema schema(Properties properties, org.apache.hadoop.hive.metastore.api.Table hmsTable) { if (properties.getProperty(InputFormatConfig.TABLE_SCHEMA) != null) { return SchemaParser.fromJson(properties.getProperty(InputFormatConfig.TABLE_SCHEMA)); + } else if (hmsTable.isSetPartitionKeys() && !hmsTable.getPartitionKeys().isEmpty()) { + // Add partitioning columns to the original column list before creating the Iceberg Schema + List cols = Lists.newArrayList(hmsTable.getSd().getCols()); + cols.addAll(hmsTable.getPartitionKeys()); + return HiveSchemaUtil.convert(cols); } else { - if (hmsTable.isSetPartitionKeys() && !hmsTable.getPartitionKeys().isEmpty()) { - // Add partitioning columns to the original column list before creating the Iceberg Schema - List cols = new ArrayList<>(hmsTable.getSd().getCols()); - cols.addAll(hmsTable.getPartitionKeys()); - return HiveSchemaUtil.convert(cols); - } else { - return HiveSchemaUtil.convert(hmsTable.getSd().getCols()); - } + return HiveSchemaUtil.convert(hmsTable.getSd().getCols()); } } @@ -224,13 +222,11 @@ private static PartitionSpec spec(Schema schema, Properties properties, "Provide only one of the following: Hive partition specification, or the " + InputFormatConfig.PARTITION_SPEC + " property"); return PartitionSpecParser.fromJson(schema, properties.getProperty(InputFormatConfig.PARTITION_SPEC)); + } else if (hmsTable.isSetPartitionKeys() && !hmsTable.getPartitionKeys().isEmpty()) { + // If the table is partitioned then generate the identity partition definitions for the Iceberg table + return HiveSchemaUtil.spec(schema, hmsTable.getPartitionKeys()); } else { - if (hmsTable.isSetPartitionKeys() && !hmsTable.getPartitionKeys().isEmpty()) { - // If the table is partitioned then generate the identity partition definitions for the Iceberg table - return HiveSchemaUtil.spec(schema, hmsTable.getPartitionKeys()); - } else { - return PartitionSpec.unpartitioned(); - } + return PartitionSpec.unpartitioned(); } } } From 82868438efc084f242710c1708d000bae04522f2 Mon Sep 17 00:00:00 2001 From: Marton Bod Date: Tue, 5 Jan 2021 19:39:30 +0100 Subject: [PATCH 02/12] Hive: Set formats and serde for tables created from Hive (#2025) Co-authored-by: Marton Bod --- .../java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java | 5 +++++ .../iceberg/mr/hive/TestHiveIcebergStorageHandler.java | 5 +++++ 2 files changed, 10 insertions(+) diff --git a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java index 6526d8820ab1..2842bd371410 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java +++ b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java @@ -76,6 +76,11 @@ public void preCreateTable(org.apache.hadoop.hive.metastore.api.Table hmsTable) BaseMetastoreTableOperations.ICEBERG_TABLE_TYPE_VALUE.toUpperCase()); if (!Catalogs.hiveCatalog(conf)) { + // For non-HiveCatalog tables too, we should set the input and output format + // so that the table can be read by other engines like Impala + hmsTable.getSd().setInputFormat(HiveIcebergInputFormat.class.getCanonicalName()); + hmsTable.getSd().setOutputFormat(HiveIcebergOutputFormat.class.getCanonicalName()); + // If not using HiveCatalog check for existing table try { this.icebergTable = Catalogs.loadTable(conf, catalogProperties); diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandler.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandler.java index 0a75cd6c990f..fd3f70867933 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandler.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandler.java @@ -382,6 +382,11 @@ public void testCreateDropTable() throws TException, IOException, InterruptedExc Assert.assertEquals(BaseMetastoreTableOperations.ICEBERG_TABLE_TYPE_VALUE.toUpperCase(), hmsTable.getParameters().get(BaseMetastoreTableOperations.TABLE_TYPE_PROP)); + // verify that storage descriptor is filled out with inputformat/outputformat/serde + Assert.assertEquals(HiveIcebergInputFormat.class.getName(), hmsTable.getSd().getInputFormat()); + Assert.assertEquals(HiveIcebergOutputFormat.class.getName(), hmsTable.getSd().getOutputFormat()); + Assert.assertEquals(HiveIcebergSerDe.class.getName(), hmsTable.getSd().getSerdeInfo().getSerializationLib()); + if (!Catalogs.hiveCatalog(shell.getHiveConf())) { Assert.assertEquals(Collections.singletonMap("dummy", "test"), icebergTable.properties()); From 0f2a164e6ac711c2f2900c91d86c7ab338ac64ee Mon Sep 17 00:00:00 2001 From: pvary Date: Tue, 5 Jan 2021 20:14:35 +0100 Subject: [PATCH 03/12] Hive: Serialize Table to avoid a load for split planning (#1920) --- .../java/org/apache/iceberg/BaseTable.java | 33 ++++++++++++++++++- .../apache/iceberg/mr/InputFormatConfig.java | 10 ++++++ .../mr/hive/HiveIcebergStorageHandler.java | 33 ++++++++++++++++--- .../mr/mapreduce/IcebergInputFormat.java | 12 ++++--- 4 files changed, 78 insertions(+), 10 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseTable.java b/core/src/main/java/org/apache/iceberg/BaseTable.java index eb6ba96b14eb..8b2c33f128d9 100644 --- a/core/src/main/java/org/apache/iceberg/BaseTable.java +++ b/core/src/main/java/org/apache/iceberg/BaseTable.java @@ -19,6 +19,7 @@ package org.apache.iceberg; +import java.io.Serializable; import java.util.List; import java.util.Map; import org.apache.iceberg.encryption.EncryptionManager; @@ -29,8 +30,12 @@ * Base {@link Table} implementation. *

* This can be extended by providing a {@link TableOperations} to the constructor. + *

+ * Serializing and deserializing a BaseTable object returns a read only implementation of the BaseTable using a + * {@link StaticTableOperations}. This way no Catalog related calls are needed when reading the table data after + * deserialization. */ -public class BaseTable implements Table, HasTableOperations { +public class BaseTable implements Table, HasTableOperations, Serializable { private final TableOperations ops; private final String name; @@ -218,4 +223,30 @@ public LocationProvider locationProvider() { public String toString() { return name(); } + + Object writeReplace() { + return new TableStub(this); + } + + private static class TableStub implements Serializable { + private FileIO io; + private String name; + private String metadataLocation; + + private TableStub(BaseTable table) { + io = table.io(); + name = table.name(); + metadataLocation = table.operations().current().metadataFileLocation(); + } + + /** + * Returns a BaseTable with {@link StaticTableOperations} so after deserialization no Catalog related calls are + * needed for accessing the table snapshot data. + * @return The BaseTable object for reading the table data at the time of the serialization of the original + * BaseTable object + */ + private Object readResolve() { + return new BaseTable(new StaticTableOperations(metadataLocation, io), name); + } + } } diff --git a/mr/src/main/java/org/apache/iceberg/mr/InputFormatConfig.java b/mr/src/main/java/org/apache/iceberg/mr/InputFormatConfig.java index 40e3106f7dad..4d3580a64fa3 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/InputFormatConfig.java +++ b/mr/src/main/java/org/apache/iceberg/mr/InputFormatConfig.java @@ -23,6 +23,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.Schema; import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.expressions.Expression; @@ -45,6 +46,7 @@ private InputFormatConfig() { public static final String TABLE_LOCATION = "iceberg.mr.table.location"; public static final String TABLE_SCHEMA = "iceberg.mr.table.schema"; public static final String PARTITION_SPEC = "iceberg.mr.table.partition.spec"; + public static final String SERIALIZED_TABLE = "iceberg.mr.serialized.table"; public static final String LOCALITY = "iceberg.mr.locality"; public static final String CATALOG = "iceberg.mr.catalog"; public static final String HADOOP_CATALOG_WAREHOUSE_LOCATION = "iceberg.mr.catalog.hadoop.warehouse.location"; @@ -85,6 +87,14 @@ public ConfigBuilder(Configuration conf) { } public Configuration conf() { + // Store the io and the current snapshot of the table in the configuration which are needed for the split + // generation + Table table = Catalogs.loadTable(conf); + + // The FileIO serializes the configuration and we might end up recursively serializing the objects. + // To avoid this unset the value before serialization and set it again in the next line. + conf.unset(InputFormatConfig.FILE_IO); + conf.set(InputFormatConfig.FILE_IO, SerializationUtil.serializeToBase64(table.io())); return conf; } diff --git a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java index ec4c9a75fb53..172ba0c9fbab 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java +++ b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java @@ -19,6 +19,7 @@ package org.apache.iceberg.mr.hive; +import java.io.Serializable; import java.util.Map; import java.util.Properties; import org.apache.hadoop.conf.Configuration; @@ -85,7 +86,11 @@ public void configureInputJobProperties(TableDesc tableDesc, Map map.put(InputFormatConfig.TABLE_IDENTIFIER, props.getProperty(Catalogs.NAME)); map.put(InputFormatConfig.TABLE_LOCATION, table.location()); map.put(InputFormatConfig.TABLE_SCHEMA, schemaJson); + if (table instanceof Serializable) { + map.put(InputFormatConfig.SERIALIZED_TABLE, SerializationUtil.serializeToBase64(table)); + } + map.put(InputFormatConfig.FILE_IO, SerializationUtil.serializeToBase64(table.io())); // save schema into table props as well to avoid repeatedly hitting the HMS during serde initializations // this is an exception to the interface documentation, but it's a safe operation to add this property props.put(InputFormatConfig.TABLE_SCHEMA, schemaJson); @@ -190,6 +195,7 @@ public static PartitionSpec spec(Configuration config) { * Stores the serializable table data in the configuration. * Currently the following is handled: *

    + *
  • - Table - in case the table is serializable
  • *
  • - Location
  • *
  • - Schema
  • *
  • - Partition specification
  • @@ -202,12 +208,31 @@ public static PartitionSpec spec(Configuration config) { */ @VisibleForTesting static void put(Configuration config, Table table) { + // The Table contains a FileIO and the FileIO serializes the configuration so we might end up recursively + // serializing the objects. To avoid this unset the values for now before serializing. + config.unset(InputFormatConfig.SERIALIZED_TABLE); + config.unset(InputFormatConfig.FILE_IO); + config.unset(InputFormatConfig.LOCATION_PROVIDER); + config.unset(InputFormatConfig.ENCRYPTION_MANAGER); + config.unset(InputFormatConfig.TABLE_LOCATION); + config.unset(InputFormatConfig.TABLE_SCHEMA); + config.unset(InputFormatConfig.PARTITION_SPEC); + + String base64Table = table instanceof Serializable ? SerializationUtil.serializeToBase64(table) : null; + String base64Io = SerializationUtil.serializeToBase64(table.io()); + String base64LocationProvider = SerializationUtil.serializeToBase64(table.locationProvider()); + String base64EncryptionManager = SerializationUtil.serializeToBase64(table.encryption()); + + if (base64Table != null) { + config.set(InputFormatConfig.SERIALIZED_TABLE, base64Table); + } + + config.set(InputFormatConfig.FILE_IO, base64Io); + config.set(InputFormatConfig.LOCATION_PROVIDER, base64LocationProvider); + config.set(InputFormatConfig.ENCRYPTION_MANAGER, base64EncryptionManager); + config.set(InputFormatConfig.TABLE_LOCATION, table.location()); config.set(InputFormatConfig.TABLE_SCHEMA, SchemaParser.toJson(table.schema())); config.set(InputFormatConfig.PARTITION_SPEC, PartitionSpecParser.toJson(table.spec())); - - config.set(InputFormatConfig.FILE_IO, SerializationUtil.serializeToBase64(table.io())); - config.set(InputFormatConfig.LOCATION_PROVIDER, SerializationUtil.serializeToBase64(table.locationProvider())); - config.set(InputFormatConfig.ENCRYPTION_MANAGER, SerializationUtil.serializeToBase64(table.encryption())); } } diff --git a/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java b/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java index 18d88f5ff6c9..116e729b76e4 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java +++ b/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java @@ -71,8 +71,6 @@ import org.apache.iceberg.types.Type; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.util.PartitionUtil; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Generic Mrv2 InputFormat API for Iceberg. @@ -80,8 +78,6 @@ * @param T is the in memory data model which can either be Pig tuples, Hive rows. Default is Iceberg records */ public class IcebergInputFormat extends InputFormat { - private static final Logger LOG = LoggerFactory.getLogger(IcebergInputFormat.class); - /** * Configures the {@code Job} to use the {@code IcebergInputFormat} and * returns a helper to add further configuration. @@ -96,7 +92,13 @@ public static InputFormatConfig.ConfigBuilder configure(Job job) { @Override public List getSplits(JobContext context) { Configuration conf = context.getConfiguration(); - Table table = Catalogs.loadTable(conf); + Table table; + if (conf.get(InputFormatConfig.SERIALIZED_TABLE) != null) { + table = SerializationUtil.deserializeFromBase64(conf.get(InputFormatConfig.SERIALIZED_TABLE)); + } else { + table = Catalogs.loadTable(conf); + } + TableScan scan = table.newScan() .caseSensitive(conf.getBoolean(InputFormatConfig.CASE_SENSITIVE, true)); long snapshotId = conf.getLong(InputFormatConfig.SNAPSHOT_ID, -1); From 4eb2010b3d51f63a6bfdafa9173f00ca354e66aa Mon Sep 17 00:00:00 2001 From: Jack Ye Date: Tue, 5 Jan 2021 13:08:28 -0800 Subject: [PATCH 04/12] AWS: Add LockManager interface and update Glue (#1823) --- .../apache/iceberg/aws/glue/GlueTestBase.java | 6 +- .../apache/iceberg/aws/s3/S3FileIOTest.java | 4 +- .../apache/iceberg/aws/glue/GlueCatalog.java | 7 +- .../iceberg/aws/glue/GlueTableOperations.java | 38 ++- .../apache/iceberg/aws/glue/LockManager.java | 50 ++++ .../apache/iceberg/aws/glue/LockManagers.java | 257 ++++++++++++++++++ .../iceberg/aws/glue/GlueCatalogTest.java | 9 +- .../aws/glue/InMemoryLockManagerTest.java | 166 +++++++++++ .../iceberg/aws/glue/LockManagersTest.java | 64 +++++ .../org/apache/iceberg/CatalogProperties.java | 22 ++ 10 files changed, 606 insertions(+), 17 deletions(-) create mode 100644 aws/src/main/java/org/apache/iceberg/aws/glue/LockManager.java create mode 100644 aws/src/main/java/org/apache/iceberg/aws/glue/LockManagers.java create mode 100644 aws/src/test/java/org/apache/iceberg/aws/glue/InMemoryLockManagerTest.java create mode 100644 aws/src/test/java/org/apache/iceberg/aws/glue/LockManagersTest.java diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java index 48a4f860e8bb..f86890abbff1 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java @@ -68,11 +68,13 @@ public static void beforeClass() { String testBucketPath = "s3://" + testBucketName + "/" + testPathPrefix; S3FileIO fileIO = new S3FileIO(clientFactory::s3); glueCatalog = new GlueCatalog(); - glueCatalog.initialize(catalogName, testBucketPath, new AwsProperties(), glue, fileIO); + glueCatalog.initialize(catalogName, testBucketPath, new AwsProperties(), glue, + LockManagers.defaultLockManager(), fileIO); AwsProperties properties = new AwsProperties(); properties.setGlueCatalogSkipArchive(true); glueCatalogWithSkip = new GlueCatalog(); - glueCatalogWithSkip.initialize(catalogName, testBucketPath, properties, glue, fileIO); + glueCatalogWithSkip.initialize(catalogName, testBucketPath, properties, glue, + LockManagers.defaultLockManager(), fileIO); } @AfterClass diff --git a/aws/src/integration/java/org/apache/iceberg/aws/s3/S3FileIOTest.java b/aws/src/integration/java/org/apache/iceberg/aws/s3/S3FileIOTest.java index 06e18b8426c9..04ddabfa6939 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/s3/S3FileIOTest.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/s3/S3FileIOTest.java @@ -36,7 +36,6 @@ import org.apache.iceberg.aws.AwsProperties; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; @@ -202,8 +201,7 @@ public void testACL() throws Exception { @Test public void testClientFactorySerialization() throws Exception { - S3FileIO fileIO = new S3FileIO(); - fileIO.initialize(Maps.newHashMap()); + S3FileIO fileIO = new S3FileIO(clientFactory::s3); write(fileIO); byte [] data = SerializationUtils.serialize(fileIO); S3FileIO fileIO2 = SerializationUtils.deserialize(data); diff --git a/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java b/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java index 2dddda9c6878..a4d407d0b8f0 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java +++ b/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java @@ -78,6 +78,7 @@ public class GlueCatalog extends BaseMetastoreCatalog implements Closeable, Supp private String warehousePath; private AwsProperties awsProperties; private FileIO fileIO; + private LockManager lockManager; /** * No-arg constructor to load the catalog dynamically. @@ -94,6 +95,7 @@ public void initialize(String name, Map properties) { properties.get(CatalogProperties.WAREHOUSE_LOCATION), new AwsProperties(properties), AwsClientFactories.from(properties).glue(), + LockManagers.from(properties), initializeFileIO(properties)); } @@ -109,11 +111,12 @@ private FileIO initializeFileIO(Map properties) { } @VisibleForTesting - void initialize(String name, String path, AwsProperties properties, GlueClient client, FileIO io) { + void initialize(String name, String path, AwsProperties properties, GlueClient client, LockManager lock, FileIO io) { this.catalogName = name; this.awsProperties = properties; this.warehousePath = cleanWarehousePath(path); this.glue = client; + this.lockManager = lock; this.fileIO = io; } @@ -130,7 +133,7 @@ private String cleanWarehousePath(String path) { @Override protected TableOperations newTableOps(TableIdentifier tableIdentifier) { - return new GlueTableOperations(glue, catalogName, awsProperties, fileIO, tableIdentifier); + return new GlueTableOperations(glue, lockManager, catalogName, awsProperties, fileIO, tableIdentifier); } /** diff --git a/aws/src/main/java/org/apache/iceberg/aws/glue/GlueTableOperations.java b/aws/src/main/java/org/apache/iceberg/aws/glue/GlueTableOperations.java index cff437b18606..832c45d4c9c7 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/glue/GlueTableOperations.java +++ b/aws/src/main/java/org/apache/iceberg/aws/glue/GlueTableOperations.java @@ -57,16 +57,20 @@ class GlueTableOperations extends BaseMetastoreTableOperations { private final String databaseName; private final String tableName; private final String fullTableName; + private final String commitLockEntityId; private final FileIO fileIO; + private final LockManager lockManager; - GlueTableOperations(GlueClient glue, String catalogName, AwsProperties awsProperties, + GlueTableOperations(GlueClient glue, LockManager lockManager, String catalogName, AwsProperties awsProperties, FileIO fileIO, TableIdentifier tableIdentifier) { this.glue = glue; this.awsProperties = awsProperties; this.databaseName = IcebergToGlueConverter.getDatabaseName(tableIdentifier); this.tableName = IcebergToGlueConverter.getTableName(tableIdentifier); this.fullTableName = String.format("%s.%s.%s", catalogName, databaseName, tableName); + this.commitLockEntityId = String.format("%s.%s", databaseName, tableName); this.fileIO = fileIO; + this.lockManager = lockManager; } @Override @@ -100,10 +104,11 @@ protected void doRefresh() { protected void doCommit(TableMetadata base, TableMetadata metadata) { String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1); boolean exceptionThrown = true; - Table glueTable = getGlueTable(); - checkMetadataLocation(glueTable, base); - Map properties = prepareProperties(glueTable, newMetadataLocation); try { + lock(newMetadataLocation); + Table glueTable = getGlueTable(); + checkMetadataLocation(glueTable, base); + Map properties = prepareProperties(glueTable, newMetadataLocation); persistGlueTable(glueTable, properties); exceptionThrown = false; } catch (ConcurrentModificationException e) { @@ -114,9 +119,14 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) { } catch (SdkException e) { throw new CommitFailedException(e, "Cannot commit %s because unexpected exception contacting AWS", tableName()); } finally { - if (exceptionThrown) { - io().deleteFile(newMetadataLocation); - } + cleanupMetadataAndUnlock(exceptionThrown, newMetadataLocation); + } + } + + private void lock(String newMetadataLocation) { + if (!lockManager.acquire(commitLockEntityId, newMetadataLocation)) { + throw new IllegalStateException(String.format("Fail to acquire lock %s to commit new metadata at %s", + commitLockEntityId, newMetadataLocation)); } } @@ -180,4 +190,18 @@ private void persistGlueTable(Table glueTable, Map parameters) { .build()); } } + + private void cleanupMetadataAndUnlock(boolean exceptionThrown, String metadataLocation) { + try { + if (exceptionThrown) { + // if anything went wrong, clean up the uncommitted metadata file + io().deleteFile(metadataLocation); + } + } catch (RuntimeException e) { + LOG.error("Fail to cleanup metadata file at {}", metadataLocation, e); + throw e; + } finally { + lockManager.release(commitLockEntityId, metadataLocation); + } + } } diff --git a/aws/src/main/java/org/apache/iceberg/aws/glue/LockManager.java b/aws/src/main/java/org/apache/iceberg/aws/glue/LockManager.java new file mode 100644 index 000000000000..16c7e25a5359 --- /dev/null +++ b/aws/src/main/java/org/apache/iceberg/aws/glue/LockManager.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.aws.glue; + +import java.util.Map; + +/** + * An interface for locking, used to ensure Glue catalog commit isolation. + */ +interface LockManager extends AutoCloseable { + + /** + * Try to acquire a lock + * @param entityId ID of the entity to lock + * @param ownerId ID of the owner if the lock + * @return if the lock for the entity is acquired by the owner + */ + boolean acquire(String entityId, String ownerId); + + /** + * Release a lock + * @param entityId ID of the entity to lock + * @param ownerId ID of the owner if the lock + * @return if the lock for the entity of the owner is released + */ + boolean release(String entityId, String ownerId); + + /** + * Initialize lock manager from catalog properties. + * @param properties catalog properties + */ + void initialize(Map properties); +} diff --git a/aws/src/main/java/org/apache/iceberg/aws/glue/LockManagers.java b/aws/src/main/java/org/apache/iceberg/aws/glue/LockManagers.java new file mode 100644 index 000000000000..02ed234e1c00 --- /dev/null +++ b/aws/src/main/java/org/apache/iceberg/aws/glue/LockManagers.java @@ -0,0 +1,257 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.aws.glue; + +import java.util.Map; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.common.DynConstructors; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors; +import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.Tasks; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class LockManagers { + + private static final LockManager LOCK_MANAGER_DEFAULT = new InMemoryLockManager(Maps.newHashMap()); + + private LockManagers() { + } + + public static LockManager defaultLockManager() { + return LOCK_MANAGER_DEFAULT; + } + + public static LockManager from(Map properties) { + if (properties.containsKey(CatalogProperties.LOCK_IMPL)) { + return loadLockManager(properties.get(CatalogProperties.LOCK_IMPL), properties); + } else { + return defaultLockManager(); + } + } + + private static LockManager loadLockManager(String impl, Map properties) { + DynConstructors.Ctor ctor; + try { + ctor = DynConstructors.builder(LockManager.class).hiddenImpl(impl).buildChecked(); + } catch (NoSuchMethodException e) { + throw new IllegalArgumentException(String.format( + "Cannot initialize LockManager, missing no-arg constructor: %s", impl), e); + } + + LockManager lockManager; + try { + lockManager = ctor.newInstance(); + } catch (ClassCastException e) { + throw new IllegalArgumentException( + String.format("Cannot initialize LockManager, %s does not implement LockManager.", impl), e); + } + + lockManager.initialize(properties); + return lockManager; + } + + abstract static class BaseLockManager implements LockManager { + + private static volatile ScheduledExecutorService scheduler; + + private long acquireTimeoutMs; + private long acquireIntervalMs; + private long heartbeatIntervalMs; + private long heartbeatTimeoutMs; + private int heartbeatThreads; + + public long heartbeatTimeoutMs() { + return heartbeatTimeoutMs; + } + + public long heartbeatIntervalMs() { + return heartbeatIntervalMs; + } + + public long acquireIntervalMs() { + return acquireIntervalMs; + } + + public long acquireTimeoutMs() { + return acquireTimeoutMs; + } + + public int heartbeatThreads() { + return heartbeatThreads; + } + + public ScheduledExecutorService scheduler() { + if (scheduler == null) { + synchronized (BaseLockManager.class) { + if (scheduler == null) { + scheduler = MoreExecutors.getExitingScheduledExecutorService( + (ScheduledThreadPoolExecutor) Executors.newScheduledThreadPool( + heartbeatThreads(), + new ThreadFactoryBuilder() + .setDaemon(true) + .setNameFormat("iceberg-lock-manager-%d") + .build())); + } + } + } + + return scheduler; + } + + @Override + public void initialize(Map properties) { + this.acquireTimeoutMs = PropertyUtil.propertyAsLong(properties, + CatalogProperties.LOCK_ACQUIRE_TIMEOUT_MS, CatalogProperties.LOCK_ACQUIRE_TIMEOUT_MS_DEFAULT); + this.acquireIntervalMs = PropertyUtil.propertyAsLong(properties, + CatalogProperties.LOCK_ACQUIRE_INTERVAL_MS, CatalogProperties.LOCK_ACQUIRE_INTERVAL_MS_DEFAULT); + this.heartbeatIntervalMs = PropertyUtil.propertyAsLong(properties, + CatalogProperties.LOCK_HEARTBEAT_INTERVAL_MS, CatalogProperties.LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT); + this.heartbeatTimeoutMs = PropertyUtil.propertyAsLong(properties, + CatalogProperties.LOCK_HEARTBEAT_TIMEOUT_MS, CatalogProperties.LOCK_HEARTBEAT_TIMEOUT_MS_DEFAULT); + this.heartbeatThreads = PropertyUtil.propertyAsInt(properties, + CatalogProperties.LOCK_HEARTBEAT_THREADS, CatalogProperties.LOCK_HEARTBEAT_THREADS_DEFAULT); + } + } + + /** + * Implementation of {@link LockManager} that uses an in-memory concurrent map for locking. + * This implementation should only be used for testing, + * or if the caller only needs locking within the same JVM during table commits. + */ + static class InMemoryLockManager extends BaseLockManager { + + private static final Logger LOG = LoggerFactory.getLogger(InMemoryLockManager.class); + + private static final Map LOCKS = Maps.newConcurrentMap(); + private static final Map> HEARTBEATS = Maps.newHashMap(); + + InMemoryLockManager(Map properties) { + initialize(properties); + } + + @VisibleForTesting + void acquireOnce(String entityId, String ownerId) { + InMemoryLockContent content = LOCKS.get(entityId); + if (content != null && content.expireMs() > System.currentTimeMillis()) { + throw new IllegalStateException(String.format("Lock for %s currently held by %s, expiration: %s", + entityId, content.ownerId(), content.expireMs())); + } + + long expiration = System.currentTimeMillis() + heartbeatTimeoutMs(); + boolean succeed; + if (content == null) { + InMemoryLockContent previous = LOCKS.putIfAbsent( + entityId, new InMemoryLockContent(ownerId, expiration)); + succeed = previous == null; + } else { + succeed = LOCKS.replace(entityId, content, new InMemoryLockContent(ownerId, expiration)); + } + + if (succeed) { + // cleanup old heartbeat + if (HEARTBEATS.containsKey(entityId)) { + HEARTBEATS.remove(entityId).cancel(false); + } + + HEARTBEATS.put(entityId, scheduler().scheduleAtFixedRate(() -> { + InMemoryLockContent lastContent = LOCKS.get(entityId); + try { + long newExpiration = System.currentTimeMillis() + heartbeatTimeoutMs(); + LOCKS.replace(entityId, lastContent, new InMemoryLockContent(ownerId, newExpiration)); + } catch (NullPointerException e) { + throw new RuntimeException("Cannot heartbeat to a deleted lock " + entityId, e); + } + + }, 0, heartbeatIntervalMs(), TimeUnit.MILLISECONDS)); + + } else { + throw new IllegalStateException("Unable to acquire lock " + entityId); + } + } + + @Override + public boolean acquire(String entityId, String ownerId) { + try { + Tasks.foreach(entityId) + .retry(Integer.MAX_VALUE - 1) + .onlyRetryOn(IllegalStateException.class) + .throwFailureWhenFinished() + .exponentialBackoff(acquireIntervalMs(), acquireIntervalMs(), acquireTimeoutMs(), 1) + .run(id -> acquireOnce(id, ownerId)); + return true; + } catch (IllegalStateException e) { + return false; + } + } + + @Override + public boolean release(String entityId, String ownerId) { + InMemoryLockContent currentContent = LOCKS.get(entityId); + if (currentContent == null) { + LOG.error("Cannot find lock for entity {}", entityId); + return false; + } + + if (!currentContent.ownerId().equals(ownerId)) { + LOG.error("Cannot unlock {} by {}, current owner: {}", entityId, ownerId, currentContent.ownerId()); + return false; + } + + HEARTBEATS.remove(entityId).cancel(false); + LOCKS.remove(entityId); + return true; + } + + @Override + public void close() { + HEARTBEATS.values().forEach(future -> future.cancel(false)); + HEARTBEATS.clear(); + LOCKS.clear(); + } + } + + private static class InMemoryLockContent { + private final String ownerId; + private final long expireMs; + + InMemoryLockContent(String ownerId, long expireMs) { + this.ownerId = ownerId; + this.expireMs = expireMs; + } + + public long expireMs() { + return expireMs; + } + + public String ownerId() { + return ownerId; + } + + } +} diff --git a/aws/src/test/java/org/apache/iceberg/aws/glue/GlueCatalogTest.java b/aws/src/test/java/org/apache/iceberg/aws/glue/GlueCatalogTest.java index 45329c1c1e77..95dfad687f06 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/glue/GlueCatalogTest.java +++ b/aws/src/test/java/org/apache/iceberg/aws/glue/GlueCatalogTest.java @@ -70,7 +70,8 @@ public class GlueCatalogTest { public void before() { glue = Mockito.mock(GlueClient.class); glueCatalog = new GlueCatalog(); - glueCatalog.initialize(CATALOG_NAME, WAREHOUSE_PATH, new AwsProperties(), glue, null); + glueCatalog.initialize(CATALOG_NAME, WAREHOUSE_PATH, new AwsProperties(), glue, + LockManagers.defaultLockManager(), null); } @Test @@ -80,14 +81,16 @@ public void constructor_emptyWarehousePath() { "Cannot initialize GlueCatalog because warehousePath must not be null", () -> { GlueCatalog catalog = new GlueCatalog(); - catalog.initialize(CATALOG_NAME, null, new AwsProperties(), glue, null); + catalog.initialize(CATALOG_NAME, null, new AwsProperties(), glue, + LockManagers.defaultLockManager(), null); }); } @Test public void constructor_warehousePathWithEndSlash() { GlueCatalog catalogWithSlash = new GlueCatalog(); - catalogWithSlash.initialize(CATALOG_NAME, WAREHOUSE_PATH + "/", new AwsProperties(), glue, null); + catalogWithSlash.initialize( + CATALOG_NAME, WAREHOUSE_PATH + "/", new AwsProperties(), glue, LockManagers.defaultLockManager(), null); Mockito.doReturn(GetDatabaseResponse.builder() .database(Database.builder().name("db").build()).build()) .when(glue).getDatabase(Mockito.any(GetDatabaseRequest.class)); diff --git a/aws/src/test/java/org/apache/iceberg/aws/glue/InMemoryLockManagerTest.java b/aws/src/test/java/org/apache/iceberg/aws/glue/InMemoryLockManagerTest.java new file mode 100644 index 000000000000..636dfdef1ca7 --- /dev/null +++ b/aws/src/test/java/org/apache/iceberg/aws/glue/InMemoryLockManagerTest.java @@ -0,0 +1,166 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.aws.glue; + +import java.util.List; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.iceberg.AssertHelpers; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; + +public class InMemoryLockManagerTest { + + private LockManagers.InMemoryLockManager lockManager; + private String lockEntityId; + private String ownerId; + + @Rule + public Timeout timeout = new Timeout(5, TimeUnit.SECONDS); + + @Before + public void before() { + lockEntityId = UUID.randomUUID().toString(); + ownerId = UUID.randomUUID().toString(); + lockManager = new LockManagers.InMemoryLockManager(Maps.newHashMap()); + } + + @After + public void after() { + lockManager.close(); + } + + @Test + public void testAcquireOnce_singleProcess() { + lockManager.acquireOnce(lockEntityId, ownerId); + AssertHelpers.assertThrows("should fail when acquire again", + IllegalStateException.class, + "currently held", + () -> lockManager.acquireOnce(lockEntityId, ownerId)); + } + + @Test + public void testAcquireOnce_multiProcess() { + List results = IntStream.range(0, 10).parallel() + .mapToObj(i -> { + try { + lockManager.acquireOnce(lockEntityId, ownerId); + return true; + } catch (IllegalStateException e) { + return false; + } + }) + .collect(Collectors.toList()); + Assert.assertEquals( + "only 1 thread should have acquired the lock", + 1, results.stream().filter(s -> s).count()); + } + + @Test + public void testReleaseAndAcquire() { + Assert.assertTrue(lockManager.acquire(lockEntityId, ownerId)); + Assert.assertTrue(lockManager.release(lockEntityId, ownerId)); + Assert.assertTrue("acquire after release should succeed", lockManager.acquire(lockEntityId, ownerId)); + } + + @Test + public void testReleaseWithWrongOwner() { + Assert.assertTrue(lockManager.acquire(lockEntityId, ownerId)); + Assert.assertFalse("should return false if ownerId is wrong", + lockManager.release(lockEntityId, UUID.randomUUID().toString())); + } + + @Test + public void testAcquire_singleProcess() throws Exception { + lockManager.initialize(ImmutableMap.of( + CatalogProperties.LOCK_ACQUIRE_INTERVAL_MS, "500", + CatalogProperties.LOCK_ACQUIRE_TIMEOUT_MS, "2000" + )); + Assert.assertTrue(lockManager.acquire(lockEntityId, ownerId)); + String oldOwner = ownerId; + + CompletableFuture.supplyAsync(() -> { + try { + Thread.sleep(200); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + Assert.assertTrue(lockManager.release(lockEntityId, oldOwner)); + return null; + }); + + ownerId = UUID.randomUUID().toString(); + long start = System.currentTimeMillis(); + Assert.assertTrue(lockManager.acquire(lockEntityId, ownerId)); + Assert.assertTrue("should succeed after 200ms", + System.currentTimeMillis() - start >= 200); + } + + @Test + public void testAcquire_multiProcess_allSucceed() { + lockManager.initialize(ImmutableMap.of( + CatalogProperties.LOCK_ACQUIRE_INTERVAL_MS, "500" + )); + long start = System.currentTimeMillis(); + List results = IntStream.range(0, 3).parallel() + .mapToObj(i -> { + String owner = UUID.randomUUID().toString(); + boolean succeeded = lockManager.acquire(lockEntityId, owner); + if (succeeded) { + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + Assert.assertTrue(lockManager.release(lockEntityId, owner)); + } + return succeeded; + }) + .collect(Collectors.toList()); + Assert.assertEquals("all lock acquire should succeed sequentially", + 3, results.stream().filter(s -> s).count()); + Assert.assertTrue("must take more than 3 seconds", System.currentTimeMillis() - start >= 3000); + } + + @Test + public void testAcquire_multiProcess_onlyOneSucceed() { + lockManager.initialize(ImmutableMap.of( + CatalogProperties.LOCK_HEARTBEAT_INTERVAL_MS, "100", + CatalogProperties.LOCK_ACQUIRE_INTERVAL_MS, "500", + CatalogProperties.LOCK_ACQUIRE_TIMEOUT_MS, "2000" + )); + + List results = IntStream.range(0, 3).parallel() + .mapToObj(i -> lockManager.acquire(lockEntityId, ownerId)) + .collect(Collectors.toList()); + Assert.assertEquals("only 1 thread should have acquired the lock", + 1, results.stream().filter(s -> s).count()); + } +} diff --git a/aws/src/test/java/org/apache/iceberg/aws/glue/LockManagersTest.java b/aws/src/test/java/org/apache/iceberg/aws/glue/LockManagersTest.java new file mode 100644 index 000000000000..b5bf054d2122 --- /dev/null +++ b/aws/src/test/java/org/apache/iceberg/aws/glue/LockManagersTest.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.aws.glue; + +import java.util.Map; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.Assert; +import org.junit.Test; + +public class LockManagersTest { + + @Test + public void testLoadDefaultLockManager() { + Assert.assertTrue(LockManagers.defaultLockManager() instanceof LockManagers.InMemoryLockManager); + } + + @Test + public void testLoadCustomLockManager() { + Map properties = Maps.newHashMap(); + properties.put(CatalogProperties.LOCK_IMPL, CustomLockManager.class.getName()); + Assert.assertTrue(LockManagers.from(properties) instanceof CustomLockManager); + } + + static class CustomLockManager implements LockManager { + + @Override + public boolean acquire(String entityId, String ownerId) { + return false; + } + + @Override + public boolean release(String entityId, String ownerId) { + return false; + } + + @Override + public void close() throws Exception { + + } + + @Override + public void initialize(Map properties) { + + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/CatalogProperties.java b/core/src/main/java/org/apache/iceberg/CatalogProperties.java index e27a507fef8b..5992810b261d 100644 --- a/core/src/main/java/org/apache/iceberg/CatalogProperties.java +++ b/core/src/main/java/org/apache/iceberg/CatalogProperties.java @@ -19,6 +19,8 @@ package org.apache.iceberg; +import java.util.concurrent.TimeUnit; + public class CatalogProperties { private CatalogProperties() { @@ -31,4 +33,24 @@ private CatalogProperties() { public static final String HIVE_URI = "uri"; public static final String HIVE_CLIENT_POOL_SIZE = "clients"; public static final int HIVE_CLIENT_POOL_SIZE_DEFAULT = 2; + + public static final String LOCK_IMPL = "lock.impl"; + + public static final String LOCK_HEARTBEAT_INTERVAL_MS = "lock.heartbeat-interval-ms"; + public static final long LOCK_HEARTBEAT_INTERVAL_MS_DEFAULT = TimeUnit.SECONDS.toMillis(3); + + public static final String LOCK_HEARTBEAT_TIMEOUT_MS = "lock.heartbeat-timeout-ms"; + public static final long LOCK_HEARTBEAT_TIMEOUT_MS_DEFAULT = TimeUnit.SECONDS.toMillis(15); + + public static final String LOCK_HEARTBEAT_THREADS = "lock.heartbeat-threads"; + public static final int LOCK_HEARTBEAT_THREADS_DEFAULT = 4; + + public static final String LOCK_ACQUIRE_INTERVAL_MS = "lock.acquire-interval-ms"; + public static final long LOCK_ACQUIRE_INTERVAL_MS_DEFAULT = TimeUnit.SECONDS.toMillis(5); + + public static final String LOCK_ACQUIRE_TIMEOUT_MS = "lock.acquire-timeout-ms"; + public static final long LOCK_ACQUIRE_TIMEOUT_MS_DEFAULT = TimeUnit.MINUTES.toMillis(3); + + public static final String LOCK_TABLE = "lock.table"; + } From 78495a20bfc207e4f9c3f6cae2b9be14923f3562 Mon Sep 17 00:00:00 2001 From: Roman Pleshkov Date: Tue, 5 Jan 2021 17:06:54 -0500 Subject: [PATCH 05/12] Parquet: Return correct length after writer is closed (#2001) --- .../apache/iceberg/parquet/ParquetWriter.java | 28 +++++++++-- .../parquet/ParquetWritingTestUtils.java | 48 ++++++++++++++----- .../apache/iceberg/parquet/TestParquet.java | 46 +++++++++++++++--- 3 files changed, 99 insertions(+), 23 deletions(-) diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriter.java index 8b5d00026887..390012624ddf 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriter.java @@ -33,6 +33,7 @@ import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.parquet.bytes.ByteBufferAllocator; import org.apache.parquet.column.ColumnWriteStore; @@ -74,6 +75,7 @@ class ParquetWriter implements FileAppender, Closeable { private long nextRowGroupSize = 0; private long recordCount = 0; private long nextCheckRecordCount = 10; + private boolean closed; private static final String COLUMN_INDEX_TRUNCATE_LENGTH = "parquet.columnindex.truncate.length"; private static final int DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH = 64; @@ -124,10 +126,23 @@ public Metrics metrics() { return ParquetUtil.footerMetrics(writer.getFooter(), model.metrics(), metricsConfig); } + /** + * Returns the approximate length of the output file produced by this writer. + *

    + * Prior to calling {@link ParquetWriter#close}, the result is approximate. After calling close, the length is + * exact. + * + * @return the approximate length of the output file produced by this writer or the exact length if this writer is + * closed. + */ @Override public long length() { try { - return writer.getPos() + (writeStore.isColumnFlushNeeded() ? writeStore.getBufferedSize() : 0); + if (closed) { + return writer.getPos(); + } else { + return writer.getPos() + (writeStore.isColumnFlushNeeded() ? writeStore.getBufferedSize() : 0); + } } catch (IOException e) { throw new RuntimeIOException(e, "Failed to get file length"); } @@ -170,6 +185,8 @@ private void flushRowGroup(boolean finished) { } private void startRowGroup() { + Preconditions.checkState(!closed, "Writer is closed"); + try { this.nextRowGroupSize = Math.min(writer.getNextRowGroupSize(), targetRowGroupSize); } catch (IOException e) { @@ -189,8 +206,11 @@ private void startRowGroup() { @Override public void close() throws IOException { - flushRowGroup(true); - writeStore.close(); - writer.end(metadata); + if (!closed) { + this.closed = true; + flushRowGroup(true); + writeStore.close(); + writer.end(metadata); + } } } diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/ParquetWritingTestUtils.java b/parquet/src/test/java/org/apache/iceberg/parquet/ParquetWritingTestUtils.java index 2a0f6a4f63f6..24effa7496a5 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/ParquetWritingTestUtils.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/ParquetWritingTestUtils.java @@ -19,6 +19,7 @@ package org.apache.iceberg.parquet; +import java.io.Closeable; import java.io.File; import java.io.IOException; import java.util.Collections; @@ -53,20 +54,41 @@ static File writeRecords(TemporaryFolder temp, Schema schema, } static File writeRecords( - TemporaryFolder temp, - Schema schema, Map properties, - Function> createWriterFunc, - GenericData.Record... records) throws IOException { - File tmpFolder = temp.newFolder("parquet"); - String filename = UUID.randomUUID().toString(); - File file = new File(tmpFolder, FileFormat.PARQUET.addExtension(filename)); - try (FileAppender writer = Parquet.write(localOutput(file)) - .schema(schema) - .setAll(properties) - .createWriterFunc(createWriterFunc) - .build()) { + TemporaryFolder temp, + Schema schema, Map properties, + Function> createWriterFunc, + GenericData.Record... records) throws IOException { + File file = createTempFile(temp); + write(file, schema, properties, createWriterFunc, records); + return file; + } + + static long write(File file, Schema schema, Map properties, + Function> createWriterFunc, + GenericData.Record... records) throws IOException { + + long len = 0; + + FileAppender writer = Parquet.write(localOutput(file)) + .schema(schema) + .setAll(properties) + .createWriterFunc(createWriterFunc) + .build(); + + try (Closeable toClose = writer) { writer.addAll(Lists.newArrayList(records)); + len = writer.length(); // in deprecated adapter we need to get the length first and then close the writer } - return file; + + if (writer instanceof ParquetWriter) { + len = writer.length(); + } + return len; + } + + static File createTempFile(TemporaryFolder temp) throws IOException { + File tmpFolder = temp.newFolder("parquet"); + String filename = UUID.randomUUID().toString(); + return new File(tmpFolder, FileFormat.PARQUET.addExtension(filename)); } } diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java index d9d5eb0af93c..c931e26601aa 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java @@ -22,6 +22,7 @@ import java.io.File; import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.function.Function; import org.apache.avro.generic.GenericData; @@ -29,6 +30,7 @@ import org.apache.iceberg.avro.AvroSchemaUtil; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Types.IntegerType; +import org.apache.iceberg.util.Pair; import org.apache.parquet.hadoop.ParquetFileReader; import org.apache.parquet.schema.MessageType; import org.junit.Assert; @@ -38,7 +40,8 @@ import static org.apache.iceberg.Files.localInput; import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES; -import static org.apache.iceberg.parquet.ParquetWritingTestUtils.writeRecords; +import static org.apache.iceberg.parquet.ParquetWritingTestUtils.createTempFile; +import static org.apache.iceberg.parquet.ParquetWritingTestUtils.write; import static org.apache.iceberg.types.Types.NestedField.optional; public class TestParquet { @@ -49,7 +52,7 @@ public class TestParquet { @Test public void testRowGroupSizeConfigurable() throws IOException { // Without an explicit writer function - File parquetFile = generateFileWithTwoRowGroups(null); + File parquetFile = generateFileWithTwoRowGroups(null).first(); try (ParquetFileReader reader = ParquetFileReader.open(ParquetIO.file(localInput(parquetFile)))) { Assert.assertEquals(2, reader.getRowGroups().size()); @@ -58,14 +61,43 @@ public void testRowGroupSizeConfigurable() throws IOException { @Test public void testRowGroupSizeConfigurableWithWriter() throws IOException { - File parquetFile = generateFileWithTwoRowGroups(ParquetAvroWriter::buildWriter); + File parquetFile = generateFileWithTwoRowGroups(ParquetAvroWriter::buildWriter).first(); try (ParquetFileReader reader = ParquetFileReader.open(ParquetIO.file(localInput(parquetFile)))) { Assert.assertEquals(2, reader.getRowGroups().size()); } } - private File generateFileWithTwoRowGroups(Function> createWriterFunc) + @Test + public void testNumberOfBytesWritten() throws IOException { + Schema schema = new Schema( + optional(1, "intCol", IntegerType.get()) + ); + + // this value was specifically derived to reproduce iss1980 + // record count grow factor is 10000 (hardcoded) + // total 10 checkSize method calls + // for the 10th time (the last call of the checkSize method) nextCheckRecordCount == 100100 + // 100099 + 1 >= 100100 + int recordCount = 100099; + File file = createTempFile(temp); + + List records = new ArrayList<>(recordCount); + org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(schema.asStruct()); + for (int i = 1; i <= recordCount; i++) { + GenericData.Record record = new GenericData.Record(avroSchema); + record.put("intCol", i); + records.add(record); + } + + long actualSize = write(file, schema, Collections.emptyMap(), ParquetAvroWriter::buildWriter, + records.toArray(new GenericData.Record[]{})); + + long expectedSize = ParquetIO.file(localInput(file)).getLength(); + Assert.assertEquals(expectedSize, actualSize); + } + + private Pair generateFileWithTwoRowGroups(Function> createWriterFunc) throws IOException { Schema schema = new Schema( optional(1, "intCol", IntegerType.get()) @@ -85,12 +117,14 @@ private File generateFileWithTwoRowGroups(Function Date: Wed, 6 Jan 2021 09:25:59 -0800 Subject: [PATCH 06/12] AWS: add client factory for assume role use case (#2002) * AWS: add client factory for assume role use case * fix tests --- .../aws/AssumeRoleAwsClientFactoryTest.java | 165 ++++++++++++++++++ .../apache/iceberg/aws/AwsIntegTestUtil.java | 8 + .../aws/AssumeRoleAwsClientFactory.java | 98 +++++++++++ .../org/apache/iceberg/aws/AwsProperties.java | 31 ++++ build.gradle | 2 + 5 files changed, 304 insertions(+) create mode 100644 aws/src/integration/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactoryTest.java create mode 100644 aws/src/main/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactory.java diff --git a/aws/src/integration/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactoryTest.java b/aws/src/integration/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactoryTest.java new file mode 100644 index 000000000000..59c7423b4e15 --- /dev/null +++ b/aws/src/integration/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactoryTest.java @@ -0,0 +1,165 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.aws; + +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.aws.glue.GlueCatalog; +import org.apache.iceberg.aws.s3.S3FileIO; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.glue.model.AccessDeniedException; +import software.amazon.awssdk.services.glue.model.GlueException; +import software.amazon.awssdk.services.iam.IamClient; +import software.amazon.awssdk.services.iam.model.CreateRoleRequest; +import software.amazon.awssdk.services.iam.model.CreateRoleResponse; +import software.amazon.awssdk.services.iam.model.DeleteRolePolicyRequest; +import software.amazon.awssdk.services.iam.model.DeleteRoleRequest; +import software.amazon.awssdk.services.iam.model.PutRolePolicyRequest; +import software.amazon.awssdk.services.s3.model.S3Exception; + +public class AssumeRoleAwsClientFactoryTest { + + private static final Logger LOG = LoggerFactory.getLogger(AssumeRoleAwsClientFactoryTest.class); + + private IamClient iam; + private String roleName; + private Map assumeRoleProperties; + private String policyName; + + @Before + public void before() { + roleName = UUID.randomUUID().toString(); + iam = IamClient.builder() + .region(Region.AWS_GLOBAL) + .httpClient(UrlConnectionHttpClient.create()) + .build(); + CreateRoleResponse response = iam.createRole(CreateRoleRequest.builder() + .roleName(roleName) + .assumeRolePolicyDocument("{" + + "\"Version\":\"2012-10-17\"," + + "\"Statement\":[{" + + "\"Effect\":\"Allow\"," + + "\"Principal\":{" + + "\"AWS\":\"arn:aws:iam::" + AwsIntegTestUtil.testAccountId() + ":root\"}," + + "\"Action\": \"sts:AssumeRole\"}]}") + .maxSessionDuration(3600) + .build()); + assumeRoleProperties = Maps.newHashMap(); + assumeRoleProperties.put(AwsProperties.CLIENT_FACTORY, AssumeRoleAwsClientFactory.class.getName()); + assumeRoleProperties.put(AwsProperties.CLIENT_ASSUME_ROLE_REGION, "us-east-1"); + assumeRoleProperties.put(AwsProperties.CLIENT_ASSUME_ROLE_ARN, response.role().arn()); + policyName = UUID.randomUUID().toString(); + } + + @After + public void after() { + iam.deleteRolePolicy(DeleteRolePolicyRequest.builder().roleName(roleName).policyName(policyName).build()); + iam.deleteRole(DeleteRoleRequest.builder().roleName(roleName).build()); + } + + @Test + public void testAssumeRole_glueCatalog() throws Exception { + String glueArnPrefix = "arn:aws:glue:*:" + AwsIntegTestUtil.testAccountId(); + iam.putRolePolicy(PutRolePolicyRequest.builder() + .roleName(roleName) + .policyName(policyName) + .policyDocument("{" + + "\"Version\":\"2012-10-17\"," + + "\"Statement\":[{" + + "\"Sid\":\"policy1\"," + + "\"Effect\":\"Allow\"," + + "\"Action\":[\"glue:CreateDatabase\",\"glue:DeleteDatabase\",\"glue:GetDatabase\",\"glue:GetTables\"]," + + "\"Resource\":[\"" + glueArnPrefix + ":catalog\"," + + "\"" + glueArnPrefix + ":database/allowed_*\"," + + "\"" + glueArnPrefix + ":table/allowed_*/*\"," + + "\"" + glueArnPrefix + ":userDefinedFunction/allowed_*/*\"]}]}") + .build()); + waitForIamConsistency(); + + GlueCatalog glueCatalog = new GlueCatalog(); + assumeRoleProperties.put("warehouse", "s3://path"); + glueCatalog.initialize("test", assumeRoleProperties); + try { + glueCatalog.createNamespace(Namespace.of("denied_" + UUID.randomUUID().toString().replace("-", ""))); + Assert.fail("Access to Glue should be denied"); + } catch (GlueException e) { + Assert.assertEquals(AccessDeniedException.class, e.getClass()); + } + + Namespace namespace = Namespace.of("allowed_" + UUID.randomUUID().toString().replace("-", "")); + try { + glueCatalog.createNamespace(namespace); + } catch (GlueException e) { + LOG.error("fail to create or delete Glue database", e); + Assert.fail("create namespace should succeed"); + } finally { + glueCatalog.dropNamespace(namespace); + } + } + + @Test + public void testAssumeRole_s3FileIO() throws Exception { + String bucketArn = "arn:aws:s3:::" + AwsIntegTestUtil.testBucketName(); + iam.putRolePolicy(PutRolePolicyRequest.builder() + .roleName(roleName) + .policyName(policyName) + .policyDocument("{" + + "\"Version\":\"2012-10-17\"," + + "\"Statement\":[{" + + "\"Sid\":\"policy1\"," + + "\"Effect\":\"Allow\"," + + "\"Action\":\"s3:ListBucket\"," + + "\"Resource\":[\"" + bucketArn + "\"]," + + "\"Condition\":{\"StringLike\":{\"s3:prefix\":[\"allowed/*\"]}}} ,{" + + "\"Sid\":\"policy2\"," + + "\"Effect\":\"Allow\"," + + "\"Action\":\"s3:GetObject\"," + + "\"Resource\":[\"" + bucketArn + "/allowed/*\"]}]}") + .build()); + waitForIamConsistency(); + + S3FileIO s3FileIO = new S3FileIO(); + s3FileIO.initialize(assumeRoleProperties); + InputFile inputFile = s3FileIO.newInputFile("s3://" + AwsIntegTestUtil.testBucketName() + "/denied/file"); + try { + inputFile.exists(); + Assert.fail("Access to s3 should be denied"); + } catch (S3Exception e) { + Assert.assertEquals("Should see 403 error code", 403, e.statusCode()); + } + + inputFile = s3FileIO.newInputFile("s3://" + AwsIntegTestUtil.testBucketName() + "/allowed/file"); + Assert.assertFalse("should be able to access file", inputFile.exists()); + } + + private void waitForIamConsistency() throws Exception { + Thread.sleep(10000); // sleep to make sure IAM up to date + } +} diff --git a/aws/src/integration/java/org/apache/iceberg/aws/AwsIntegTestUtil.java b/aws/src/integration/java/org/apache/iceberg/aws/AwsIntegTestUtil.java index d8faaf8b438c..89b1620d7339 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/AwsIntegTestUtil.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/AwsIntegTestUtil.java @@ -47,6 +47,14 @@ public static String testBucketName() { return System.getenv("AWS_TEST_BUCKET"); } + /** + * Set the environment variable AWS_TEST_ACCOUNT_ID for a default account to use for testing + * @return account id + */ + public static String testAccountId() { + return System.getenv("AWS_TEST_ACCOUNT_ID"); + } + public static void cleanS3Bucket(S3Client s3, String bucketName, String prefix) { boolean hasContent = true; while (hasContent) { diff --git a/aws/src/main/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactory.java b/aws/src/main/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactory.java new file mode 100644 index 000000000000..0275c633bb4f --- /dev/null +++ b/aws/src/main/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactory.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.aws; + +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.PropertyUtil; +import software.amazon.awssdk.awscore.client.builder.AwsClientBuilder; +import software.amazon.awssdk.awscore.client.builder.AwsSyncClientBuilder; +import software.amazon.awssdk.http.SdkHttpClient; +import software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.glue.GlueClient; +import software.amazon.awssdk.services.kms.KmsClient; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.sts.StsClient; +import software.amazon.awssdk.services.sts.auth.StsAssumeRoleCredentialsProvider; +import software.amazon.awssdk.services.sts.model.AssumeRoleRequest; + +public class AssumeRoleAwsClientFactory implements AwsClientFactory { + + private static final SdkHttpClient HTTP_CLIENT_DEFAULT = UrlConnectionHttpClient.create(); + + private String roleArn; + private String externalId; + private int timeout; + private String region; + + @Override + public S3Client s3() { + return S3Client.builder().applyMutation(this::configure).build(); + } + + @Override + public GlueClient glue() { + return GlueClient.builder().applyMutation(this::configure).build(); + } + + @Override + public KmsClient kms() { + return KmsClient.builder().applyMutation(this::configure).build(); + } + + @Override + public void initialize(Map properties) { + roleArn = properties.get(AwsProperties.CLIENT_ASSUME_ROLE_ARN); + Preconditions.checkNotNull(roleArn, + "Cannot initialize AssumeRoleClientConfigFactory with null role ARN"); + timeout = PropertyUtil.propertyAsInt(properties, + AwsProperties.CLIENT_ASSUME_ROLE_TIMEOUT_SEC, AwsProperties.CLIENT_ASSUME_ROLE_TIMEOUT_SEC_DEFAULT); + externalId = properties.get(AwsProperties.CLIENT_ASSUME_ROLE_EXTERNAL_ID); + + region = properties.get(AwsProperties.CLIENT_ASSUME_ROLE_REGION); + Preconditions.checkNotNull(region, "Cannot initialize AssumeRoleClientConfigFactory with null region"); + } + + private T configure(T clientBuilder) { + AssumeRoleRequest request = AssumeRoleRequest.builder() + .roleArn(roleArn) + .roleSessionName(genSessionName()) + .durationSeconds(timeout) + .externalId(externalId) + .build(); + + clientBuilder.credentialsProvider( + StsAssumeRoleCredentialsProvider.builder() + .stsClient(StsClient.builder().httpClient(HTTP_CLIENT_DEFAULT).build()) + .refreshRequest(request) + .build()); + + clientBuilder.region(Region.of(region)); + clientBuilder.httpClient(HTTP_CLIENT_DEFAULT); + + return clientBuilder; + } + + private String genSessionName() { + return String.format("iceberg-aws-%s", UUID.randomUUID()); + } +} diff --git a/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java b/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java index 80ad9bdb8760..2ff51aa4c75a 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java @@ -137,6 +137,37 @@ public class AwsProperties implements Serializable { */ public static final String CLIENT_FACTORY = "client.factory"; + /** + * Used by {@link AssumeRoleAwsClientFactory}. + * If set, all AWS clients will assume a role of the given ARN, instead of using the default credential chain. + */ + public static final String CLIENT_ASSUME_ROLE_ARN = "client.assume-role.arn"; + + /** + * Used by {@link AssumeRoleAwsClientFactory}. + * The timeout of the assume role session in seconds, default to 1 hour. + * At the end of the timeout, a new set of role session credentials will be fetched through a STS client. + */ + public static final String CLIENT_ASSUME_ROLE_TIMEOUT_SEC = "client.assume-role.timeout-sec"; + public static final int CLIENT_ASSUME_ROLE_TIMEOUT_SEC_DEFAULT = 3600; + + /** + * Used by {@link AssumeRoleAwsClientFactory}. + * Optional external ID used to assume an IAM role. + *

    + * For more details, see https://docs.aws.amazon.com/IAM/latest/UserGuide/id_roles_create_for-user_externalid.html + */ + public static final String CLIENT_ASSUME_ROLE_EXTERNAL_ID = "client.assume-role.external-id"; + + /** + * Used by {@link AssumeRoleAwsClientFactory}. + * If set, all AWS clients except STS client will use the given region instead of the default region chain. + *

    + * The value must be one of {@link software.amazon.awssdk.regions.Region}, such as 'us-east-1'. + * For more details, see https://docs.aws.amazon.com/general/latest/gr/rande.html + */ + public static final String CLIENT_ASSUME_ROLE_REGION = "client.assume-role.region"; + private String s3FileIoSseType; private String s3FileIoSseKey; private String s3FileIoSseMd5; diff --git a/build.gradle b/build.gradle index 4fa875306869..62154aab60b4 100644 --- a/build.gradle +++ b/build.gradle @@ -269,6 +269,7 @@ project(':iceberg-aws') { compileOnly 'software.amazon.awssdk:s3' compileOnly 'software.amazon.awssdk:kms' compileOnly 'software.amazon.awssdk:glue' + compileOnly 'software.amazon.awssdk:sts' compileOnly("org.apache.hadoop:hadoop-common") { exclude group: 'org.apache.avro', module: 'avro' @@ -277,6 +278,7 @@ project(':iceberg-aws') { exclude group: 'com.google.code.gson', module: 'gson' } + testCompile 'software.amazon.awssdk:iam' testCompile project(path: ':iceberg-api', configuration: 'testArtifacts') testCompile("com.adobe.testing:s3mock-junit4") { exclude module: "spring-boot-starter-logging" From 9c5948fd23aa664e649b3ab81a999b89f68b6857 Mon Sep 17 00:00:00 2001 From: pvary Date: Wed, 6 Jan 2021 20:21:21 +0100 Subject: [PATCH 07/12] Hive: Run fewer combinations in TestHiveIcebergStorageHandler (#1924) (#2030) --- .../HiveIcebergStorageHandlerTestUtils.java | 94 ++ .../hive/TestHiveIcebergStorageHandler.java | 1067 ----------------- ...estHiveIcebergStorageHandlerLocalScan.java | 560 +++++++++ .../TestHiveIcebergStorageHandlerNoScan.java | 495 ++++++++ ...stHiveIcebergStorageHandlerWithEngine.java | 206 ++++ 5 files changed, 1355 insertions(+), 1067 deletions(-) create mode 100644 mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerTestUtils.java delete mode 100644 mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandler.java create mode 100644 mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerLocalScan.java create mode 100644 mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java create mode 100644 mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithEngine.java diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerTestUtils.java b/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerTestUtils.java new file mode 100644 index 000000000000..97e0cb92288e --- /dev/null +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerTestUtils.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.mr.hive; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.mr.TestHelper; +import org.apache.iceberg.types.Types; +import org.junit.rules.TemporaryFolder; + +import static org.apache.iceberg.types.Types.NestedField.optional; + +public class HiveIcebergStorageHandlerTestUtils { + static final FileFormat[] FILE_FORMATS = + new FileFormat[] {FileFormat.AVRO, FileFormat.ORC, FileFormat.PARQUET}; + + static final Schema CUSTOMER_SCHEMA = new Schema( + optional(1, "customer_id", Types.LongType.get()), + optional(2, "first_name", Types.StringType.get()), + optional(3, "last_name", Types.StringType.get()) + ); + + static final List CUSTOMER_RECORDS = TestHelper.RecordsBuilder.newInstance(CUSTOMER_SCHEMA) + .add(0L, "Alice", "Brown") + .add(1L, "Bob", "Green") + .add(2L, "Trudy", "Pink") + .build(); + + private HiveIcebergStorageHandlerTestUtils() { + // Empty constructor for the utility class + } + + static TestHiveShell shell() { + TestHiveShell shell = new TestHiveShell(); + shell.setHiveConfValue("hive.notification.event.poll.interval", "-1"); + shell.setHiveConfValue("hive.tez.exec.print.summary", "true"); + shell.start(); + return shell; + } + + static TestTables testTables(TestHiveShell shell, TestTables.TestTableType testTableType, TemporaryFolder temp) + throws IOException { + + return testTableType.instance(shell.metastore().hiveConf(), temp); + } + + static void init(TestHiveShell shell, TestTables testTables, TemporaryFolder temp, String engine) { + shell.openSession(); + + for (Map.Entry property : testTables.properties().entrySet()) { + shell.setHiveSessionValue(property.getKey(), property.getValue()); + } + + shell.setHiveSessionValue("hive.execution.engine", engine); + shell.setHiveSessionValue("hive.jar.directory", temp.getRoot().getAbsolutePath()); + shell.setHiveSessionValue("tez.staging-dir", temp.getRoot().getAbsolutePath()); + + // temporarily disabling vectorization in Tez, since it doesn't work with projection pruning (fix: TEZ-4248) + // TODO: remove this once TEZ-4248 has been released and the Tez dependencies updated here + if (engine.equals("tez")) { + shell.setHiveSessionValue("hive.vectorized.execution.enabled", "false"); + } + + } + + static void close(TestHiveShell shell) throws Exception { + shell.closeSession(); + shell.metastore().reset(); + // HiveServer2 thread pools are using thread local Hive -> HMSClient objects. These are not cleaned up when the + // HiveServer2 is stopped. Only Finalizer closes the HMS connections. + System.gc(); + } +} diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandler.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandler.java deleted file mode 100644 index fd3f70867933..000000000000 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandler.java +++ /dev/null @@ -1,1067 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iceberg.mr.hive; - -import java.io.IOException; -import java.math.BigDecimal; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.common.StatsSetupConst; -import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; -import org.apache.iceberg.AssertHelpers; -import org.apache.iceberg.BaseMetastoreTableOperations; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.PartitionSpecParser; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SchemaParser; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.TestHelpers.Row; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.exceptions.NoSuchTableException; -import org.apache.iceberg.hadoop.Util; -import org.apache.iceberg.mr.Catalogs; -import org.apache.iceberg.mr.InputFormatConfig; -import org.apache.iceberg.mr.TestHelper; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; -import org.apache.thrift.TException; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import static org.apache.iceberg.types.Types.NestedField.optional; -import static org.apache.iceberg.types.Types.NestedField.required; -import static org.junit.runners.Parameterized.Parameter; -import static org.junit.runners.Parameterized.Parameters; - -@RunWith(Parameterized.class) -public class TestHiveIcebergStorageHandler { - private static final FileFormat[] FILE_FORMATS = - new FileFormat[] {FileFormat.AVRO, FileFormat.ORC, FileFormat.PARQUET}; - - private static final String[] EXECUTION_ENGINES = new String[] {"mr", "tez"}; - - private static final Schema CUSTOMER_SCHEMA = new Schema( - optional(1, "customer_id", Types.LongType.get()), - optional(2, "first_name", Types.StringType.get()), - optional(3, "last_name", Types.StringType.get()) - ); - - private static final List CUSTOMER_RECORDS = TestHelper.RecordsBuilder.newInstance(CUSTOMER_SCHEMA) - .add(0L, "Alice", "Brown") - .add(1L, "Bob", "Green") - .add(2L, "Trudy", "Pink") - .build(); - - private static final Schema ORDER_SCHEMA = new Schema( - required(1, "order_id", Types.LongType.get()), - required(2, "customer_id", Types.LongType.get()), - required(3, "total", Types.DoubleType.get())); - - private static final List ORDER_RECORDS = TestHelper.RecordsBuilder.newInstance(ORDER_SCHEMA) - .add(100L, 0L, 11.11d) - .add(101L, 0L, 22.22d) - .add(102L, 1L, 33.33d) - .build(); - - private static final PartitionSpec SPEC = PartitionSpec.unpartitioned(); - - private static final PartitionSpec IDENTITY_SPEC = - PartitionSpec.builderFor(CUSTOMER_SCHEMA).identity("customer_id").build(); - - private static final Schema COMPLEX_SCHEMA = new Schema( - optional(1, "id", Types.LongType.get()), - optional(2, "name", Types.StringType.get()), - optional(3, "employee_info", Types.StructType.of( - optional(7, "employer", Types.StringType.get()), - optional(8, "id", Types.LongType.get()), - optional(9, "address", Types.StringType.get()) - )), - optional(4, "places_lived", Types.ListType.ofOptional(10, Types.StructType.of( - optional(11, "street", Types.StringType.get()), - optional(12, "city", Types.StringType.get()), - optional(13, "country", Types.StringType.get()) - ))), - optional(5, "memorable_moments", Types.MapType.ofOptional(14, 15, - Types.StringType.get(), - Types.StructType.of( - optional(16, "year", Types.IntegerType.get()), - optional(17, "place", Types.StringType.get()), - optional(18, "details", Types.StringType.get()) - ))), - optional(6, "current_address", Types.StructType.of( - optional(19, "street_address", Types.StructType.of( - optional(22, "street_number", Types.IntegerType.get()), - optional(23, "street_name", Types.StringType.get()), - optional(24, "street_type", Types.StringType.get()) - )), - optional(20, "country", Types.StringType.get()), - optional(21, "postal_code", Types.StringType.get()) - )) - ); - - private static final Set IGNORED_PARAMS = - ImmutableSet.of("bucketing_version", StatsSetupConst.ROW_COUNT, - StatsSetupConst.RAW_DATA_SIZE, StatsSetupConst.TOTAL_SIZE, StatsSetupConst.NUM_FILES, "numFilesErasureCoded"); - - private static final List SUPPORTED_TYPES = - ImmutableList.of(Types.BooleanType.get(), Types.IntegerType.get(), Types.LongType.get(), - Types.FloatType.get(), Types.DoubleType.get(), Types.DateType.get(), Types.TimestampType.withZone(), - Types.TimestampType.withoutZone(), Types.StringType.get(), Types.BinaryType.get(), - Types.DecimalType.of(3, 1)); - - @Parameters(name = "fileFormat={0}, engine={1}, catalog={2}") - public static Collection parameters() { - String javaVersion = System.getProperty("java.specification.version"); - - Collection testParams = new ArrayList<>(); - for (FileFormat fileFormat : FILE_FORMATS) { - for (String engine : EXECUTION_ENGINES) { - // include Tez tests only for Java 8 - if (javaVersion.equals("1.8") || "mr".equals(engine)) { - for (TestTables.TestTableType testTableType : TestTables.ALL_TABLE_TYPES) { - testParams.add(new Object[] {fileFormat, engine, testTableType}); - } - } - } - } - - return testParams; - } - - private static TestHiveShell shell; - - private TestTables testTables; - - @Parameter(0) - public FileFormat fileFormat; - - @Parameter(1) - public String executionEngine; - - @Parameter(2) - public TestTables.TestTableType testTableType; - - @Rule - public TemporaryFolder temp = new TemporaryFolder(); - - @BeforeClass - public static void beforeClass() { - shell = new TestHiveShell(); - shell.setHiveConfValue("hive.notification.event.poll.interval", "-1"); - shell.setHiveConfValue("hive.tez.exec.print.summary", "true"); - shell.start(); - } - - @AfterClass - public static void afterClass() { - shell.stop(); - } - - @Before - public void before() throws IOException { - shell.openSession(); - testTables = testTableType.instance(shell.metastore().hiveConf(), temp); - for (Map.Entry property : testTables.properties().entrySet()) { - shell.setHiveSessionValue(property.getKey(), property.getValue()); - } - shell.setHiveSessionValue("hive.execution.engine", executionEngine); - shell.setHiveSessionValue("hive.jar.directory", temp.getRoot().getAbsolutePath()); - shell.setHiveSessionValue("tez.staging-dir", temp.getRoot().getAbsolutePath()); - // temporarily disabling vectorization in Tez, since it doesn't work with projection pruning (fix: TEZ-4248) - // TODO: remove this once TEZ-4248 has been released and the Tez dependencies updated here - if (executionEngine.equals("tez")) { - shell.setHiveSessionValue("hive.vectorized.execution.enabled", "false"); - } - } - - @After - public void after() throws Exception { - shell.closeSession(); - shell.metastore().reset(); - // HiveServer2 thread pools are using thread local Hive -> HMSClient objects. These are not cleaned up when the - // HiveServer2 is stopped. Only Finalizer closes the HMS connections. - System.gc(); - } - - @Test - public void testScanEmptyTable() throws IOException { - Schema emptySchema = new Schema(required(1, "empty", Types.StringType.get())); - testTables.createTable(shell, "empty", emptySchema, fileFormat, ImmutableList.of()); - - List rows = shell.executeStatement("SELECT * FROM default.empty"); - Assert.assertEquals(0, rows.size()); - } - - @Test - public void testScanTable() throws IOException { - testTables.createTable(shell, "customers", CUSTOMER_SCHEMA, fileFormat, CUSTOMER_RECORDS); - - // Single fetch task: no MR job. - List rows = shell.executeStatement("SELECT * FROM default.customers"); - - Assert.assertEquals(3, rows.size()); - Assert.assertArrayEquals(new Object[] {0L, "Alice", "Brown"}, rows.get(0)); - Assert.assertArrayEquals(new Object[] {1L, "Bob", "Green"}, rows.get(1)); - Assert.assertArrayEquals(new Object[] {2L, "Trudy", "Pink"}, rows.get(2)); - - // Adding the ORDER BY clause will cause Hive to spawn a local MR job this time. - List descRows = - shell.executeStatement("SELECT first_name, customer_id FROM default.customers ORDER BY customer_id DESC"); - - Assert.assertEquals(3, descRows.size()); - Assert.assertArrayEquals(new Object[] {"Trudy", 2L}, descRows.get(0)); - Assert.assertArrayEquals(new Object[] {"Bob", 1L}, descRows.get(1)); - Assert.assertArrayEquals(new Object[] {"Alice", 0L}, descRows.get(2)); - } - - @Test - public void testJoinTables() throws IOException { - testTables.createTable(shell, "customers", CUSTOMER_SCHEMA, fileFormat, CUSTOMER_RECORDS); - testTables.createTable(shell, "orders", ORDER_SCHEMA, fileFormat, ORDER_RECORDS); - - List rows = shell.executeStatement( - "SELECT c.customer_id, c.first_name, o.order_id, o.total " + - "FROM default.customers c JOIN default.orders o ON c.customer_id = o.customer_id " + - "ORDER BY c.customer_id, o.order_id" - ); - - Assert.assertArrayEquals(new Object[] {0L, "Alice", 100L, 11.11d}, rows.get(0)); - Assert.assertArrayEquals(new Object[] {0L, "Alice", 101L, 22.22d}, rows.get(1)); - Assert.assertArrayEquals(new Object[] {1L, "Bob", 102L, 33.33d}, rows.get(2)); - } - - @Test - public void testDecimalTableWithPredicateLiterals() throws IOException { - Schema schema = new Schema(required(1, "decimal_field", Types.DecimalType.of(7, 2))); - List records = TestHelper.RecordsBuilder.newInstance(schema) - .add(new BigDecimal("85.00")) - .add(new BigDecimal("100.56")) - .add(new BigDecimal("100.57")) - .build(); - testTables.createTable(shell, "dec_test", schema, fileFormat, records); - - // Use integer literal in predicate - List rows = shell.executeStatement("SELECT * FROM default.dec_test where decimal_field >= 85"); - Assert.assertEquals(3, rows.size()); - Assert.assertArrayEquals(new Object[] {"85.00"}, rows.get(0)); - Assert.assertArrayEquals(new Object[] {"100.56"}, rows.get(1)); - Assert.assertArrayEquals(new Object[] {"100.57"}, rows.get(2)); - - // Use decimal literal in predicate with smaller scale than schema type definition - rows = shell.executeStatement("SELECT * FROM default.dec_test where decimal_field > 99.1"); - Assert.assertEquals(2, rows.size()); - Assert.assertArrayEquals(new Object[] {"100.56"}, rows.get(0)); - Assert.assertArrayEquals(new Object[] {"100.57"}, rows.get(1)); - - // Use decimal literal in predicate with higher scale than schema type definition - rows = shell.executeStatement("SELECT * FROM default.dec_test where decimal_field > 100.565"); - Assert.assertEquals(1, rows.size()); - Assert.assertArrayEquals(new Object[] {"100.57"}, rows.get(0)); - - // Use decimal literal in predicate with the same scale as schema type definition - rows = shell.executeStatement("SELECT * FROM default.dec_test where decimal_field > 640.34"); - Assert.assertEquals(0, rows.size()); - } - - @Test - public void testJoinTablesSupportedTypes() throws IOException { - for (int i = 0; i < SUPPORTED_TYPES.size(); i++) { - Type type = SUPPORTED_TYPES.get(i); - String tableName = type.typeId().toString().toLowerCase() + "_table_" + i; - String columnName = type.typeId().toString().toLowerCase() + "_column"; - - Schema schema = new Schema(required(1, columnName, type)); - List records = TestHelper.generateRandomRecords(schema, 1, 0L); - - testTables.createTable(shell, tableName, schema, fileFormat, records); - List queryResult = shell.executeStatement("select s." + columnName + ", h." + columnName + - " from default." + tableName + " s join default." + tableName + " h on h." + columnName + "=s." + - columnName); - Assert.assertEquals("Non matching record count for table " + tableName + " with type " + type, - 1, queryResult.size()); - } - } - - @Test - public void testSelectDistinctFromTable() throws IOException { - for (int i = 0; i < SUPPORTED_TYPES.size(); i++) { - Type type = SUPPORTED_TYPES.get(i); - String tableName = type.typeId().toString().toLowerCase() + "_table_" + i; - String columnName = type.typeId().toString().toLowerCase() + "_column"; - - Schema schema = new Schema(required(1, columnName, type)); - List records = TestHelper.generateRandomRecords(schema, 4, 0L); - int size = records.stream().map(r -> r.getField(columnName)).collect(Collectors.toSet()).size(); - testTables.createTable(shell, tableName, schema, fileFormat, records); - List queryResult = shell.executeStatement("select count(distinct(" + columnName + - ")) from default." + tableName); - int distincIds = ((Long) queryResult.get(0)[0]).intValue(); - Assert.assertEquals(tableName, size, distincIds); - } - } - - @Test - public void testCreateDropTable() throws TException, IOException, InterruptedException { - TableIdentifier identifier = TableIdentifier.of("default", "customers"); - - shell.executeStatement("CREATE EXTERNAL TABLE customers " + - "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + - testTables.locationForCreateTableSQL(identifier) + - "TBLPROPERTIES ('" + InputFormatConfig.TABLE_SCHEMA + "'='" + SchemaParser.toJson(CUSTOMER_SCHEMA) + "', " + - "'" + InputFormatConfig.PARTITION_SPEC + "'='" + PartitionSpecParser.toJson(IDENTITY_SPEC) + "', " + - "'dummy'='test')"); - - // Check the Iceberg table data - org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier); - Assert.assertEquals(CUSTOMER_SCHEMA.asStruct(), icebergTable.schema().asStruct()); - Assert.assertEquals(IDENTITY_SPEC, icebergTable.spec()); - - // Check the HMS table parameters - org.apache.hadoop.hive.metastore.api.Table hmsTable = - shell.metastore().clientPool().run(client -> client.getTable("default", "customers")); - - Map hmsParams = hmsTable.getParameters(); - IGNORED_PARAMS.forEach(hmsParams::remove); - - // This is only set for HiveCatalog based tables. Check the value, then remove it so the other checks can be general - if (Catalogs.hiveCatalog(shell.getHiveConf())) { - Assert.assertTrue(hmsParams.get(BaseMetastoreTableOperations.METADATA_LOCATION_PROP) - .startsWith(icebergTable.location())); - hmsParams.remove(BaseMetastoreTableOperations.METADATA_LOCATION_PROP); - } - - // General metadata checks - Assert.assertEquals(6, hmsParams.size()); - Assert.assertEquals("test", hmsParams.get("dummy")); - Assert.assertEquals("TRUE", hmsParams.get(InputFormatConfig.EXTERNAL_TABLE_PURGE)); - Assert.assertEquals("TRUE", hmsParams.get("EXTERNAL")); - Assert.assertNotNull(hmsParams.get(hive_metastoreConstants.DDL_TIME)); - Assert.assertEquals(HiveIcebergStorageHandler.class.getName(), - hmsTable.getParameters().get(hive_metastoreConstants.META_TABLE_STORAGE)); - Assert.assertEquals(BaseMetastoreTableOperations.ICEBERG_TABLE_TYPE_VALUE.toUpperCase(), - hmsTable.getParameters().get(BaseMetastoreTableOperations.TABLE_TYPE_PROP)); - - // verify that storage descriptor is filled out with inputformat/outputformat/serde - Assert.assertEquals(HiveIcebergInputFormat.class.getName(), hmsTable.getSd().getInputFormat()); - Assert.assertEquals(HiveIcebergOutputFormat.class.getName(), hmsTable.getSd().getOutputFormat()); - Assert.assertEquals(HiveIcebergSerDe.class.getName(), hmsTable.getSd().getSerdeInfo().getSerializationLib()); - - if (!Catalogs.hiveCatalog(shell.getHiveConf())) { - Assert.assertEquals(Collections.singletonMap("dummy", "test"), icebergTable.properties()); - - shell.executeStatement("DROP TABLE customers"); - - // Check if the table was really dropped even from the Catalog - AssertHelpers.assertThrows("should throw exception", NoSuchTableException.class, - "Table does not exist", () -> { - testTables.loadTable(identifier); - } - ); - } else { - Map expectedIcebergProperties = new HashMap<>(2); - expectedIcebergProperties.put("dummy", "test"); - expectedIcebergProperties.put(TableProperties.ENGINE_HIVE_ENABLED, "true"); - Assert.assertEquals(expectedIcebergProperties, icebergTable.properties()); - - // Check the HMS table parameters - hmsTable = shell.metastore().clientPool().run(client -> client.getTable("default", "customers")); - Path hmsTableLocation = new Path(hmsTable.getSd().getLocation()); - - // Drop the table - shell.executeStatement("DROP TABLE customers"); - - // Check if we drop an exception when trying to load the table - AssertHelpers.assertThrows("should throw exception", NoSuchTableException.class, - "Table does not exist", () -> { - testTables.loadTable(identifier); - } - ); - - // Check if the files are removed - FileSystem fs = Util.getFs(hmsTableLocation, shell.getHiveConf()); - if (fs.exists(hmsTableLocation)) { - // if table directory has been deleted, we're good. This is the expected behavior in Hive4. - // if table directory exists, its contents should have been cleaned up, save for an empty metadata dir (Hive3). - Assert.assertEquals(1, fs.listStatus(hmsTableLocation).length); - Assert.assertEquals(0, fs.listStatus(new Path(hmsTableLocation, "metadata")).length); - } - } - } - - @Test - public void testCreateTableWithoutSpec() throws TException, InterruptedException { - TableIdentifier identifier = TableIdentifier.of("default", "customers"); - - shell.executeStatement("CREATE EXTERNAL TABLE customers " + - "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + - testTables.locationForCreateTableSQL(identifier) + - "TBLPROPERTIES ('" + InputFormatConfig.TABLE_SCHEMA + "'='" + SchemaParser.toJson(CUSTOMER_SCHEMA) + "')"); - - // Check the Iceberg table partition data - org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier); - Assert.assertEquals(SPEC, icebergTable.spec()); - - // Check the HMS table parameters - org.apache.hadoop.hive.metastore.api.Table hmsTable = - shell.metastore().clientPool().run(client -> client.getTable("default", "customers")); - - Map hmsParams = hmsTable.getParameters(); - IGNORED_PARAMS.forEach(hmsParams::remove); - - // Just check that the PartitionSpec is not set in the metadata - Assert.assertNull(hmsParams.get(InputFormatConfig.PARTITION_SPEC)); - - if (Catalogs.hiveCatalog(shell.getHiveConf())) { - Assert.assertEquals(6, hmsParams.size()); - } else { - Assert.assertEquals(5, hmsParams.size()); - } - } - - @Test - public void testCreateTableWithUnpartitionedSpec() throws TException, InterruptedException { - TableIdentifier identifier = TableIdentifier.of("default", "customers"); - - // We need the location for HadoopTable based tests only - shell.executeStatement("CREATE EXTERNAL TABLE customers " + - "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + - testTables.locationForCreateTableSQL(identifier) + - "TBLPROPERTIES ('" + InputFormatConfig.TABLE_SCHEMA + "'='" + SchemaParser.toJson(CUSTOMER_SCHEMA) + "', " + - "'" + InputFormatConfig.PARTITION_SPEC + "'='" + PartitionSpecParser.toJson(SPEC) + "')"); - - // Check the Iceberg table partition data - org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier); - Assert.assertEquals(SPEC, icebergTable.spec()); - - // Check the HMS table parameters - org.apache.hadoop.hive.metastore.api.Table hmsTable = - shell.metastore().clientPool().run(client -> client.getTable("default", "customers")); - - Map hmsParams = hmsTable.getParameters(); - IGNORED_PARAMS.forEach(hmsParams::remove); - - // Just check that the PartitionSpec is not set in the metadata - Assert.assertNull(hmsParams.get(InputFormatConfig.PARTITION_SPEC)); - if (Catalogs.hiveCatalog(shell.getHiveConf())) { - Assert.assertEquals(6, hmsParams.size()); - } else { - Assert.assertEquals(5, hmsParams.size()); - } - } - - @Test - public void testDeleteBackingTable() throws TException, IOException, InterruptedException { - TableIdentifier identifier = TableIdentifier.of("default", "customers"); - - shell.executeStatement("CREATE EXTERNAL TABLE customers " + - "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + - testTables.locationForCreateTableSQL(identifier) + - "TBLPROPERTIES ('" + InputFormatConfig.TABLE_SCHEMA + "'='" + SchemaParser.toJson(CUSTOMER_SCHEMA) + "', " + - "'" + InputFormatConfig.EXTERNAL_TABLE_PURGE + "'='FALSE')"); - - if (!Catalogs.hiveCatalog(shell.getHiveConf())) { - shell.executeStatement("DROP TABLE customers"); - - // Check if the table remains - testTables.loadTable(identifier); - } else { - // Check the HMS table parameters - org.apache.hadoop.hive.metastore.api.Table hmsTable = - shell.metastore().clientPool().run(client -> client.getTable("default", "customers")); - Path hmsTableLocation = new Path(hmsTable.getSd().getLocation()); - - // Drop the table - shell.executeStatement("DROP TABLE customers"); - - // Check if we drop an exception when trying to drop the table - AssertHelpers.assertThrows("should throw exception", NoSuchTableException.class, - "Table does not exist", () -> { - testTables.loadTable(identifier); - } - ); - - // Check if the files are kept - FileSystem fs = Util.getFs(hmsTableLocation, shell.getHiveConf()); - Assert.assertEquals(1, fs.listStatus(hmsTableLocation).length); - Assert.assertEquals(1, fs.listStatus(new Path(hmsTableLocation, "metadata")).length); - } - } - - @Test - public void testCreateTableError() { - TableIdentifier identifier = TableIdentifier.of("default", "withShell2"); - - // Wrong schema - AssertHelpers.assertThrows("should throw exception", IllegalArgumentException.class, - "Unrecognized token 'WrongSchema'", () -> { - shell.executeStatement("CREATE EXTERNAL TABLE withShell2 " + - "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + - testTables.locationForCreateTableSQL(identifier) + - "TBLPROPERTIES ('" + InputFormatConfig.TABLE_SCHEMA + "'='WrongSchema')"); - } - ); - - // Missing schema, we try to get the schema from the table and fail - AssertHelpers.assertThrows("should throw exception", IllegalArgumentException.class, - "Please provide ", () -> { - shell.executeStatement("CREATE EXTERNAL TABLE withShell2 " + - "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + - testTables.locationForCreateTableSQL(identifier)); - } - ); - - if (!testTables.locationForCreateTableSQL(identifier).isEmpty()) { - // Only test this if the location is required - AssertHelpers.assertThrows("should throw exception", IllegalArgumentException.class, - "Table location not set", () -> { - shell.executeStatement("CREATE EXTERNAL TABLE withShell2 " + - "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + - "TBLPROPERTIES ('" + InputFormatConfig.TABLE_SCHEMA + "'='" + - SchemaParser.toJson(CUSTOMER_SCHEMA) + "')"); - } - ); - } - } - - @Test - public void testCreateTableAboveExistingTable() throws TException, IOException, InterruptedException { - // Create the Iceberg table - testTables.createIcebergTable(shell.getHiveConf(), "customers", COMPLEX_SCHEMA, FileFormat.PARQUET, - Collections.emptyList()); - - if (Catalogs.hiveCatalog(shell.getHiveConf())) { - - // In HiveCatalog we just expect an exception since the table is already exists - AssertHelpers.assertThrows("should throw exception", IllegalArgumentException.class, - "customers already exists", () -> { - shell.executeStatement("CREATE EXTERNAL TABLE customers " + - "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + - "TBLPROPERTIES ('" + InputFormatConfig.TABLE_SCHEMA + "'='" + - SchemaParser.toJson(CUSTOMER_SCHEMA) + "')"); - } - ); - } else { - shell.executeStatement("CREATE EXTERNAL TABLE customers " + - "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + - testTables.locationForCreateTableSQL(TableIdentifier.of("default", "customers"))); - - // Check the HMS table parameters - org.apache.hadoop.hive.metastore.api.Table hmsTable = - shell.metastore().clientPool().run(client -> client.getTable("default", "customers")); - - Map hmsParams = hmsTable.getParameters(); - IGNORED_PARAMS.forEach(hmsParams::remove); - - Assert.assertEquals(4, hmsParams.size()); - Assert.assertEquals("TRUE", hmsParams.get("EXTERNAL")); - Assert.assertNotNull(hmsParams.get(hive_metastoreConstants.DDL_TIME)); - Assert.assertEquals(HiveIcebergStorageHandler.class.getName(), - hmsTable.getParameters().get(hive_metastoreConstants.META_TABLE_STORAGE)); - Assert.assertEquals(BaseMetastoreTableOperations.ICEBERG_TABLE_TYPE_VALUE.toUpperCase(), - hmsTable.getParameters().get(BaseMetastoreTableOperations.TABLE_TYPE_PROP)); - } - } - - @Test - public void testColumnSelection() throws IOException { - testTables.createTable(shell, "customers", CUSTOMER_SCHEMA, fileFormat, CUSTOMER_RECORDS); - - List outOfOrderColumns = shell - .executeStatement("SELECT first_name, customer_id, last_name FROM default.customers"); - - Assert.assertEquals(3, outOfOrderColumns.size()); - Assert.assertArrayEquals(new Object[] {"Alice", 0L, "Brown"}, outOfOrderColumns.get(0)); - Assert.assertArrayEquals(new Object[] {"Bob", 1L, "Green"}, outOfOrderColumns.get(1)); - Assert.assertArrayEquals(new Object[] {"Trudy", 2L, "Pink"}, outOfOrderColumns.get(2)); - - List allButFirstColumn = shell.executeStatement("SELECT first_name, last_name FROM default.customers"); - - Assert.assertEquals(3, allButFirstColumn.size()); - Assert.assertArrayEquals(new Object[] {"Alice", "Brown"}, allButFirstColumn.get(0)); - Assert.assertArrayEquals(new Object[] {"Bob", "Green"}, allButFirstColumn.get(1)); - Assert.assertArrayEquals(new Object[] {"Trudy", "Pink"}, allButFirstColumn.get(2)); - - List allButMiddleColumn = shell.executeStatement("SELECT customer_id, last_name FROM default.customers"); - - Assert.assertEquals(3, allButMiddleColumn.size()); - Assert.assertArrayEquals(new Object[] {0L, "Brown"}, allButMiddleColumn.get(0)); - Assert.assertArrayEquals(new Object[] {1L, "Green"}, allButMiddleColumn.get(1)); - Assert.assertArrayEquals(new Object[] {2L, "Pink"}, allButMiddleColumn.get(2)); - - List allButLastColumn = shell.executeStatement("SELECT customer_id, first_name FROM default.customers"); - - Assert.assertEquals(3, allButLastColumn.size()); - Assert.assertArrayEquals(new Object[] {0L, "Alice"}, allButLastColumn.get(0)); - Assert.assertArrayEquals(new Object[] {1L, "Bob"}, allButLastColumn.get(1)); - Assert.assertArrayEquals(new Object[] {2L, "Trudy"}, allButLastColumn.get(2)); - } - - @Test - public void selectSameColumnTwice() throws IOException { - testTables.createTable(shell, "customers", CUSTOMER_SCHEMA, fileFormat, CUSTOMER_RECORDS); - - List columns = shell.executeStatement("SELECT first_name, first_name FROM default.customers"); - - Assert.assertEquals(3, columns.size()); - Assert.assertArrayEquals(new Object[] {"Alice", "Alice"}, columns.get(0)); - Assert.assertArrayEquals(new Object[] {"Bob", "Bob"}, columns.get(1)); - Assert.assertArrayEquals(new Object[] {"Trudy", "Trudy"}, columns.get(2)); - } - - @Test - public void testCreateTableWithColumnSpecification() throws IOException { - TableIdentifier identifier = TableIdentifier.of("default", "customers"); - Map> data = new HashMap<>(1); - data.put(null, CUSTOMER_RECORDS); - String createSql = "CREATE EXTERNAL TABLE " + identifier + - " (customer_id BIGINT, first_name STRING, last_name STRING)" + - " STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + - testTables.locationForCreateTableSQL(identifier); - runCreateAndReadTest(identifier, createSql, CUSTOMER_SCHEMA, PartitionSpec.unpartitioned(), data); - } - - @Test - public void testCreateTableWithColumnSpecificationPartitioned() throws IOException { - TableIdentifier identifier = TableIdentifier.of("default", "customers"); - PartitionSpec spec = PartitionSpec.builderFor(CUSTOMER_SCHEMA).identity("last_name").build(); - Map> data = ImmutableMap.of( - Row.of("Brown"), Collections.singletonList(CUSTOMER_RECORDS.get(0)), - Row.of("Green"), Collections.singletonList(CUSTOMER_RECORDS.get(1)), - Row.of("Pink"), Collections.singletonList(CUSTOMER_RECORDS.get(2))); - String createSql = "CREATE EXTERNAL TABLE " + identifier + - " (customer_id BIGINT, first_name STRING) PARTITIONED BY (last_name STRING) " + - "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + - testTables.locationForCreateTableSQL(identifier); - runCreateAndReadTest(identifier, createSql, CUSTOMER_SCHEMA, spec, data); - } - - @Test - public void testCreatePartitionedTableByProperty() throws IOException { - TableIdentifier identifier = TableIdentifier.of("default", "customers"); - PartitionSpec spec = PartitionSpec.builderFor(CUSTOMER_SCHEMA).identity("last_name").build(); - Map> data = ImmutableMap.of( - Row.of("Brown"), Collections.singletonList(CUSTOMER_RECORDS.get(0)), - Row.of("Green"), Collections.singletonList(CUSTOMER_RECORDS.get(1)), - Row.of("Pink"), Collections.singletonList(CUSTOMER_RECORDS.get(2))); - String createSql = "CREATE EXTERNAL TABLE " + identifier + - " STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + - testTables.locationForCreateTableSQL(identifier) + - "TBLPROPERTIES ('" + InputFormatConfig.PARTITION_SPEC + "'='" + PartitionSpecParser.toJson(spec) + "', " + - "'" + InputFormatConfig.TABLE_SCHEMA + "'='" + SchemaParser.toJson(CUSTOMER_SCHEMA) + "')"; - runCreateAndReadTest(identifier, createSql, CUSTOMER_SCHEMA, spec, data); - } - - @Test - public void testCreatePartitionedTableWithPropertiesAndWithColumnSpecification() { - PartitionSpec spec = PartitionSpec.builderFor(CUSTOMER_SCHEMA).identity("last_name").build(); - - AssertHelpers.assertThrows("should throw exception", IllegalArgumentException.class, - "Provide only one of the following", () -> { - shell.executeStatement("CREATE EXTERNAL TABLE customers (customer_id BIGINT) " + - "PARTITIONED BY (first_name STRING) " + - "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + - testTables.locationForCreateTableSQL(TableIdentifier.of("default", "customers")) + - " TBLPROPERTIES ('" + InputFormatConfig.PARTITION_SPEC + "'='" + - PartitionSpecParser.toJson(spec) + "')"); - } - ); - } - - @Test - public void testCreateTableWithColumnSpecificationMultilevelPartitioned() throws IOException { - TableIdentifier identifier = TableIdentifier.of("default", "customers"); - PartitionSpec spec = PartitionSpec.builderFor(CUSTOMER_SCHEMA).identity("first_name").identity("last_name").build(); - Map> data = ImmutableMap.of( - Row.of("Alice", "Brown"), Collections.singletonList(CUSTOMER_RECORDS.get(0)), - Row.of("Bob", "Green"), Collections.singletonList(CUSTOMER_RECORDS.get(1)), - Row.of("Trudy", "Pink"), Collections.singletonList(CUSTOMER_RECORDS.get(2))); - String createSql = "CREATE EXTERNAL TABLE " + identifier + " (customer_id BIGINT) " + - "PARTITIONED BY (first_name STRING, last_name STRING) " + - "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + - testTables.locationForCreateTableSQL(identifier); - runCreateAndReadTest(identifier, createSql, CUSTOMER_SCHEMA, spec, data); - } - - @Test - public void testCreateTableWithColumnSpecificationHierarchy() { - TableIdentifier identifier = TableIdentifier.of("default", "customers"); - - shell.executeStatement("CREATE EXTERNAL TABLE customers (" + - "id BIGINT, name STRING, " + - "employee_info STRUCT < employer: STRING, id: BIGINT, address: STRING >, " + - "places_lived ARRAY < STRUCT >, " + - "memorable_moments MAP < STRING, STRUCT < year: INT, place: STRING, details: STRING >>, " + - "current_address STRUCT < street_address: STRUCT " + - ", country: STRING, postal_code: STRING >) " + - "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + - testTables.locationForCreateTableSQL(identifier)); - - // Check the Iceberg table data - org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier); - Assert.assertEquals(COMPLEX_SCHEMA.asStruct(), icebergTable.schema().asStruct()); - } - - @Test - public void testCreateTableWithAllSupportedTypes() { - TableIdentifier identifier = TableIdentifier.of("default", "all_types"); - Schema allSupportedSchema = new Schema( - optional(1, "t_float", Types.FloatType.get()), - optional(2, "t_double", Types.DoubleType.get()), - optional(3, "t_boolean", Types.BooleanType.get()), - optional(4, "t_int", Types.IntegerType.get()), - optional(5, "t_bigint", Types.LongType.get()), - optional(6, "t_binary", Types.BinaryType.get()), - optional(7, "t_string", Types.StringType.get()), - optional(8, "t_timestamp", Types.TimestampType.withoutZone()), - optional(9, "t_date", Types.DateType.get()), - optional(10, "t_decimal", Types.DecimalType.of(3, 2)) - ); - - // Intentionally adding some mixed letters to test that we handle them correctly - shell.executeStatement("CREATE EXTERNAL TABLE all_types (" + - "t_Float FLOaT, t_dOuble DOUBLE, t_boolean BOOLEAN, t_int INT, t_bigint BIGINT, t_binary BINARY, " + - "t_string STRING, t_timestamp TIMESTAMP, t_date DATE, t_decimal DECIMAL(3,2)) " + - "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + - testTables.locationForCreateTableSQL(identifier)); - - // Check the Iceberg table data - org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier); - Assert.assertEquals(allSupportedSchema.asStruct(), icebergTable.schema().asStruct()); - } - - @Test - public void testCreateTableWithNotSupportedTypes() { - TableIdentifier identifier = TableIdentifier.of("default", "not_supported_types"); - // Can not create INTERVAL types from normal create table, so leave them out from this test - String[] notSupportedTypes = new String[] { "TINYINT", "SMALLINT", "VARCHAR(1)", "CHAR(1)" }; - - for (String notSupportedType : notSupportedTypes) { - AssertHelpers.assertThrows("should throw exception", IllegalArgumentException.class, - "Unsupported Hive type", () -> { - shell.executeStatement("CREATE EXTERNAL TABLE not_supported_types " + - "(not_supported " + notSupportedType + ") " + - "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + - testTables.locationForCreateTableSQL(identifier)); - } - ); - } - } - - @Test - public void testArrayOfPrimitivesInTable() throws IOException { - Schema schema = - new Schema(required(1, "arrayofprimitives", Types.ListType.ofRequired(2, Types.IntegerType.get()))); - List records = testTables.createTableWithGeneratedRecords(shell, "arraytable", schema, fileFormat, 1); - // access a single element from the array - for (int i = 0; i < records.size(); i++) { - List expectedList = (List) records.get(i).getField("arrayofprimitives"); - for (int j = 0; j < expectedList.size(); j++) { - List queryResult = shell.executeStatement( - String.format("SELECT arrayofprimitives[%d] FROM default.arraytable " + "LIMIT 1 OFFSET %d", j, i)); - Assert.assertEquals(expectedList.get(j), queryResult.get(0)[0]); - } - } - } - - @Test - public void testArrayOfArraysInTable() throws IOException { - Schema schema = - new Schema( - required(1, "arrayofarrays", - Types.ListType.ofRequired(2, Types.ListType.ofRequired(3, Types.DateType.get())))); - List records = testTables.createTableWithGeneratedRecords(shell, "arraytable", schema, fileFormat, 1); - // access an element from a matrix - for (int i = 0; i < records.size(); i++) { - List expectedList = (List) records.get(i).getField("arrayofarrays"); - for (int j = 0; j < expectedList.size(); j++) { - List expectedInnerList = (List) expectedList.get(j); - for (int k = 0; k < expectedInnerList.size(); k++) { - List queryResult = shell.executeStatement( - String.format("SELECT arrayofarrays[%d][%d] FROM default.arraytable " + "LIMIT 1 OFFSET %d", - j, k, i)); - Assert.assertEquals(expectedInnerList.get(k).toString(), queryResult.get(0)[0]); - } - } - } - } - - @Test - public void testArrayOfMapsInTable() throws IOException { - Schema schema = - new Schema(required(1, "arrayofmaps", Types.ListType - .ofRequired(2, Types.MapType.ofRequired(3, 4, Types.StringType.get(), - Types.BooleanType.get())))); - List records = testTables.createTableWithGeneratedRecords(shell, "arraytable", schema, fileFormat, 1); - // access an element from a map in an array - for (int i = 0; i < records.size(); i++) { - List expectedList = (List) records.get(i).getField("arrayofmaps"); - for (int j = 0; j < expectedList.size(); j++) { - Map expectedMap = (Map) expectedList.get(j); - for (Map.Entry entry : expectedMap.entrySet()) { - List queryResult = shell.executeStatement(String - .format("SELECT arrayofmaps[%d][\"%s\"] FROM default.arraytable LIMIT 1 OFFSET %d", j, - entry.getKey(), i)); - Assert.assertEquals(entry.getValue(), queryResult.get(0)[0]); - } - } - } - } - - @Test - public void testArrayOfStructsInTable() throws IOException { - Schema schema = - new Schema( - required(1, "arrayofstructs", Types.ListType.ofRequired(2, Types.StructType - .of(required(3, "something", Types.DoubleType.get()), required(4, "someone", - Types.LongType.get()), required(5, "somewhere", Types.StringType.get()))))); - List records = testTables.createTableWithGeneratedRecords(shell, "arraytable", schema, fileFormat, 1); - // access an element from a struct in an array - for (int i = 0; i < records.size(); i++) { - List expectedList = (List) records.get(i).getField("arrayofstructs"); - for (int j = 0; j < expectedList.size(); j++) { - List queryResult = shell.executeStatement(String.format("SELECT arrayofstructs[%d].something, " + - "arrayofstructs[%d].someone, arrayofstructs[%d].somewhere FROM default.arraytable LIMIT 1 " + - "OFFSET %d", j, j, j, i)); - GenericRecord genericRecord = (GenericRecord) expectedList.get(j); - Assert.assertEquals(genericRecord.getField("something"), queryResult.get(0)[0]); - Assert.assertEquals(genericRecord.getField("someone"), queryResult.get(0)[1]); - Assert.assertEquals(genericRecord.getField("somewhere"), queryResult.get(0)[2]); - } - } - } - - @Test - public void testMapOfPrimitivesInTable() throws IOException { - Schema schema = new Schema( - required(1, "mapofprimitives", Types.MapType.ofRequired(2, 3, Types.StringType.get(), - Types.IntegerType.get()))); - List records = testTables.createTableWithGeneratedRecords(shell, "maptable", schema, fileFormat, 1); - // access a single value from the map - for (int i = 0; i < records.size(); i++) { - Map expectedMap = (Map) records.get(i).getField("mapofprimitives"); - for (Map.Entry entry : expectedMap.entrySet()) { - List queryResult = shell.executeStatement(String - .format("SELECT mapofprimitives[\"%s\"] " + "FROM default.maptable LIMIT 1 OFFSET %d", entry.getKey(), - i)); - Assert.assertEquals(entry.getValue(), queryResult.get(0)[0]); - } - } - } - - @Test - public void testMapOfArraysInTable() throws IOException { - Schema schema = new Schema( - required(1, "mapofarrays", - Types.MapType.ofRequired(2, 3, Types.StringType.get(), Types.ListType.ofRequired(4, - Types.DateType.get())))); - List records = testTables.createTableWithGeneratedRecords(shell, "maptable", schema, fileFormat, 1); - // access a single element from a list in a map - for (int i = 0; i < records.size(); i++) { - Map expectedMap = (Map) records.get(i).getField("mapofarrays"); - for (Map.Entry entry : expectedMap.entrySet()) { - List expectedList = (List) entry.getValue(); - for (int j = 0; j < expectedList.size(); j++) { - List queryResult = shell.executeStatement(String - .format("SELECT mapofarrays[\"%s\"]" + "[%d] FROM maptable LIMIT 1 OFFSET %d", entry.getKey(), j, i)); - Assert.assertEquals(expectedList.get(j).toString(), queryResult.get(0)[0]); - } - } - } - } - - @Test - public void testMapOfMapsInTable() throws IOException { - Schema schema = new Schema( - required(1, "mapofmaps", Types.MapType.ofRequired(2, 3, Types.StringType.get(), - Types.MapType.ofRequired(4, 5, Types.StringType.get(), Types.StringType.get())))); - List records = testTables.createTableWithGeneratedRecords(shell, "maptable", schema, fileFormat, 1); - // access a single element from a map in a map - for (int i = 0; i < records.size(); i++) { - Map expectedMap = (Map) records.get(i).getField("mapofmaps"); - for (Map.Entry entry : expectedMap.entrySet()) { - Map expectedInnerMap = (Map) entry.getValue(); - for (Map.Entry innerEntry : expectedInnerMap.entrySet()) { - List queryResult = shell.executeStatement(String - .format("SELECT mapofmaps[\"%s\"]" + "[\"%s\"] FROM maptable LIMIT 1 OFFSET %d", entry.getKey(), - innerEntry.getKey(), i)); - Assert.assertEquals(innerEntry.getValue(), queryResult.get(0)[0]); - } - } - } - } - - @Test - public void testMapOfStructsInTable() throws IOException { - Schema schema = new Schema( - required(1, "mapofstructs", Types.MapType.ofRequired(2, 3, Types.StringType.get(), - Types.StructType.of(required(4, "something", Types.DoubleType.get()), - required(5, "someone", Types.LongType.get()), - required(6, "somewhere", Types.StringType.get()))))); - List records = testTables.createTableWithGeneratedRecords(shell, "maptable", schema, fileFormat, 1); - // access a single element from a struct in a map - for (int i = 0; i < records.size(); i++) { - Map expectedMap = (Map) records.get(i).getField("mapofstructs"); - for (Map.Entry entry : expectedMap.entrySet()) { - List queryResult = shell.executeStatement(String.format("SELECT mapofstructs[\"%s\"].something, " + - "mapofstructs[\"%s\"].someone, mapofstructs[\"%s\"].somewhere FROM default.maptable LIMIT 1 " + - "OFFSET %d", entry.getKey(), entry.getKey(), entry.getKey(), i)); - GenericRecord genericRecord = (GenericRecord) entry.getValue(); - Assert.assertEquals(genericRecord.getField("something"), queryResult.get(0)[0]); - Assert.assertEquals(genericRecord.getField("someone"), queryResult.get(0)[1]); - Assert.assertEquals(genericRecord.getField("somewhere"), queryResult.get(0)[2]); - } - } - } - - @Test - public void testStructOfPrimitivesInTable() throws IOException { - Schema schema = new Schema(required(1, "structofprimitives", - Types.StructType.of(required(2, "key", Types.StringType.get()), required(3, "value", - Types.IntegerType.get())))); - List records = testTables.createTableWithGeneratedRecords(shell, "structtable", schema, fileFormat, 1); - // access a single value in a struct - for (int i = 0; i < records.size(); i++) { - GenericRecord expectedStruct = (GenericRecord) records.get(i).getField("structofprimitives"); - List queryResult = shell.executeStatement(String.format( - "SELECT structofprimitives.key, structofprimitives.value FROM default.structtable LIMIT 1 OFFSET %d", i)); - Assert.assertEquals(expectedStruct.getField("key"), queryResult.get(0)[0]); - Assert.assertEquals(expectedStruct.getField("value"), queryResult.get(0)[1]); - } - } - - @Test - public void testStructOfArraysInTable() throws IOException { - Schema schema = new Schema( - required(1, "structofarrays", Types.StructType - .of(required(2, "names", Types.ListType.ofRequired(3, Types.StringType.get())), - required(4, "birthdays", Types.ListType.ofRequired(5, - Types.DateType.get()))))); - List records = testTables.createTableWithGeneratedRecords(shell, "structtable", schema, fileFormat, 1); - // access an element of an array inside a struct - for (int i = 0; i < records.size(); i++) { - GenericRecord expectedStruct = (GenericRecord) records.get(i).getField("structofarrays"); - List expectedList = (List) expectedStruct.getField("names"); - for (int j = 0; j < expectedList.size(); j++) { - List queryResult = shell.executeStatement( - String.format("SELECT structofarrays.names[%d] FROM default.structtable LIMIT 1 OFFSET %d", j, i)); - Assert.assertEquals(expectedList.get(j), queryResult.get(0)[0]); - } - expectedList = (List) expectedStruct.getField("birthdays"); - for (int j = 0; j < expectedList.size(); j++) { - List queryResult = shell.executeStatement( - String.format("SELECT structofarrays.birthdays[%d] FROM default.structtable LIMIT 1 OFFSET %d", j, i)); - Assert.assertEquals(expectedList.get(j).toString(), queryResult.get(0)[0]); - } - } - } - - @Test - public void testStructOfMapsInTable() throws IOException { - Schema schema = new Schema( - required(1, "structofmaps", Types.StructType - .of(required(2, "map1", Types.MapType.ofRequired(3, 4, - Types.StringType.get(), Types.StringType.get())), required(5, "map2", - Types.MapType.ofRequired(6, 7, Types.StringType.get(), - Types.IntegerType.get()))))); - List records = testTables.createTableWithGeneratedRecords(shell, "structtable", schema, fileFormat, 1); - // access a map entry inside a struct - for (int i = 0; i < records.size(); i++) { - GenericRecord expectedStruct = (GenericRecord) records.get(i).getField("structofmaps"); - Map expectedMap = (Map) expectedStruct.getField("map1"); - for (Map.Entry entry : expectedMap.entrySet()) { - List queryResult = shell.executeStatement(String - .format("SELECT structofmaps.map1[\"%s\"] from default.structtable LIMIT 1 OFFSET %d", entry.getKey(), - i)); - Assert.assertEquals(entry.getValue(), queryResult.get(0)[0]); - } - expectedMap = (Map) expectedStruct.getField("map2"); - for (Map.Entry entry : expectedMap.entrySet()) { - List queryResult = shell.executeStatement(String - .format("SELECT structofmaps.map2[\"%s\"] from default.structtable LIMIT 1 OFFSET %d", entry.getKey(), - i)); - Assert.assertEquals(entry.getValue(), queryResult.get(0)[0]); - } - } - } - - @Test - public void testStructOfStructsInTable() throws IOException { - Schema schema = new Schema( - required(1, "structofstructs", Types.StructType.of(required(2, "struct1", Types.StructType - .of(required(3, "key", Types.StringType.get()), required(4, "value", - Types.IntegerType.get())))))); - List records = testTables.createTableWithGeneratedRecords(shell, "structtable", schema, fileFormat, 1); - // access a struct element inside a struct - for (int i = 0; i < records.size(); i++) { - GenericRecord expectedStruct = (GenericRecord) records.get(i).getField("structofstructs"); - GenericRecord expectedInnerStruct = (GenericRecord) expectedStruct.getField("struct1"); - List queryResult = shell.executeStatement(String.format( - "SELECT structofstructs.struct1.key, structofstructs.struct1.value FROM default.structtable " + - "LIMIT 1 OFFSET %d", i)); - Assert.assertEquals(expectedInnerStruct.getField("key"), queryResult.get(0)[0]); - Assert.assertEquals(expectedInnerStruct.getField("value"), queryResult.get(0)[1]); - } - } - - private void runCreateAndReadTest(TableIdentifier identifier, String createSQL, Schema expectedSchema, - PartitionSpec expectedSpec, Map> data) throws IOException { - shell.executeStatement(createSQL); - - org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier); - Assert.assertEquals(expectedSchema.asStruct(), icebergTable.schema().asStruct()); - Assert.assertEquals(expectedSpec, icebergTable.spec()); - - List expected = Lists.newArrayList(); - for (StructLike partition : data.keySet()) { - testTables.appendIcebergTable(shell.getHiveConf(), icebergTable, fileFormat, partition, data.get(partition)); - expected.addAll(data.get(partition)); - } - - List descRows = shell.executeStatement("SELECT * FROM " + identifier.toString() + - " ORDER BY " + expectedSchema.columns().get(0).name() + " DESC"); - List records = HiveIcebergTestUtils.valueForRow(icebergTable.schema(), descRows); - - HiveIcebergTestUtils.validateData(expected, records, 0); - } -} diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerLocalScan.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerLocalScan.java new file mode 100644 index 000000000000..33203f921ba7 --- /dev/null +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerLocalScan.java @@ -0,0 +1,560 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.mr.hive; + +import java.io.IOException; +import java.math.BigDecimal; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.PartitionSpecParser; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.TestHelpers.Row; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.mr.InputFormatConfig; +import org.apache.iceberg.mr.TestHelper; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.junit.runners.Parameterized.Parameter; +import static org.junit.runners.Parameterized.Parameters; + +@RunWith(Parameterized.class) +public class TestHiveIcebergStorageHandlerLocalScan { + + @Parameters(name = "fileFormat={0}, catalog={1}") + public static Collection parameters() { + Collection testParams = new ArrayList<>(); + + // Run tests with every FileFormat for a single Catalog (HiveCatalog) + for (FileFormat fileFormat : HiveIcebergStorageHandlerTestUtils.FILE_FORMATS) { + testParams.add(new Object[] {fileFormat, TestTables.TestTableType.HIVE_CATALOG}); + } + + // Run tests for every Catalog for a single FileFormat (PARQUET) - skip HiveCatalog tests as they are added before + for (TestTables.TestTableType testTableType : TestTables.ALL_TABLE_TYPES) { + if (!TestTables.TestTableType.HIVE_CATALOG.equals(testTableType)) { + testParams.add(new Object[]{FileFormat.PARQUET, testTableType}); + } + } + + return testParams; + } + + private static TestHiveShell shell; + + private TestTables testTables; + + @Parameter(0) + public FileFormat fileFormat; + + @Parameter(1) + public TestTables.TestTableType testTableType; + + @Rule + public TemporaryFolder temp = new TemporaryFolder(); + + @BeforeClass + public static void beforeClass() { + shell = HiveIcebergStorageHandlerTestUtils.shell(); + } + + @AfterClass + public static void afterClass() { + shell.stop(); + } + + @Before + public void before() throws IOException { + testTables = HiveIcebergStorageHandlerTestUtils.testTables(shell, testTableType, temp); + // Uses spark as an engine so we can detect if we unintentionally try to use any execution engines + HiveIcebergStorageHandlerTestUtils.init(shell, testTables, temp, "spark"); + } + + @After + public void after() throws Exception { + HiveIcebergStorageHandlerTestUtils.close(shell); + } + + @Test + public void testScanEmptyTable() throws IOException { + Schema emptySchema = new Schema(required(1, "empty", Types.StringType.get())); + testTables.createTable(shell, "empty", emptySchema, fileFormat, ImmutableList.of()); + + List rows = shell.executeStatement("SELECT * FROM default.empty"); + Assert.assertEquals(0, rows.size()); + } + + @Test + public void testScanTable() throws IOException { + testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, fileFormat, + HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS); + + // Single fetch task: no MR job. + List rows = shell.executeStatement("SELECT * FROM default.customers"); + + Assert.assertEquals(3, rows.size()); + Assert.assertArrayEquals(new Object[] {0L, "Alice", "Brown"}, rows.get(0)); + Assert.assertArrayEquals(new Object[] {1L, "Bob", "Green"}, rows.get(1)); + Assert.assertArrayEquals(new Object[] {2L, "Trudy", "Pink"}, rows.get(2)); + } + + @Test + public void testDecimalTableWithPredicateLiterals() throws IOException { + Schema schema = new Schema(required(1, "decimal_field", Types.DecimalType.of(7, 2))); + List records = TestHelper.RecordsBuilder.newInstance(schema) + .add(new BigDecimal("85.00")) + .add(new BigDecimal("100.56")) + .add(new BigDecimal("100.57")) + .build(); + testTables.createTable(shell, "dec_test", schema, fileFormat, records); + + // Use integer literal in predicate + List rows = shell.executeStatement("SELECT * FROM default.dec_test where decimal_field >= 85"); + Assert.assertEquals(3, rows.size()); + Assert.assertArrayEquals(new Object[] {"85.00"}, rows.get(0)); + Assert.assertArrayEquals(new Object[] {"100.56"}, rows.get(1)); + Assert.assertArrayEquals(new Object[] {"100.57"}, rows.get(2)); + + // Use decimal literal in predicate with smaller scale than schema type definition + rows = shell.executeStatement("SELECT * FROM default.dec_test where decimal_field > 99.1"); + Assert.assertEquals(2, rows.size()); + Assert.assertArrayEquals(new Object[] {"100.56"}, rows.get(0)); + Assert.assertArrayEquals(new Object[] {"100.57"}, rows.get(1)); + + // Use decimal literal in predicate with higher scale than schema type definition + rows = shell.executeStatement("SELECT * FROM default.dec_test where decimal_field > 100.565"); + Assert.assertEquals(1, rows.size()); + Assert.assertArrayEquals(new Object[] {"100.57"}, rows.get(0)); + + // Use decimal literal in predicate with the same scale as schema type definition + rows = shell.executeStatement("SELECT * FROM default.dec_test where decimal_field > 640.34"); + Assert.assertEquals(0, rows.size()); + } + + @Test + public void testColumnSelection() throws IOException { + testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, fileFormat, + HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS); + + List outOfOrderColumns = shell + .executeStatement("SELECT first_name, customer_id, last_name FROM default.customers"); + + Assert.assertEquals(3, outOfOrderColumns.size()); + Assert.assertArrayEquals(new Object[] {"Alice", 0L, "Brown"}, outOfOrderColumns.get(0)); + Assert.assertArrayEquals(new Object[] {"Bob", 1L, "Green"}, outOfOrderColumns.get(1)); + Assert.assertArrayEquals(new Object[] {"Trudy", 2L, "Pink"}, outOfOrderColumns.get(2)); + + List allButFirstColumn = shell.executeStatement("SELECT first_name, last_name FROM default.customers"); + + Assert.assertEquals(3, allButFirstColumn.size()); + Assert.assertArrayEquals(new Object[] {"Alice", "Brown"}, allButFirstColumn.get(0)); + Assert.assertArrayEquals(new Object[] {"Bob", "Green"}, allButFirstColumn.get(1)); + Assert.assertArrayEquals(new Object[] {"Trudy", "Pink"}, allButFirstColumn.get(2)); + + List allButMiddleColumn = shell.executeStatement("SELECT customer_id, last_name FROM default.customers"); + + Assert.assertEquals(3, allButMiddleColumn.size()); + Assert.assertArrayEquals(new Object[] {0L, "Brown"}, allButMiddleColumn.get(0)); + Assert.assertArrayEquals(new Object[] {1L, "Green"}, allButMiddleColumn.get(1)); + Assert.assertArrayEquals(new Object[] {2L, "Pink"}, allButMiddleColumn.get(2)); + + List allButLastColumn = shell.executeStatement("SELECT customer_id, first_name FROM default.customers"); + + Assert.assertEquals(3, allButLastColumn.size()); + Assert.assertArrayEquals(new Object[] {0L, "Alice"}, allButLastColumn.get(0)); + Assert.assertArrayEquals(new Object[] {1L, "Bob"}, allButLastColumn.get(1)); + Assert.assertArrayEquals(new Object[] {2L, "Trudy"}, allButLastColumn.get(2)); + } + + @Test + public void selectSameColumnTwice() throws IOException { + testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, fileFormat, + HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS); + + List columns = shell.executeStatement("SELECT first_name, first_name FROM default.customers"); + + Assert.assertEquals(3, columns.size()); + Assert.assertArrayEquals(new Object[] {"Alice", "Alice"}, columns.get(0)); + Assert.assertArrayEquals(new Object[] {"Bob", "Bob"}, columns.get(1)); + Assert.assertArrayEquals(new Object[] {"Trudy", "Trudy"}, columns.get(2)); + } + + @Test + public void testCreateTableWithColumnSpecification() throws IOException { + TableIdentifier identifier = TableIdentifier.of("default", "customers"); + Map> data = new HashMap<>(1); + data.put(null, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS); + String createSql = "CREATE EXTERNAL TABLE " + identifier + + " (customer_id BIGINT, first_name STRING, last_name STRING)" + + " STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + + testTables.locationForCreateTableSQL(identifier); + runCreateAndReadTest(identifier, createSql, HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, + PartitionSpec.unpartitioned(), data); + } + + @Test + public void testCreateTableWithColumnSpecificationPartitioned() throws IOException { + TableIdentifier identifier = TableIdentifier.of("default", "customers"); + PartitionSpec spec = + PartitionSpec.builderFor(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA).identity("last_name").build(); + Map> data = ImmutableMap.of( + Row.of("Brown"), Collections.singletonList(HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS.get(0)), + Row.of("Green"), Collections.singletonList(HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS.get(1)), + Row.of("Pink"), Collections.singletonList(HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS.get(2))); + String createSql = "CREATE EXTERNAL TABLE " + identifier + + " (customer_id BIGINT, first_name STRING) PARTITIONED BY (last_name STRING) " + + "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + + testTables.locationForCreateTableSQL(identifier); + runCreateAndReadTest(identifier, createSql, HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, spec, data); + } + + @Test + public void testCreatePartitionedTableByProperty() throws IOException { + TableIdentifier identifier = TableIdentifier.of("default", "customers"); + PartitionSpec spec = + PartitionSpec.builderFor(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA).identity("last_name").build(); + Map> data = ImmutableMap.of( + Row.of("Brown"), Collections.singletonList(HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS.get(0)), + Row.of("Green"), Collections.singletonList(HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS.get(1)), + Row.of("Pink"), Collections.singletonList(HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS.get(2))); + String createSql = "CREATE EXTERNAL TABLE " + identifier + + " STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + + testTables.locationForCreateTableSQL(identifier) + + "TBLPROPERTIES ('" + InputFormatConfig.PARTITION_SPEC + "'='" + PartitionSpecParser.toJson(spec) + "', " + + "'" + InputFormatConfig.TABLE_SCHEMA + "'='" + + SchemaParser.toJson(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA) + "')"; + runCreateAndReadTest(identifier, createSql, HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, spec, data); + } + + @Test + public void testCreateTableWithColumnSpecificationMultilevelPartitioned() throws IOException { + TableIdentifier identifier = TableIdentifier.of("default", "customers"); + PartitionSpec spec = PartitionSpec.builderFor(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA) + .identity("first_name").identity("last_name").build(); + Map> data = ImmutableMap.of( + Row.of("Alice", "Brown"), Collections.singletonList(HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS.get(0)), + Row.of("Bob", "Green"), Collections.singletonList(HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS.get(1)), + Row.of("Trudy", "Pink"), Collections.singletonList(HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS.get(2))); + String createSql = "CREATE EXTERNAL TABLE " + identifier + " (customer_id BIGINT) " + + "PARTITIONED BY (first_name STRING, last_name STRING) " + + "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + + testTables.locationForCreateTableSQL(identifier); + runCreateAndReadTest(identifier, createSql, HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, spec, data); + } + + @Test + public void testArrayOfPrimitivesInTable() throws IOException { + Schema schema = + new Schema(required(1, "arrayofprimitives", Types.ListType.ofRequired(2, Types.IntegerType.get()))); + List records = testTables.createTableWithGeneratedRecords(shell, "arraytable", schema, fileFormat, 1); + // access a single element from the array + for (int i = 0; i < records.size(); i++) { + List expectedList = (List) records.get(i).getField("arrayofprimitives"); + for (int j = 0; j < expectedList.size(); j++) { + List queryResult = shell.executeStatement( + String.format("SELECT arrayofprimitives[%d] FROM default.arraytable " + "LIMIT 1 OFFSET %d", j, i)); + Assert.assertEquals(expectedList.get(j), queryResult.get(0)[0]); + } + } + } + + @Test + public void testArrayOfArraysInTable() throws IOException { + Schema schema = + new Schema( + required(1, "arrayofarrays", + Types.ListType.ofRequired(2, Types.ListType.ofRequired(3, Types.DateType.get())))); + List records = testTables.createTableWithGeneratedRecords(shell, "arraytable", schema, fileFormat, 1); + // access an element from a matrix + for (int i = 0; i < records.size(); i++) { + List expectedList = (List) records.get(i).getField("arrayofarrays"); + for (int j = 0; j < expectedList.size(); j++) { + List expectedInnerList = (List) expectedList.get(j); + for (int k = 0; k < expectedInnerList.size(); k++) { + List queryResult = shell.executeStatement( + String.format("SELECT arrayofarrays[%d][%d] FROM default.arraytable " + "LIMIT 1 OFFSET %d", + j, k, i)); + Assert.assertEquals(expectedInnerList.get(k).toString(), queryResult.get(0)[0]); + } + } + } + } + + @Test + public void testArrayOfMapsInTable() throws IOException { + Schema schema = + new Schema(required(1, "arrayofmaps", Types.ListType + .ofRequired(2, Types.MapType.ofRequired(3, 4, Types.StringType.get(), + Types.BooleanType.get())))); + List records = testTables.createTableWithGeneratedRecords(shell, "arraytable", schema, fileFormat, 1); + // access an element from a map in an array + for (int i = 0; i < records.size(); i++) { + List expectedList = (List) records.get(i).getField("arrayofmaps"); + for (int j = 0; j < expectedList.size(); j++) { + Map expectedMap = (Map) expectedList.get(j); + for (Map.Entry entry : expectedMap.entrySet()) { + List queryResult = shell.executeStatement(String + .format("SELECT arrayofmaps[%d][\"%s\"] FROM default.arraytable LIMIT 1 OFFSET %d", j, + entry.getKey(), i)); + Assert.assertEquals(entry.getValue(), queryResult.get(0)[0]); + } + } + } + } + + @Test + public void testArrayOfStructsInTable() throws IOException { + Schema schema = + new Schema( + required(1, "arrayofstructs", Types.ListType.ofRequired(2, Types.StructType + .of(required(3, "something", Types.DoubleType.get()), required(4, "someone", + Types.LongType.get()), required(5, "somewhere", Types.StringType.get()))))); + List records = testTables.createTableWithGeneratedRecords(shell, "arraytable", schema, fileFormat, 1); + // access an element from a struct in an array + for (int i = 0; i < records.size(); i++) { + List expectedList = (List) records.get(i).getField("arrayofstructs"); + for (int j = 0; j < expectedList.size(); j++) { + List queryResult = shell.executeStatement(String.format("SELECT arrayofstructs[%d].something, " + + "arrayofstructs[%d].someone, arrayofstructs[%d].somewhere FROM default.arraytable LIMIT 1 " + + "OFFSET %d", j, j, j, i)); + GenericRecord genericRecord = (GenericRecord) expectedList.get(j); + Assert.assertEquals(genericRecord.getField("something"), queryResult.get(0)[0]); + Assert.assertEquals(genericRecord.getField("someone"), queryResult.get(0)[1]); + Assert.assertEquals(genericRecord.getField("somewhere"), queryResult.get(0)[2]); + } + } + } + + @Test + public void testMapOfPrimitivesInTable() throws IOException { + Schema schema = new Schema( + required(1, "mapofprimitives", Types.MapType.ofRequired(2, 3, Types.StringType.get(), + Types.IntegerType.get()))); + List records = testTables.createTableWithGeneratedRecords(shell, "maptable", schema, fileFormat, 1); + // access a single value from the map + for (int i = 0; i < records.size(); i++) { + Map expectedMap = (Map) records.get(i).getField("mapofprimitives"); + for (Map.Entry entry : expectedMap.entrySet()) { + List queryResult = shell.executeStatement(String + .format("SELECT mapofprimitives[\"%s\"] " + "FROM default.maptable LIMIT 1 OFFSET %d", entry.getKey(), + i)); + Assert.assertEquals(entry.getValue(), queryResult.get(0)[0]); + } + } + } + + @Test + public void testMapOfArraysInTable() throws IOException { + Schema schema = new Schema( + required(1, "mapofarrays", + Types.MapType.ofRequired(2, 3, Types.StringType.get(), Types.ListType.ofRequired(4, + Types.DateType.get())))); + List records = testTables.createTableWithGeneratedRecords(shell, "maptable", schema, fileFormat, 1); + // access a single element from a list in a map + for (int i = 0; i < records.size(); i++) { + Map expectedMap = (Map) records.get(i).getField("mapofarrays"); + for (Map.Entry entry : expectedMap.entrySet()) { + List expectedList = (List) entry.getValue(); + for (int j = 0; j < expectedList.size(); j++) { + List queryResult = shell.executeStatement(String + .format("SELECT mapofarrays[\"%s\"]" + "[%d] FROM maptable LIMIT 1 OFFSET %d", entry.getKey(), j, i)); + Assert.assertEquals(expectedList.get(j).toString(), queryResult.get(0)[0]); + } + } + } + } + + @Test + public void testMapOfMapsInTable() throws IOException { + Schema schema = new Schema( + required(1, "mapofmaps", Types.MapType.ofRequired(2, 3, Types.StringType.get(), + Types.MapType.ofRequired(4, 5, Types.StringType.get(), Types.StringType.get())))); + List records = testTables.createTableWithGeneratedRecords(shell, "maptable", schema, fileFormat, 1); + // access a single element from a map in a map + for (int i = 0; i < records.size(); i++) { + Map expectedMap = (Map) records.get(i).getField("mapofmaps"); + for (Map.Entry entry : expectedMap.entrySet()) { + Map expectedInnerMap = (Map) entry.getValue(); + for (Map.Entry innerEntry : expectedInnerMap.entrySet()) { + List queryResult = shell.executeStatement(String + .format("SELECT mapofmaps[\"%s\"]" + "[\"%s\"] FROM maptable LIMIT 1 OFFSET %d", entry.getKey(), + innerEntry.getKey(), i)); + Assert.assertEquals(innerEntry.getValue(), queryResult.get(0)[0]); + } + } + } + } + + @Test + public void testMapOfStructsInTable() throws IOException { + Schema schema = new Schema( + required(1, "mapofstructs", Types.MapType.ofRequired(2, 3, Types.StringType.get(), + Types.StructType.of(required(4, "something", Types.DoubleType.get()), + required(5, "someone", Types.LongType.get()), + required(6, "somewhere", Types.StringType.get()))))); + List records = testTables.createTableWithGeneratedRecords(shell, "maptable", schema, fileFormat, 1); + // access a single element from a struct in a map + for (int i = 0; i < records.size(); i++) { + Map expectedMap = (Map) records.get(i).getField("mapofstructs"); + for (Map.Entry entry : expectedMap.entrySet()) { + List queryResult = shell.executeStatement(String.format("SELECT mapofstructs[\"%s\"].something, " + + "mapofstructs[\"%s\"].someone, mapofstructs[\"%s\"].somewhere FROM default.maptable LIMIT 1 " + + "OFFSET %d", entry.getKey(), entry.getKey(), entry.getKey(), i)); + GenericRecord genericRecord = (GenericRecord) entry.getValue(); + Assert.assertEquals(genericRecord.getField("something"), queryResult.get(0)[0]); + Assert.assertEquals(genericRecord.getField("someone"), queryResult.get(0)[1]); + Assert.assertEquals(genericRecord.getField("somewhere"), queryResult.get(0)[2]); + } + } + } + + @Test + public void testStructOfPrimitivesInTable() throws IOException { + Schema schema = new Schema(required(1, "structofprimitives", + Types.StructType.of(required(2, "key", Types.StringType.get()), required(3, "value", + Types.IntegerType.get())))); + List records = testTables.createTableWithGeneratedRecords(shell, "structtable", schema, fileFormat, 1); + // access a single value in a struct + for (int i = 0; i < records.size(); i++) { + GenericRecord expectedStruct = (GenericRecord) records.get(i).getField("structofprimitives"); + List queryResult = shell.executeStatement(String.format( + "SELECT structofprimitives.key, structofprimitives.value FROM default.structtable LIMIT 1 OFFSET %d", i)); + Assert.assertEquals(expectedStruct.getField("key"), queryResult.get(0)[0]); + Assert.assertEquals(expectedStruct.getField("value"), queryResult.get(0)[1]); + } + } + + @Test + public void testStructOfArraysInTable() throws IOException { + Schema schema = new Schema( + required(1, "structofarrays", Types.StructType + .of(required(2, "names", Types.ListType.ofRequired(3, Types.StringType.get())), + required(4, "birthdays", Types.ListType.ofRequired(5, + Types.DateType.get()))))); + List records = testTables.createTableWithGeneratedRecords(shell, "structtable", schema, fileFormat, 1); + // access an element of an array inside a struct + for (int i = 0; i < records.size(); i++) { + GenericRecord expectedStruct = (GenericRecord) records.get(i).getField("structofarrays"); + List expectedList = (List) expectedStruct.getField("names"); + for (int j = 0; j < expectedList.size(); j++) { + List queryResult = shell.executeStatement( + String.format("SELECT structofarrays.names[%d] FROM default.structtable LIMIT 1 OFFSET %d", j, i)); + Assert.assertEquals(expectedList.get(j), queryResult.get(0)[0]); + } + expectedList = (List) expectedStruct.getField("birthdays"); + for (int j = 0; j < expectedList.size(); j++) { + List queryResult = shell.executeStatement( + String.format("SELECT structofarrays.birthdays[%d] FROM default.structtable LIMIT 1 OFFSET %d", j, i)); + Assert.assertEquals(expectedList.get(j).toString(), queryResult.get(0)[0]); + } + } + } + + @Test + public void testStructOfMapsInTable() throws IOException { + Schema schema = new Schema( + required(1, "structofmaps", Types.StructType + .of(required(2, "map1", Types.MapType.ofRequired(3, 4, + Types.StringType.get(), Types.StringType.get())), required(5, "map2", + Types.MapType.ofRequired(6, 7, Types.StringType.get(), + Types.IntegerType.get()))))); + List records = testTables.createTableWithGeneratedRecords(shell, "structtable", schema, fileFormat, 1); + // access a map entry inside a struct + for (int i = 0; i < records.size(); i++) { + GenericRecord expectedStruct = (GenericRecord) records.get(i).getField("structofmaps"); + Map expectedMap = (Map) expectedStruct.getField("map1"); + for (Map.Entry entry : expectedMap.entrySet()) { + List queryResult = shell.executeStatement(String + .format("SELECT structofmaps.map1[\"%s\"] from default.structtable LIMIT 1 OFFSET %d", entry.getKey(), + i)); + Assert.assertEquals(entry.getValue(), queryResult.get(0)[0]); + } + expectedMap = (Map) expectedStruct.getField("map2"); + for (Map.Entry entry : expectedMap.entrySet()) { + List queryResult = shell.executeStatement(String + .format("SELECT structofmaps.map2[\"%s\"] from default.structtable LIMIT 1 OFFSET %d", entry.getKey(), + i)); + Assert.assertEquals(entry.getValue(), queryResult.get(0)[0]); + } + } + } + + @Test + public void testStructOfStructsInTable() throws IOException { + Schema schema = new Schema( + required(1, "structofstructs", Types.StructType.of(required(2, "struct1", Types.StructType + .of(required(3, "key", Types.StringType.get()), required(4, "value", + Types.IntegerType.get())))))); + List records = testTables.createTableWithGeneratedRecords(shell, "structtable", schema, fileFormat, 1); + // access a struct element inside a struct + for (int i = 0; i < records.size(); i++) { + GenericRecord expectedStruct = (GenericRecord) records.get(i).getField("structofstructs"); + GenericRecord expectedInnerStruct = (GenericRecord) expectedStruct.getField("struct1"); + List queryResult = shell.executeStatement(String.format( + "SELECT structofstructs.struct1.key, structofstructs.struct1.value FROM default.structtable " + + "LIMIT 1 OFFSET %d", i)); + Assert.assertEquals(expectedInnerStruct.getField("key"), queryResult.get(0)[0]); + Assert.assertEquals(expectedInnerStruct.getField("value"), queryResult.get(0)[1]); + } + } + + private void runCreateAndReadTest(TableIdentifier identifier, String createSQL, Schema expectedSchema, + PartitionSpec expectedSpec, Map> data) throws IOException { + shell.executeStatement(createSQL); + + org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier); + Assert.assertEquals(expectedSchema.asStruct(), icebergTable.schema().asStruct()); + Assert.assertEquals(expectedSpec, icebergTable.spec()); + + List expected = Lists.newArrayList(); + for (StructLike partition : data.keySet()) { + testTables.appendIcebergTable(shell.getHiveConf(), icebergTable, fileFormat, partition, data.get(partition)); + expected.addAll(data.get(partition)); + } + + List descRows = shell.executeStatement("SELECT * FROM " + identifier.toString()); + List records = HiveIcebergTestUtils.valueForRow(icebergTable.schema(), descRows); + + HiveIcebergTestUtils.validateData(expected, records, 0); + } +} diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java new file mode 100644 index 000000000000..096c9b40d58b --- /dev/null +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java @@ -0,0 +1,495 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.mr.hive; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.StatsSetupConst; +import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; +import org.apache.iceberg.AssertHelpers; +import org.apache.iceberg.BaseMetastoreTableOperations; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.PartitionSpecParser; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.hadoop.Util; +import org.apache.iceberg.mr.Catalogs; +import org.apache.iceberg.mr.InputFormatConfig; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.types.Types; +import org.apache.thrift.TException; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.junit.runners.Parameterized.Parameter; +import static org.junit.runners.Parameterized.Parameters; + +@RunWith(Parameterized.class) +public class TestHiveIcebergStorageHandlerNoScan { + private static final PartitionSpec SPEC = PartitionSpec.unpartitioned(); + + private static final Schema COMPLEX_SCHEMA = new Schema( + optional(1, "id", Types.LongType.get()), + optional(2, "name", Types.StringType.get()), + optional(3, "employee_info", Types.StructType.of( + optional(7, "employer", Types.StringType.get()), + optional(8, "id", Types.LongType.get()), + optional(9, "address", Types.StringType.get()) + )), + optional(4, "places_lived", Types.ListType.ofOptional(10, Types.StructType.of( + optional(11, "street", Types.StringType.get()), + optional(12, "city", Types.StringType.get()), + optional(13, "country", Types.StringType.get()) + ))), + optional(5, "memorable_moments", Types.MapType.ofOptional(14, 15, + Types.StringType.get(), + Types.StructType.of( + optional(16, "year", Types.IntegerType.get()), + optional(17, "place", Types.StringType.get()), + optional(18, "details", Types.StringType.get()) + ))), + optional(6, "current_address", Types.StructType.of( + optional(19, "street_address", Types.StructType.of( + optional(22, "street_number", Types.IntegerType.get()), + optional(23, "street_name", Types.StringType.get()), + optional(24, "street_type", Types.StringType.get()) + )), + optional(20, "country", Types.StringType.get()), + optional(21, "postal_code", Types.StringType.get()) + )) + ); + + private static final Set IGNORED_PARAMS = + ImmutableSet.of("bucketing_version", StatsSetupConst.ROW_COUNT, + StatsSetupConst.RAW_DATA_SIZE, StatsSetupConst.TOTAL_SIZE, StatsSetupConst.NUM_FILES, "numFilesErasureCoded"); + + @Parameters(name = "catalog={0}") + public static Collection parameters() { + Collection testParams = new ArrayList<>(); + for (TestTables.TestTableType testTableType : TestTables.ALL_TABLE_TYPES) { + testParams.add(new Object[] {testTableType}); + } + + return testParams; + } + + private static TestHiveShell shell; + + private TestTables testTables; + + @Parameter(0) + public TestTables.TestTableType testTableType; + + @Rule + public TemporaryFolder temp = new TemporaryFolder(); + + @BeforeClass + public static void beforeClass() { + shell = HiveIcebergStorageHandlerTestUtils.shell(); + } + + @AfterClass + public static void afterClass() { + shell.stop(); + } + + @Before + public void before() throws IOException { + testTables = HiveIcebergStorageHandlerTestUtils.testTables(shell, testTableType, temp); + // Uses spark as an engine so we can detect if we unintentionally try to use any execution engines + HiveIcebergStorageHandlerTestUtils.init(shell, testTables, temp, "spark"); + } + + @After + public void after() throws Exception { + HiveIcebergStorageHandlerTestUtils.close(shell); + } + + @Test + public void testCreateDropTable() throws TException, IOException, InterruptedException { + TableIdentifier identifier = TableIdentifier.of("default", "customers"); + + shell.executeStatement("CREATE EXTERNAL TABLE customers " + + "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + + testTables.locationForCreateTableSQL(identifier) + + "TBLPROPERTIES ('" + InputFormatConfig.TABLE_SCHEMA + "'='" + + SchemaParser.toJson(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA) + "', " + + "'" + InputFormatConfig.PARTITION_SPEC + "'='" + + PartitionSpecParser.toJson(PartitionSpec.unpartitioned()) + "', " + + "'dummy'='test')"); + + // Check the Iceberg table data + org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier); + Assert.assertEquals(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA.asStruct(), + icebergTable.schema().asStruct()); + Assert.assertEquals(PartitionSpec.unpartitioned(), icebergTable.spec()); + + // Check the HMS table parameters + org.apache.hadoop.hive.metastore.api.Table hmsTable = + shell.metastore().clientPool().run(client -> client.getTable("default", "customers")); + + Map hmsParams = hmsTable.getParameters(); + IGNORED_PARAMS.forEach(hmsParams::remove); + + // This is only set for HiveCatalog based tables. Check the value, then remove it so the other checks can be general + if (Catalogs.hiveCatalog(shell.getHiveConf())) { + Assert.assertTrue(hmsParams.get(BaseMetastoreTableOperations.METADATA_LOCATION_PROP) + .startsWith(icebergTable.location())); + hmsParams.remove(BaseMetastoreTableOperations.METADATA_LOCATION_PROP); + } + + // General metadata checks + Assert.assertEquals(6, hmsParams.size()); + Assert.assertEquals("test", hmsParams.get("dummy")); + Assert.assertEquals("TRUE", hmsParams.get(InputFormatConfig.EXTERNAL_TABLE_PURGE)); + Assert.assertEquals("TRUE", hmsParams.get("EXTERNAL")); + Assert.assertNotNull(hmsParams.get(hive_metastoreConstants.DDL_TIME)); + Assert.assertEquals(HiveIcebergStorageHandler.class.getName(), + hmsTable.getParameters().get(hive_metastoreConstants.META_TABLE_STORAGE)); + Assert.assertEquals(BaseMetastoreTableOperations.ICEBERG_TABLE_TYPE_VALUE.toUpperCase(), + hmsTable.getParameters().get(BaseMetastoreTableOperations.TABLE_TYPE_PROP)); + + // verify that storage descriptor is filled out with inputformat/outputformat/serde + Assert.assertEquals(HiveIcebergInputFormat.class.getName(), hmsTable.getSd().getInputFormat()); + Assert.assertEquals(HiveIcebergOutputFormat.class.getName(), hmsTable.getSd().getOutputFormat()); + Assert.assertEquals(HiveIcebergSerDe.class.getName(), hmsTable.getSd().getSerdeInfo().getSerializationLib()); + + if (!Catalogs.hiveCatalog(shell.getHiveConf())) { + Assert.assertEquals(Collections.singletonMap("dummy", "test"), icebergTable.properties()); + + shell.executeStatement("DROP TABLE customers"); + + // Check if the table was really dropped even from the Catalog + AssertHelpers.assertThrows("should throw exception", NoSuchTableException.class, + "Table does not exist", () -> { + testTables.loadTable(identifier); + } + ); + } else { + Map expectedIcebergProperties = new HashMap<>(2); + expectedIcebergProperties.put("dummy", "test"); + expectedIcebergProperties.put(TableProperties.ENGINE_HIVE_ENABLED, "true"); + Assert.assertEquals(expectedIcebergProperties, icebergTable.properties()); + + // Check the HMS table parameters + hmsTable = shell.metastore().clientPool().run(client -> client.getTable("default", "customers")); + Path hmsTableLocation = new Path(hmsTable.getSd().getLocation()); + + // Drop the table + shell.executeStatement("DROP TABLE customers"); + + // Check if we drop an exception when trying to load the table + AssertHelpers.assertThrows("should throw exception", NoSuchTableException.class, + "Table does not exist", () -> { + testTables.loadTable(identifier); + } + ); + + // Check if the files are removed + FileSystem fs = Util.getFs(hmsTableLocation, shell.getHiveConf()); + if (fs.exists(hmsTableLocation)) { + // if table directory has been deleted, we're good. This is the expected behavior in Hive4. + // if table directory exists, its contents should have been cleaned up, save for an empty metadata dir (Hive3). + Assert.assertEquals(1, fs.listStatus(hmsTableLocation).length); + Assert.assertEquals(0, fs.listStatus(new Path(hmsTableLocation, "metadata")).length); + } + } + } + + @Test + public void testCreateTableWithoutSpec() throws TException, InterruptedException { + TableIdentifier identifier = TableIdentifier.of("default", "customers"); + + shell.executeStatement("CREATE EXTERNAL TABLE customers " + + "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + + testTables.locationForCreateTableSQL(identifier) + + "TBLPROPERTIES ('" + InputFormatConfig.TABLE_SCHEMA + "'='" + + SchemaParser.toJson(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA) + "')"); + + // Check the Iceberg table partition data + org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier); + Assert.assertEquals(PartitionSpec.unpartitioned(), icebergTable.spec()); + + // Check the HMS table parameters + org.apache.hadoop.hive.metastore.api.Table hmsTable = + shell.metastore().clientPool().run(client -> client.getTable("default", "customers")); + + Map hmsParams = hmsTable.getParameters(); + IGNORED_PARAMS.forEach(hmsParams::remove); + + // Just check that the PartitionSpec is not set in the metadata + Assert.assertNull(hmsParams.get(InputFormatConfig.PARTITION_SPEC)); + + if (Catalogs.hiveCatalog(shell.getHiveConf())) { + Assert.assertEquals(6, hmsParams.size()); + } else { + Assert.assertEquals(5, hmsParams.size()); + } + } + + @Test + public void testCreateTableWithUnpartitionedSpec() throws TException, InterruptedException { + TableIdentifier identifier = TableIdentifier.of("default", "customers"); + + // We need the location for HadoopTable based tests only + shell.executeStatement("CREATE EXTERNAL TABLE customers " + + "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + + testTables.locationForCreateTableSQL(identifier) + + "TBLPROPERTIES ('" + InputFormatConfig.TABLE_SCHEMA + "'='" + + SchemaParser.toJson(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA) + "', " + + "'" + InputFormatConfig.PARTITION_SPEC + "'='" + + PartitionSpecParser.toJson(PartitionSpec.unpartitioned()) + "')"); + + // Check the Iceberg table partition data + org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier); + Assert.assertEquals(SPEC, icebergTable.spec()); + + // Check the HMS table parameters + org.apache.hadoop.hive.metastore.api.Table hmsTable = + shell.metastore().clientPool().run(client -> client.getTable("default", "customers")); + + Map hmsParams = hmsTable.getParameters(); + IGNORED_PARAMS.forEach(hmsParams::remove); + + // Just check that the PartitionSpec is not set in the metadata + Assert.assertNull(hmsParams.get(InputFormatConfig.PARTITION_SPEC)); + if (Catalogs.hiveCatalog(shell.getHiveConf())) { + Assert.assertEquals(6, hmsParams.size()); + } else { + Assert.assertEquals(5, hmsParams.size()); + } + } + + @Test + public void testDeleteBackingTable() throws TException, IOException, InterruptedException { + TableIdentifier identifier = TableIdentifier.of("default", "customers"); + + shell.executeStatement("CREATE EXTERNAL TABLE customers " + + "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + + testTables.locationForCreateTableSQL(identifier) + + "TBLPROPERTIES ('" + InputFormatConfig.TABLE_SCHEMA + "'='" + + SchemaParser.toJson(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA) + "', " + + "'" + InputFormatConfig.EXTERNAL_TABLE_PURGE + "'='FALSE')"); + + if (!Catalogs.hiveCatalog(shell.getHiveConf())) { + shell.executeStatement("DROP TABLE customers"); + + // Check if the table remains + testTables.loadTable(identifier); + } else { + // Check the HMS table parameters + org.apache.hadoop.hive.metastore.api.Table hmsTable = + shell.metastore().clientPool().run(client -> client.getTable("default", "customers")); + Path hmsTableLocation = new Path(hmsTable.getSd().getLocation()); + + // Drop the table + shell.executeStatement("DROP TABLE customers"); + + // Check if we drop an exception when trying to drop the table + AssertHelpers.assertThrows("should throw exception", NoSuchTableException.class, + "Table does not exist", () -> { + testTables.loadTable(identifier); + } + ); + + // Check if the files are kept + FileSystem fs = Util.getFs(hmsTableLocation, shell.getHiveConf()); + Assert.assertEquals(1, fs.listStatus(hmsTableLocation).length); + Assert.assertEquals(1, fs.listStatus(new Path(hmsTableLocation, "metadata")).length); + } + } + + @Test + public void testCreateTableError() { + TableIdentifier identifier = TableIdentifier.of("default", "withShell2"); + + // Wrong schema + AssertHelpers.assertThrows("should throw exception", IllegalArgumentException.class, + "Unrecognized token 'WrongSchema'", () -> { + shell.executeStatement("CREATE EXTERNAL TABLE withShell2 " + + "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + + testTables.locationForCreateTableSQL(identifier) + + "TBLPROPERTIES ('" + InputFormatConfig.TABLE_SCHEMA + "'='WrongSchema')"); + } + ); + + // Missing schema, we try to get the schema from the table and fail + AssertHelpers.assertThrows("should throw exception", IllegalArgumentException.class, + "Please provide ", () -> { + shell.executeStatement("CREATE EXTERNAL TABLE withShell2 " + + "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + + testTables.locationForCreateTableSQL(identifier)); + } + ); + + if (!testTables.locationForCreateTableSQL(identifier).isEmpty()) { + // Only test this if the location is required + AssertHelpers.assertThrows("should throw exception", IllegalArgumentException.class, + "Table location not set", () -> { + shell.executeStatement("CREATE EXTERNAL TABLE withShell2 " + + "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + + "TBLPROPERTIES ('" + InputFormatConfig.TABLE_SCHEMA + "'='" + + SchemaParser.toJson(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA) + "')"); + } + ); + } + } + + @Test + public void testCreateTableAboveExistingTable() throws TException, IOException, InterruptedException { + // Create the Iceberg table + testTables.createIcebergTable(shell.getHiveConf(), "customers", COMPLEX_SCHEMA, FileFormat.PARQUET, + Collections.emptyList()); + + if (Catalogs.hiveCatalog(shell.getHiveConf())) { + + // In HiveCatalog we just expect an exception since the table is already exists + AssertHelpers.assertThrows("should throw exception", IllegalArgumentException.class, + "customers already exists", () -> { + shell.executeStatement("CREATE EXTERNAL TABLE customers " + + "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + + "TBLPROPERTIES ('" + InputFormatConfig.TABLE_SCHEMA + "'='" + + SchemaParser.toJson(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA) + "')"); + } + ); + } else { + shell.executeStatement("CREATE EXTERNAL TABLE customers " + + "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + + testTables.locationForCreateTableSQL(TableIdentifier.of("default", "customers"))); + + // Check the HMS table parameters + org.apache.hadoop.hive.metastore.api.Table hmsTable = + shell.metastore().clientPool().run(client -> client.getTable("default", "customers")); + + Map hmsParams = hmsTable.getParameters(); + IGNORED_PARAMS.forEach(hmsParams::remove); + + Assert.assertEquals(4, hmsParams.size()); + Assert.assertEquals("TRUE", hmsParams.get("EXTERNAL")); + Assert.assertNotNull(hmsParams.get(hive_metastoreConstants.DDL_TIME)); + Assert.assertEquals(HiveIcebergStorageHandler.class.getName(), + hmsTable.getParameters().get(hive_metastoreConstants.META_TABLE_STORAGE)); + Assert.assertEquals(BaseMetastoreTableOperations.ICEBERG_TABLE_TYPE_VALUE.toUpperCase(), + hmsTable.getParameters().get(BaseMetastoreTableOperations.TABLE_TYPE_PROP)); + } + } + + @Test + public void testCreatePartitionedTableWithPropertiesAndWithColumnSpecification() { + PartitionSpec spec = + PartitionSpec.builderFor(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA).identity("last_name").build(); + + AssertHelpers.assertThrows("should throw exception", IllegalArgumentException.class, + "Provide only one of the following", () -> { + shell.executeStatement("CREATE EXTERNAL TABLE customers (customer_id BIGINT) " + + "PARTITIONED BY (first_name STRING) " + + "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + + testTables.locationForCreateTableSQL(TableIdentifier.of("default", "customers")) + + " TBLPROPERTIES ('" + InputFormatConfig.PARTITION_SPEC + "'='" + + PartitionSpecParser.toJson(spec) + "')"); + } + ); + } + + @Test + public void testCreateTableWithColumnSpecificationHierarchy() { + TableIdentifier identifier = TableIdentifier.of("default", "customers"); + + shell.executeStatement("CREATE EXTERNAL TABLE customers (" + + "id BIGINT, name STRING, " + + "employee_info STRUCT < employer: STRING, id: BIGINT, address: STRING >, " + + "places_lived ARRAY < STRUCT >, " + + "memorable_moments MAP < STRING, STRUCT < year: INT, place: STRING, details: STRING >>, " + + "current_address STRUCT < street_address: STRUCT " + + ", country: STRING, postal_code: STRING >) " + + "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + + testTables.locationForCreateTableSQL(identifier)); + + // Check the Iceberg table data + org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier); + Assert.assertEquals(COMPLEX_SCHEMA.asStruct(), icebergTable.schema().asStruct()); + } + + @Test + public void testCreateTableWithAllSupportedTypes() { + TableIdentifier identifier = TableIdentifier.of("default", "all_types"); + Schema allSupportedSchema = new Schema( + optional(1, "t_float", Types.FloatType.get()), + optional(2, "t_double", Types.DoubleType.get()), + optional(3, "t_boolean", Types.BooleanType.get()), + optional(4, "t_int", Types.IntegerType.get()), + optional(5, "t_bigint", Types.LongType.get()), + optional(6, "t_binary", Types.BinaryType.get()), + optional(7, "t_string", Types.StringType.get()), + optional(8, "t_timestamp", Types.TimestampType.withoutZone()), + optional(9, "t_date", Types.DateType.get()), + optional(10, "t_decimal", Types.DecimalType.of(3, 2)) + ); + + // Intentionally adding some mixed letters to test that we handle them correctly + shell.executeStatement("CREATE EXTERNAL TABLE all_types (" + + "t_Float FLOaT, t_dOuble DOUBLE, t_boolean BOOLEAN, t_int INT, t_bigint BIGINT, t_binary BINARY, " + + "t_string STRING, t_timestamp TIMESTAMP, t_date DATE, t_decimal DECIMAL(3,2)) " + + "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + + testTables.locationForCreateTableSQL(identifier)); + + // Check the Iceberg table data + org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier); + Assert.assertEquals(allSupportedSchema.asStruct(), icebergTable.schema().asStruct()); + } + + @Test + public void testCreateTableWithNotSupportedTypes() { + TableIdentifier identifier = TableIdentifier.of("default", "not_supported_types"); + // Can not create INTERVAL types from normal create table, so leave them out from this test + String[] notSupportedTypes = new String[] { "TINYINT", "SMALLINT", "VARCHAR(1)", "CHAR(1)" }; + + for (String notSupportedType : notSupportedTypes) { + AssertHelpers.assertThrows("should throw exception", IllegalArgumentException.class, + "Unsupported Hive type", () -> { + shell.executeStatement("CREATE EXTERNAL TABLE not_supported_types " + + "(not_supported " + notSupportedType + ") " + + "STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' " + + testTables.locationForCreateTableSQL(identifier)); + } + ); + } + } +} diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithEngine.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithEngine.java new file mode 100644 index 000000000000..c545a19d138f --- /dev/null +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithEngine.java @@ -0,0 +1,206 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.mr.hive; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.mr.TestHelper; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.junit.runners.Parameterized.Parameter; +import static org.junit.runners.Parameterized.Parameters; + +@RunWith(Parameterized.class) +public class TestHiveIcebergStorageHandlerWithEngine { + + private static final String[] EXECUTION_ENGINES = new String[] {"tez", "mr"}; + + private static final Schema ORDER_SCHEMA = new Schema( + required(1, "order_id", Types.LongType.get()), + required(2, "customer_id", Types.LongType.get()), + required(3, "total", Types.DoubleType.get())); + + private static final List ORDER_RECORDS = TestHelper.RecordsBuilder.newInstance(ORDER_SCHEMA) + .add(100L, 0L, 11.11d) + .add(101L, 0L, 22.22d) + .add(102L, 1L, 33.33d) + .build(); + + private static final List SUPPORTED_TYPES = + ImmutableList.of(Types.BooleanType.get(), Types.IntegerType.get(), Types.LongType.get(), + Types.FloatType.get(), Types.DoubleType.get(), Types.DateType.get(), Types.TimestampType.withZone(), + Types.TimestampType.withoutZone(), Types.StringType.get(), Types.BinaryType.get(), + Types.DecimalType.of(3, 1)); + + @Parameters(name = "fileFormat={0}, engine={1}, catalog={2}") + public static Collection parameters() { + Collection testParams = new ArrayList<>(); + String javaVersion = System.getProperty("java.specification.version"); + + // Run tests with every FileFormat for a single Catalog (HiveCatalog) + for (FileFormat fileFormat : HiveIcebergStorageHandlerTestUtils.FILE_FORMATS) { + for (String engine : EXECUTION_ENGINES) { + // include Tez tests only for Java 8 + if (javaVersion.equals("1.8") || "mr".equals(engine)) { + testParams.add(new Object[] {fileFormat, engine, TestTables.TestTableType.HIVE_CATALOG}); + } + } + } + + // Run tests for every Catalog for a single FileFormat (PARQUET) and execution engine (mr) + // skip HiveCatalog tests as they are added before + for (TestTables.TestTableType testTableType : TestTables.ALL_TABLE_TYPES) { + if (!TestTables.TestTableType.HIVE_CATALOG.equals(testTableType)) { + testParams.add(new Object[]{FileFormat.PARQUET, "mr", testTableType}); + } + } + + return testParams; + } + + private static TestHiveShell shell; + + private TestTables testTables; + + @Parameter(0) + public FileFormat fileFormat; + + @Parameter(1) + public String executionEngine; + + @Parameter(2) + public TestTables.TestTableType testTableType; + + @Rule + public TemporaryFolder temp = new TemporaryFolder(); + + @BeforeClass + public static void beforeClass() { + shell = HiveIcebergStorageHandlerTestUtils.shell(); + } + + @AfterClass + public static void afterClass() { + shell.stop(); + } + + @Before + public void before() throws IOException { + testTables = HiveIcebergStorageHandlerTestUtils.testTables(shell, testTableType, temp); + HiveIcebergStorageHandlerTestUtils.init(shell, testTables, temp, executionEngine); + } + + @After + public void after() throws Exception { + shell.closeSession(); + shell.metastore().reset(); + // HiveServer2 thread pools are using thread local Hive -> HMSClient objects. These are not cleaned up when the + // HiveServer2 is stopped. Only Finalizer closes the HMS connections. + System.gc(); + } + + @Test + public void testScanTable() throws IOException { + testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, fileFormat, + HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS); + + // Adding the ORDER BY clause will cause Hive to spawn a local MR job this time. + List descRows = + shell.executeStatement("SELECT first_name, customer_id FROM default.customers ORDER BY customer_id DESC"); + + Assert.assertEquals(3, descRows.size()); + Assert.assertArrayEquals(new Object[] {"Trudy", 2L}, descRows.get(0)); + Assert.assertArrayEquals(new Object[] {"Bob", 1L}, descRows.get(1)); + Assert.assertArrayEquals(new Object[] {"Alice", 0L}, descRows.get(2)); + } + + @Test + public void testJoinTables() throws IOException { + testTables.createTable(shell, "customers", HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, fileFormat, + HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS); + testTables.createTable(shell, "orders", ORDER_SCHEMA, fileFormat, ORDER_RECORDS); + + List rows = shell.executeStatement( + "SELECT c.customer_id, c.first_name, o.order_id, o.total " + + "FROM default.customers c JOIN default.orders o ON c.customer_id = o.customer_id " + + "ORDER BY c.customer_id, o.order_id" + ); + + Assert.assertArrayEquals(new Object[] {0L, "Alice", 100L, 11.11d}, rows.get(0)); + Assert.assertArrayEquals(new Object[] {0L, "Alice", 101L, 22.22d}, rows.get(1)); + Assert.assertArrayEquals(new Object[] {1L, "Bob", 102L, 33.33d}, rows.get(2)); + } + + @Test + public void testJoinTablesSupportedTypes() throws IOException { + for (int i = 0; i < SUPPORTED_TYPES.size(); i++) { + Type type = SUPPORTED_TYPES.get(i); + String tableName = type.typeId().toString().toLowerCase() + "_table_" + i; + String columnName = type.typeId().toString().toLowerCase() + "_column"; + + Schema schema = new Schema(required(1, columnName, type)); + List records = TestHelper.generateRandomRecords(schema, 1, 0L); + + testTables.createTable(shell, tableName, schema, fileFormat, records); + List queryResult = shell.executeStatement("select s." + columnName + ", h." + columnName + + " from default." + tableName + " s join default." + tableName + " h on h." + columnName + "=s." + + columnName); + Assert.assertEquals("Non matching record count for table " + tableName + " with type " + type, + 1, queryResult.size()); + } + } + + @Test + public void testSelectDistinctFromTable() throws IOException { + for (int i = 0; i < SUPPORTED_TYPES.size(); i++) { + Type type = SUPPORTED_TYPES.get(i); + String tableName = type.typeId().toString().toLowerCase() + "_table_" + i; + String columnName = type.typeId().toString().toLowerCase() + "_column"; + + Schema schema = new Schema(required(1, columnName, type)); + List records = TestHelper.generateRandomRecords(schema, 4, 0L); + int size = records.stream().map(r -> r.getField(columnName)).collect(Collectors.toSet()).size(); + testTables.createTable(shell, tableName, schema, fileFormat, records); + List queryResult = shell.executeStatement("select count(distinct(" + columnName + + ")) from default." + tableName); + int distinctIds = ((Long) queryResult.get(0)[0]).intValue(); + Assert.assertEquals(tableName, size, distinctIds); + } + } +} From c8543bbf8b807cc9e38e8440b3dd12a2d8477847 Mon Sep 17 00:00:00 2001 From: Steven Wu Date: Mon, 14 Dec 2020 15:17:33 -0800 Subject: [PATCH 08/12] upgrade flink version from 1.11.0 to 1.12.0 --- .../main/java/org/apache/iceberg/flink/FlinkCatalog.java | 2 +- .../main/java/org/apache/iceberg/flink/sink/FlinkSink.java | 4 ++-- .../java/org/apache/iceberg/flink/source/FlinkSource.java | 7 ++++--- .../test/java/org/apache/iceberg/flink/FlinkTestBase.java | 2 +- .../org/apache/iceberg/flink/TestFlinkCatalogTable.java | 6 +++--- .../test/java/org/apache/iceberg/flink/TestHelpers.java | 6 ++---- .../apache/iceberg/flink/sink/TestFlinkIcebergSink.java | 4 ++-- versions.props | 2 +- 8 files changed, 16 insertions(+), 17 deletions(-) diff --git a/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java b/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java index 2fd569628864..23de494b4dd0 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java +++ b/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java @@ -439,7 +439,7 @@ private static void validateFlinkTable(CatalogBaseTable table) { TableSchema schema = table.getSchema(); schema.getTableColumns().forEach(column -> { - if (column.isGenerated()) { + if (!column.isPhysical()) { throw new UnsupportedOperationException("Creating table with computed columns is not supported yet."); } }); diff --git a/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java index 8c4486aa28c6..96af3f54e783 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java @@ -33,7 +33,7 @@ import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.util.DataFormatConverters; -import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.types.Row; @@ -94,7 +94,7 @@ public static Builder forRow(DataStream input, TableSchema tableSchema) { DataType[] fieldDataTypes = tableSchema.getFieldDataTypes(); DataFormatConverters.RowConverter rowConverter = new DataFormatConverters.RowConverter(fieldDataTypes); - return builderFor(input, rowConverter::toInternal, RowDataTypeInfo.of(rowType)) + return builderFor(input, rowConverter::toInternal, InternalTypeInfo.of(rowType)) .tableSchema(tableSchema); } diff --git a/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java b/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java index e4b5907af585..4c5304c9cf02 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java +++ b/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java @@ -23,11 +23,12 @@ import java.io.UncheckedIOException; import java.util.List; import java.util.Map; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; -import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; @@ -73,7 +74,7 @@ public static class Builder { private long limit; private ScanContext context = new ScanContext(); - private RowDataTypeInfo rowTypeInfo; + private TypeInformation rowTypeInfo; public Builder tableLoader(TableLoader newLoader) { this.tableLoader = newLoader; @@ -178,7 +179,7 @@ public FlinkInputFormat buildFormat() { encryption = table.encryption(); } - rowTypeInfo = RowDataTypeInfo.of((RowType) ( + rowTypeInfo = InternalTypeInfo.of((RowType) ( projectedSchema == null ? FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergSchema)) : projectedSchema).toRowDataType().getLogicalType()); diff --git a/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java b/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java index 6782267fdd86..af53d990b14f 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java +++ b/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java @@ -76,7 +76,7 @@ protected List sql(String query, Object... args) { TableResult tableResult = getTableEnv().executeSql(String.format(query, args)); tableResult.getJobClient().ifPresent(c -> { try { - c.getJobExecutionResult(Thread.currentThread().getContextClassLoader()).get(); + c.getJobExecutionResult().get(); } catch (InterruptedException | ExecutionException e) { throw new RuntimeException(e); } diff --git a/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java b/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java index 724fd110348e..28439fb2c450 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java +++ b/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java @@ -84,8 +84,8 @@ public void testGetTable() { Types.NestedField.optional(1, "strV", Types.StringType.get()))); Assert.assertEquals( Arrays.asList( - TableColumn.of("id", DataTypes.BIGINT()), - TableColumn.of("strV", DataTypes.STRING())), + TableColumn.physical("id", DataTypes.BIGINT()), + TableColumn.physical("strV", DataTypes.STRING())), getTableEnv().from("tl").getSchema().getTableColumns()); Assert.assertTrue(getTableEnv().getCatalog(catalogName).get().tableExists(ObjectPath.fromString("db.tl"))); } @@ -105,7 +105,7 @@ public void testRenameTable() { () -> getTableEnv().from("tl") ); Assert.assertEquals( - Collections.singletonList(TableColumn.of("id", DataTypes.BIGINT())), + Collections.singletonList(TableColumn.physical("id", DataTypes.BIGINT())), getTableEnv().from("tl2").getSchema().getTableColumns()); } diff --git a/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java b/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java index e4ca091f76d9..62865f5a4990 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java +++ b/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java @@ -31,7 +31,6 @@ import java.util.Map; import java.util.UUID; import java.util.stream.Collectors; -import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.DecimalData; @@ -41,7 +40,7 @@ import org.apache.flink.table.data.TimestampData; import org.apache.flink.table.data.conversion.DataStructureConverter; import org.apache.flink.table.data.conversion.DataStructureConverters; -import org.apache.flink.table.runtime.types.InternalSerializers; +import org.apache.flink.table.runtime.typeutils.InternalSerializers; import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.MapType; @@ -62,9 +61,8 @@ private TestHelpers() { } public static RowData copyRowData(RowData from, RowType rowType) { - ExecutionConfig config = new ExecutionConfig(); TypeSerializer[] fieldSerializers = rowType.getChildren().stream() - .map((LogicalType type) -> InternalSerializers.create(type, config)) + .map((LogicalType type) -> InternalSerializers.create(type)) .toArray(TypeSerializer[]::new); // Use rowType field count to avoid copy metadata column in case of merging position deletes diff --git a/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java index 90ac3a2e3187..1c612f8e57b2 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java +++ b/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java @@ -32,7 +32,7 @@ import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.util.DataFormatConverters; -import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.test.util.AbstractTestBase; import org.apache.flink.types.Row; import org.apache.iceberg.FileFormat; @@ -129,7 +129,7 @@ public void testWriteRowData() throws Exception { Row.of(3, "foo") ); DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) - .map(CONVERTER::toInternal, RowDataTypeInfo.of(SimpleDataUtil.ROW_TYPE)); + .map(CONVERTER::toInternal, InternalTypeInfo.of(SimpleDataUtil.ROW_TYPE)); FlinkSink.forRowData(dataStream) .table(table) diff --git a/versions.props b/versions.props index 43ce4017a93c..bb55b65b4578 100644 --- a/versions.props +++ b/versions.props @@ -1,6 +1,6 @@ org.slf4j:* = 1.7.25 org.apache.avro:avro = 1.9.2 -org.apache.flink:* = 1.11.0 +org.apache.flink:* = 1.12.0 org.apache.hadoop:* = 2.7.3 org.apache.hive:hive-metastore = 2.3.7 org.apache.hive:hive-serde = 2.3.7 From b257ebf9becd684e74c93522ceeede5b74174ff0 Mon Sep 17 00:00:00 2001 From: Steven Wu Date: Fri, 18 Dec 2020 10:24:11 -0800 Subject: [PATCH 09/12] close table result iterator in FlinkTestBase for sql execution --- .../test/java/org/apache/iceberg/flink/FlinkTestBase.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java b/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java index af53d990b14f..b4d386bc681b 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java +++ b/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java @@ -87,6 +87,11 @@ protected List sql(String query, Object... args) { Row row = iter.next(); results.add(IntStream.range(0, row.getArity()).mapToObj(row::getField).toArray(Object[]::new)); } + try { + iter.close(); + } catch (Exception e) { + throw new RuntimeException("failed to close table result iterator", e); + } return results; } } From b87293c6cfe9da03a278397753307147b5df0db3 Mon Sep 17 00:00:00 2001 From: Steven Wu Date: Tue, 22 Dec 2020 10:09:20 -0800 Subject: [PATCH 10/12] only create table env once and close the table result itertor in TestFlinkScanSql --- .../flink/source/TestFlinkScanSql.java | 33 +++++++++++++++---- 1 file changed, 27 insertions(+), 6 deletions(-) diff --git a/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java b/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java index 4231e335bd2b..2d753dc0d9b3 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java +++ b/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java @@ -26,6 +26,7 @@ import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; import org.apache.iceberg.DataFile; import org.apache.iceberg.Table; import org.apache.iceberg.TestHelpers; @@ -43,7 +44,7 @@ */ public class TestFlinkScanSql extends TestFlinkScan { - private TableEnvironment tEnv; + private volatile TableEnvironment tEnv; public TestFlinkScanSql(String fileFormat) { super(fileFormat); @@ -52,12 +53,25 @@ public TestFlinkScanSql(String fileFormat) { @Override public void before() throws IOException { super.before(); - tEnv = TableEnvironment.create(EnvironmentSettings.newInstance().useBlinkPlanner().inBatchMode().build()); - tEnv.executeSql(String.format( + getTableEnv().executeSql(String.format( "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", warehouse)); - tEnv.executeSql("use catalog iceberg_catalog"); - tEnv.getConfig().getConfiguration().set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); + getTableEnv().executeSql("use catalog iceberg_catalog"); + getTableEnv().getConfig().getConfiguration().set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); + } + + private TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + if (tEnv == null) { + this.tEnv = TableEnvironment.create(EnvironmentSettings + .newInstance() + .useBlinkPlanner() + .inBatchMode().build()); + } + } + } + return tEnv; } @Override @@ -107,7 +121,14 @@ public void testResiduals() throws Exception { } private List executeSQL(String sql) { - return Lists.newArrayList(tEnv.executeSql(sql).collect()); + CloseableIterator iter = getTableEnv().executeSql(sql).collect(); + List results = Lists.newArrayList(iter); + try { + iter.close(); + } catch (Exception e) { + throw new RuntimeException("failed to close table result iterator", e); + } + return results; } private String optionToKv(String key, Object value) { From d35f96d4ddbddc9c4fb7e8fd18afb45cc64c8431 Mon Sep 17 00:00:00 2001 From: Steven Wu Date: Wed, 30 Dec 2020 15:13:01 -0800 Subject: [PATCH 11/12] disable CoreOptions.CHECK_LEAKED_CLASSLOADER for FlinkTestBase --- .../apache/iceberg/flink/FlinkTestBase.java | 27 +++++++++++++++++-- 1 file changed, 25 insertions(+), 2 deletions(-) diff --git a/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java b/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java index b4d386bc681b..2e3efbe238a7 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java +++ b/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java @@ -22,10 +22,13 @@ import java.util.List; import java.util.concurrent.ExecutionException; import java.util.stream.IntStream; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.TableResult; -import org.apache.flink.test.util.AbstractTestBase; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.util.TestBaseUtils; import org.apache.flink.types.Row; import org.apache.flink.util.CloseableIterator; import org.apache.hadoop.hive.conf.HiveConf; @@ -34,8 +37,28 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.rules.TemporaryFolder; -public abstract class FlinkTestBase extends AbstractTestBase { +public abstract class FlinkTestBase extends TestBaseUtils { + + private static final int DEFAULT_PARALLELISM = 4; + + private static final org.apache.flink.configuration.Configuration config = + new org.apache.flink.configuration.Configuration() + // disable classloader check as Avro may cache class/object in the serializers. + .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); + + @ClassRule + public static MiniClusterWithClientResource miniClusterResource = new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) + .setConfiguration(config) + .build()); + + @ClassRule + public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); private static TestHiveMetastore metastore = null; protected static HiveConf hiveConf = null; From f33e94c29845ffee9a2586098a942182f965a623 Mon Sep 17 00:00:00 2001 From: Steven Wu Date: Mon, 4 Jan 2021 19:58:32 -0800 Subject: [PATCH 12/12] use try-with-resources statement for collecting table result --- .../apache/iceberg/flink/FlinkTestBase.java | 24 ++++++------------- .../flink/source/TestFlinkScanSql.java | 12 +++++----- 2 files changed, 13 insertions(+), 23 deletions(-) diff --git a/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java b/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java index 2e3efbe238a7..cedecb314570 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java +++ b/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java @@ -20,7 +20,6 @@ package org.apache.iceberg.flink; import java.util.List; -import java.util.concurrent.ExecutionException; import java.util.stream.IntStream; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; @@ -97,24 +96,15 @@ protected TableEnvironment getTableEnv() { protected List sql(String query, Object... args) { TableResult tableResult = getTableEnv().executeSql(String.format(query, args)); - tableResult.getJobClient().ifPresent(c -> { - try { - c.getJobExecutionResult().get(); - } catch (InterruptedException | ExecutionException e) { - throw new RuntimeException(e); + try (CloseableIterator iter = tableResult.collect()) { + List results = Lists.newArrayList(); + while (iter.hasNext()) { + Row row = iter.next(); + results.add(IntStream.range(0, row.getArity()).mapToObj(row::getField).toArray(Object[]::new)); } - }); - CloseableIterator iter = tableResult.collect(); - List results = Lists.newArrayList(); - while (iter.hasNext()) { - Row row = iter.next(); - results.add(IntStream.range(0, row.getArity()).mapToObj(row::getField).toArray(Object[]::new)); - } - try { - iter.close(); + return results; } catch (Exception e) { - throw new RuntimeException("failed to close table result iterator", e); + throw new RuntimeException("Failed to collect table result", e); } - return results; } } diff --git a/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java b/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java index 2d753dc0d9b3..d4bf9303d943 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java +++ b/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java @@ -24,6 +24,7 @@ import java.util.Map; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.types.Row; import org.apache.flink.util.CloseableIterator; @@ -121,14 +122,13 @@ public void testResiduals() throws Exception { } private List executeSQL(String sql) { - CloseableIterator iter = getTableEnv().executeSql(sql).collect(); - List results = Lists.newArrayList(iter); - try { - iter.close(); + TableResult tableResult = getTableEnv().executeSql(sql); + try (CloseableIterator iter = tableResult.collect()) { + List results = Lists.newArrayList(iter); + return results; } catch (Exception e) { - throw new RuntimeException("failed to close table result iterator", e); + throw new RuntimeException("Failed to collect table result", e); } - return results; } private String optionToKv(String key, Object value) {