silvermissile opened a new issue #2670: URL: https://github.com/apache/iceberg/issues/2670
I use hive sql to alter iceberg table createed by flinkSQL's iceberg-type catalog, but meet thie [Could Hive alter iceberg table? #2063](https://github.com/apache/iceberg/issues/2063) So I verify schema evolution though spark , but not work either. My attempt is as follows ## 1. create table by spark ``` java SparkSession spark = SparkSession.builder() .master("local[*]") .appName("iceberg demo") .enableHiveSupport().getOrCreate(); Configuration hadoopConfiguration = spark.sparkContext().hadoopConfiguration(); hadoopConfiguration.set(ConfigProperties.ENGINE_HIVE_ENABLED, "true"); Map<String, String> tableProperties = new HashMap<>(); tableProperties.put(TableProperties.ENGINE_HIVE_ENABLED, "true"); List<Types.NestedField> fields = new LinkedList<Types.NestedField>(); fields.add(Types.NestedField.optional(1, "city_name", Types.StringType.get())); fields.add(Types.NestedField.optional(2, "item_id", Types.LongType.get())); fields.add(Types.NestedField.optional(3, "price", Types.DoubleType.get())); fields.add(Types.NestedField.optional(4, "time_stamp", Types.LongType.get())); fields.add(Types.NestedField.optional(5, "user_id", Types.LongType.get())); Schema schema = new Schema(fields); TableIdentifier tableIdentifier = TableIdentifier.of(Namespace.of("tmp"), "schematest0603"); HiveCatalog catalog = new HiveCatalog(hadoopConfiguration); catalog.createTable(tableIdentifier, schema, null, tableProperties); ``` ## 2. alter talbe by hive ``` sh 0: jdbc:hive2://> show create table schematest0603; +----------------------------------------------------+ | createtab_stmt | +----------------------------------------------------+ | CREATE EXTERNAL TABLE `schematest0603`( | | `city_name` string COMMENT 'from deserializer', | | `item_id` bigint COMMENT 'from deserializer', | | `price` double COMMENT 'from deserializer', | | `time_stamp` bigint COMMENT 'from deserializer', | | `user_id` bigint COMMENT 'from deserializer') | | ROW FORMAT SERDE | | 'org.apache.iceberg.mr.hive.HiveIcebergSerDe' | | STORED BY | | 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' | | | | LOCATION | | 'hdfs://nameservice-nkb/data/hive236/warehouse/tmp.db/schematest0603' | | TBLPROPERTIES ( | | 'metadata_location'='hdfs://nameservice-nkb/data/hive236/warehouse/tmp.db/schematest0603/metadata/00000-e7f67410-d346-42d5-a887-9b9ac03665b6.metadata.json', | | 'spark.sql.create.version'='2.2 or prior', | | 'spark.sql.sources.schema.numParts'='1', | | 'spark.sql.sources.schema.part.0'='{"type":"struct","fields":[{"name":"city_name","type":"string","nullable":true,"metadata":{"comment":"from deserializer"}},{"name":"item_id","type":"long","nullable":true,"metadata":{"comment":"from deserializer"}},{"name":"price","type":"double","nullable":true,"metadata":{"comment":"from deserializer"}},{"name":"time_stamp","type":"long","nullable":true,"metadata":{"comment":"from deserializer"}},{"name":"user_id","type":"long","nullable":true,"metadata":{"comment":"from deserializer"}},{"name":"age","type":"string","nullable":true,"metadata":{"comment":"age"}}]}', | | 'table_type'='ICEBERG', | | 'transient_lastDdlTime'='1622710912') | 21/06/03 09:10:43 [main]: DEBUG logs.InPlaceUpdateStream$EventNotifier: progress bar is complete 21/06/03 09:10:43 [main]: DEBUG transport.TSaslTransport: writing data length: 112 21/06/03 09:10:43 [main]: DEBUG transport.TSaslTransport: CLIENT: reading data length: 96 +----------------------------------------------------+ 0: jdbc:hive2://> ALTER TABLE tmp.schematest0603 ADD COLUMNS ( `age` string comment 'age' ); Error: Error while compiling statement: FAILED: SemanticException [Error 10134]: ALTER TABLE cannot be used for a non-native table schematest0603 (state=42000,code=10134) ``` ## 3. alter table by sparkSQL shell client `[spark-3.0.1-bin-hadoop2.7]$ bin/spark-sql --conf spark.sql.catalog.spark_catalog=org.apache.iceberg.spark.SparkSessionCatalog --conf spark.sql.catalog.spark_catalog.type=hive --jars jars/iceberg-spark3-runtime-0517.jar,iceberg-hive-runtime-0517.jar` ``` sh spark-sql> show create table spark_catalog.tmp.schematest0603 as serde; Error in query: Failed to execute SHOW CREATE TABLE against table/view `tmp`.`schematest0603`, which is created by Hive and uses the following unsupported feature(s) - storage handler; spark-sql> ALTER TABLE spark_catalog.tmp.schematest0603 ADD COLUMNS ( `age` string comment 'age' ); spark-sql> desc spark_catalog.tmp.schematest0603; 21/06/03 09:02:05 WARN HiveExternalCatalog: The table schema given by Hive metastore(struct<city_name:string,item_id:bigint,price:double,time_stamp:bigint,user_id:bigint>) is different from the schema when this table was created by Spark SQL(struct<city_name:string,item_id:bigint,price:double,time_stamp:bigint,user_id:bigint,age:string>). We have to fall back to the table schema from Hive metastore which is not case preserving. Hive metastore(struct<city_name:string,item_id:bigint,price:double,time_stamp:bigint,user_id:bigint>) SparkSQL (struct<city_name:string,item_id:bigint,price:double,time_stamp:bigint,user_id:bigint,age:string>). We have to fall back to the table schema from Hive metastore which is not case preserving. city_name string from deserializer item_id bigint from deserializer price double from deserializer time_stamp bigint from deserializer user_id bigint from deserializer Time taken: 0.475 seconds, Fetched 5 row(s) ``` `SparkSQL (struct<city_name:string,item_id:bigint,price:double,time_stamp:bigint,user_id:bigint,age:string>).` is shown in log but not effective; -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
