+1 (binding) from my side Here are the checks what I have done: Downloaded the source and built the code Checked the size of the iceberg-hive-runtime-0.11.0.jar Tried out the jar on a CDP cluster using Hue Create a table Inserted values into the table Selected the value from the table Selected the value from the table with join Created another table and inserted values into it using a select
Here are the settings I have used in my tests: SET iceberg.mr.catalog=hive; SET hive.execution.engine=mr; ADD JAR /tmp/iceberg-hive-runtime-0.11.0.jar; ADD JAR /opt/cloudera/parcels/CDH/jars/libfb303-0.9.3.jar It might be good to know that we have problems writing several types ATM. See: https://github.com/apache/iceberg/pull/2126 <https://github.com/apache/iceberg/pull/2126> I do not think it is blocker since writes are only experimental. Thanks, Peter > On Jan 25, 2021, at 10:24, Ryan Murray <rym...@dremio.com> wrote: > > I have moved back to +1 (non-binding) > > As you said Ryan, the error message is bad and hides the real error. While I > was testing misconfigured catalogs I kept getting the error 'Cannot > initialize Catalog, missing no-arg constructor: > org.apache.iceberg.hive.HiveCatalog' when the real error is (in this case) a > misconfigured Hive. I have raised #2145 > <https://github.com/apache/iceberg/issues/2145> to address this as it isn't > critical to the release. > > Best, > Ryan > > On Mon, Jan 25, 2021 at 12:05 AM Ryan Blue <rb...@netflix.com.invalid> wrote: > +1 (binding) > > Downloaded, validated checksum and signature, ran RAT checks, built binaries > and tested. > > Also checked Spark 2, Spark 3, and Hive 2: > > Created a new table in Spark 3.1.1 release candidate without the USING clause > Created a table in Spark 3.0.1 with CTAS and a USING clause > Created a new database in Spark 3.0.1 and validated the warehouse location > for new tables > Used Spark 3 extensions in 3.0.1 to add bucketing to a table > Deleted data from a table in Spark 3.0.1 > Ran merge statements in Spark 3.0.1 and validated join type optimizations > Used multi-catalog support in Spark 2.4.5 to read from testhive and prodhive > catalogs using the same config as Spark 3 > Tested multi-catalog metadata tables in Spark 2.4.5 > Tested input_file_name() in Spark 2.4.5 > Read from a Hive catalog table in Hive 2 > Here’s my command to start Spark 3: > > /home/blue/Apps/spark-3.0.1-bin-hadoop2.7/bin/spark-shell \ > --driver-java-options > -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005 \ > --conf > spark.jars.repositories=https://repository.apache.org/content/repositories/orgapacheiceberg-1015/ > <https://repository.apache.org/content/repositories/orgapacheiceberg-1015/> \ > --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0 \ > --conf > spark.sql.extensions=org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions > \ > --conf spark.hadoop.hive.metastore.uris=thrift://localhost:32917 \ > --conf spark.sql.catalog.local=org.apache.iceberg.spark.SparkCatalog \ > --conf spark.sql.catalog.local.type=hadoop \ > --conf spark.sql.catalog.local.warehouse=/home/blue/tmp/hadoop-warehouse \ > --conf spark.sql.catalog.prodhive=org.apache.iceberg.spark.SparkCatalog \ > --conf spark.sql.catalog.prodhive.type=hive \ > --conf spark.sql.catalog.prodhive.warehouse=/home/blue/tmp/prod-warehouse > \ > --conf spark.sql.catalog.prodhive.default-namespace=default \ > --conf spark.sql.catalog.testhive=org.apache.iceberg.spark.SparkCatalog \ > --conf spark.sql.catalog.testhive.type=hive \ > --conf spark.sql.catalog.testhive.uri=thrift://localhost:34847 \ > --conf spark.sql.catalog.testhive.warehouse=/home/blue/tmp/test-warehouse > \ > --conf spark.sql.catalog.testhive.default-namespace=default \ > --conf spark.sql.defaultCatalog=prodhive > And here’s a script to start Hive: > > /home/blue/Apps/apache-hive-2.3.7-bin/bin/hive --hiveconf > hive.metastore.uris=thrift://localhost:32917 > hive> SET iceberg.mr.catalog=hive; > hive> ADD JAR /home/blue/Downloads/iceberg-hive-runtime-0.11.0.jar; > The only issue I found is that the Spark 3.1.1 release candidate can’t use > the extensions module because an internal variable substitution class changed > in 3.1.x. I don’t think that should fail this release, we can do more > thorough testing with 3.1.1 once it is released and fix problems in a point > release. > > > On Fri, Jan 22, 2021 at 3:26 PM Jack Ye <yezhao...@gmail.com > <mailto:yezhao...@gmail.com>> wrote: > Hi everyone, > > I propose the following RC to be released as the official Apache Iceberg > 0.11.0 release. The RC is also reviewed and signed by Ryan Blue. > > The commit id is ad78cc6cf259b7a0c66ab5de6675cc005febd939 > > This corresponds to the tag: apache-iceberg-0.11.0-rc0 > * https://github.com/apache/iceberg/commits/apache-iceberg-0.11.0-rc0 > <https://github.com/apache/iceberg/commits/apache-iceberg-0.11.0-rc0> > * https://github.com/apache/iceberg/tree/apache-iceberg-0.11.0-rc0 > <https://github.com/apache/iceberg/tree/apache-iceberg-0.11.0-rc0> > > The release tarball, signature, and checksums are here: > * https://dist.apache.org/repos/dist/dev/iceberg/apache-iceberg-0.11.0-rc0 > <https://dist.apache.org/repos/dist/dev/iceberg/apache-iceberg-0.11.0-rc0> > > You can find the KEYS file here: > * https://dist.apache.org/repos/dist/dev/iceberg/KEYS > <https://dist.apache.org/repos/dist/dev/iceberg/KEYS> > > Convenience binary artifacts are staged in Nexus. The Maven repository URL is: > * https://repository.apache.org/content/repositories/orgapacheiceberg-1015 > <https://repository.apache.org/content/repositories/orgapacheiceberg-1015> > > This release includes the following changes: > > High-level features > Core API now supports partition spec and sort order evolution > Spark 3 now supports the following SQL extensions: > MERGE INTO > DELETE FROM > ALTER TABLE ... ADD/DROP PARTITION > ALTER TABLE ... WRITE ORDERED BY > invoke stored procedures using CALL > Flink now supports streaming reads, CDC writes (experimental), and filter > pushdown > AWS module is added to support better integration with AWS, with AWS Glue > catalog <https://aws.amazon.com/glue> support and dedicated S3 FileIO > implementation > Nessie module is added to support integration with project Nessie > <https://projectnessie.org/> > Important bug fixes > #1981 fixes date and timestamp transforms > #2091 fixes Parquet vectorized reads when column types are promoted > #1962 fixes Parquet vectorized position reader > #1991 fixes Avro schema conversions to preserve field docs > #1811 makes refreshing Spark cache optional > #1798 fixes read failure when encountering duplicate entries of data files > #1785 fixes invalidation of metadata tables in CachingCatalog > #1784 fixes resolving of SparkSession table's metadata tables > Other notable changes > NaN counter is added to format v2 metrics > Shared catalog properties are added in core library to standardize catalog > level configurations > Spark and Flink now supports dynamically loading customized `Catalog` and > `FileIO` implementations > Spark now supports loading tables with file paths via HadoopTables > Spark 2 now supports loading tables from other catalogs, like Spark 3 > Spark 3 now supports catalog names in DataFrameReader when using Iceberg as a > format > Hive now supports INSERT INTO, case insensitive query, projection pushdown, > create DDL with schema and auto type conversion > ORC now supports reading tinyint, smallint, char, varchar types > Hadoop catalog now supports role-based access of table listing > Please download, verify, and test. > > Please vote in the next 72 hours. > > [ ] +1 Release this as Apache Iceberg 0.11.0 > [ ] +0 > [ ] -1 Do not release this because... > > > -- > Ryan Blue > Software Engineer > Netflix