+1 (binding) I think the issue that was brought up by Dongjoon is valid and we should document the current caching behavior. The problem is also more generic and does not apply only to views as operations that are happening through the source directly may not propagated to the catalog.
I think it is worth exploring SupportsCatalogOptions but that would be a substantial change. I’m in favor of releasing this RC so that people can start testing Iceberg with Spark 3. - Anton > On 13 Jul 2020, at 11:55, Ryan Blue <rb...@netflix.com.INVALID> wrote: > > One more thing: a work-around is to redefine the view. That discards the > original logical plan and table and returns the expected result in Spark 3. > > On Mon, Jul 13, 2020 at 11:53 AM Ryan Blue <rb...@netflix.com > <mailto:rb...@netflix.com>> wrote: > Dongjoon, > > Thanks for raising this issue. I did some digging and the problem is that in > Spark 3.0, the logical plans saves a Table instance with the current state > when it was loaded -- when the `createOrReplaceTempView` call happened. That > never gets refreshed, which is why you get stale data. In 2.4, there is no > "table" in Spark so it gets reloaded every time. > > In the SQL path, this problem is avoided because the same table instance is > returned to Spark when the table is loaded from the catalog because of table > caching. The write will use the same table instance and keep it up to date, > and a call to `REFRESH TABLE` would also update it. > > I'm not sure the right way to avoid this behavior. We purposely don't refresh > the table each time Spark plans a scan because we want the version of a table > that is used to be consistent. A query that performs a self-join, for > example, should always use the same version of the table. On the other hand, > when a table reference is held in a logical plan like this, there is no > mechanism to update it. Maybe each table instance should keep a timer and > refresh after some interval? > > Either way, I agree that this is something we should add to documentation. I > don't think that this should fail the release, but we should fix it by the > next one. Does that sound reasonable to you? > > On Sun, Jul 12, 2020 at 6:16 PM Dongjoon Hyun <dongjoon.h...@gmail.com > <mailto:dongjoon.h...@gmail.com>> wrote: > I verified the hash/sign/build/UT and manual testing with Apache Spark 2.4.6 > (hadoop-2.7) and 3.0.0 (hadoop-3.2) on Apache Hive 2.3.7 metastore. > (BTW, for spark-3.0.0-bin-hadoop3.2, I used Ryan's example command with > `spark.sql.warehouse.dir` instead of `spark.warehouse.path`) > > 1. Iceberg 0.9 + Spark 2.4.6 works as expected. > 2. Iceberg 0.9 + Spark 3.0.0 works as expected mostly, but views work > differently from Iceberg 0.9 + Spark 2.4.6. > > The following is the example. > > ... > scala> > spark.read.format("iceberg").load("/tmp/t1").createOrReplaceTempView("t2") > > scala> sql("select * from t2").show > +---+---+ > | a| b| > +---+---+ > +---+---+ > > scala> Seq(("a", 1), ("b", 2)).toDF("a", > "b").write.format("iceberg").mode("overwrite").save("/tmp/t1") > > scala> sql("select * from t2").show // Iceberg 0.9 with Spark 2.4.6 shows the > updated result correctly here. > +---+---+ > | a| b| > +---+---+ > +---+---+ > > scala> spark.read.format("iceberg").load("/tmp/t1").show > +---+---+ > | a| b| > +---+---+ > | a| 1| > | b| 2| > +---+---+ > > So far, I'm not sure which part (Iceberg or Spark) introduces this difference, > but it would be nice if we had some document about the difference between > versions if this is designed like this. > > Thanks, > Dongjoon. > > On Sun, Jul 12, 2020 at 12:32 PM Daniel Weeks <dwe...@apache.org > <mailto:dwe...@apache.org>> wrote: > +1 (binding) > > Verified sigs/sums/license/build/test > > I did have an issue with the test metastore for the spark3 tests on the first > run, but couldn't replicate it in subsequent tests. > > -Dan > > On Fri, Jul 10, 2020 at 10:42 AM Ryan Blue <rb...@netflix.com.invalid> wrote: > +1 (binding) > > Verified checksums, ran tests, staged convenience binaries. > > I also ran a few tests using Spark 3.0.0 and Spark 2.4.5 and the runtime > Jars. For anyone that would like to use spark-sql or spark-shell, here are > the commands that I used: > > ~/Apps/spark-3.0.0-bin-hadoop2.7/bin/spark-sql \ > --repositories > https://repository.apache.org/content/repositories/orgapacheiceberg-1008 > <https://repository.apache.org/content/repositories/orgapacheiceberg-1008> \ > --packages org.apache.iceberg:iceberg-spark3-runtime:0.9.0 \ > --conf spark.warehouse.path=$PWD/spark-warehouse \ > --conf spark.hadoop.hive.metastore.uris=thrift://localhost:42745 \ > --conf > spark.sql.catalog.spark_catalog=org.apache.iceberg.spark.SparkSessionCatalog \ > --conf spark.sql.catalog.spark_catalog.type=hive \ > --conf spark.sql.catalog.hive_prod=org.apache.iceberg.spark.SparkCatalog \ > --conf spark.sql.catalog.hive_prod.type=hive \ > --conf > spark.sql.catalog.hadoop_prod=org.apache.iceberg.spark.SparkCatalog \ > --conf spark.sql.catalog.hadoop_prod.type=hadoop \ > --conf spark.sql.catalog.hadoop_prod.warehouse=$PWD/hadoop-warehouse > ~/Apps/spark-2.4.5-bin-hadoop2.7/bin/spark-shell \ > --repositories > https://repository.apache.org/content/repositories/orgapacheiceberg-1008 > <https://repository.apache.org/content/repositories/orgapacheiceberg-1008> \ > --packages org.apache.iceberg:iceberg-spark-runtime:0.9.0 \ > --conf spark.hadoop.hive.metastore.uris=thrift://localhost:42745 \ > --conf spark.warehouse.path=$PWD/spark-warehouse > The Spark 3 command sets up 3 catalogs: > > A wrapper around the built-in catalog, spark_catalog, that adds support for > Iceberg tables > A hive_prod Iceberg catalog that uses the same metastore as the session > catalog > A hadoop_prod Iceberg catalog that stores tables in a hadoop-warehouse folder > Everything worked great, except for a minor issue with CTAS, #1194 > <https://github.com/apache/iceberg/pull/1194>. I’m okay to release with that > issue, but we can always build a new RC if anyone thinks it is a blocker. > > If you’d like to run tests in a downstream project, you can use the staged > binary artifacts by adding this to your gradle build: > > repositories { > maven { > name 'stagedIceberg' > url > 'https://repository.apache.org/content/repositories/orgapacheiceberg-1008/ > <https://repository.apache.org/content/repositories/orgapacheiceberg-1008/>' > } > } > > ext { > icebergVersion = '0.9.0' > } > > On Fri, Jul 10, 2020 at 9:20 AM Ryan Murray <rym...@dremio.com > <mailto:rym...@dremio.com>> wrote: > 1. Verify the signature: OK > 2. Verify the checksum: OK > 3. Untar the archive tarball: OK > 4. Run RAT checks to validate license headers: RAT checks passed > 5. Build and test the project: all unit tests passed. > > +1 (non-binding) > > I did see that my build took >12 minutes and used all 100% of all 8 cores & > 32GB of memory (openjdk-8 ubuntu 18.04) which I haven't noticed before. > On Fri, Jul 10, 2020 at 4:37 AM OpenInx <open...@gmail.com > <mailto:open...@gmail.com>> wrote: > I followed the verify guide here > (https://lists.apache.org/thread.html/rd5e6b1656ac80252a9a7d473b36b6227da91d07d86d4ba4bee10df66%40%3Cdev.iceberg.apache.org%3E > > <https://lists.apache.org/thread.html/rd5e6b1656ac80252a9a7d473b36b6227da91d07d86d4ba4bee10df66%40%3Cdev.iceberg.apache.org%3E>) > : > > 1. Verify the signature: OK > 2. Verify the checksum: OK > 3. Untar the archive tarball: OK > 4. Run RAT checks to validate license headers: RAT checks passed > 5. Build and test the project: all unit tests passed. > > +1 (non-binding). > > On Fri, Jul 10, 2020 at 9:46 AM Ryan Blue <rb...@netflix.com.invalid> wrote: > Hi everyone, > > I propose the following RC to be released as the official Apache Iceberg > 0.9.0 release. > > The commit id is 4e66b4c10603e762129bc398146e02d21689e6dd > * This corresponds to the tag: apache-iceberg-0.9.0-rc5 > * https://github.com/apache/iceberg/commits/apache-iceberg-0.9.0-rc5 > <https://github.com/apache/iceberg/commits/apache-iceberg-0.9.0-rc5> > * https://github.com/apache/iceberg/tree/4e66b4c1 > <https://github.com/apache/iceberg/tree/4e66b4c1> > > The release tarball, signature, and checksums are here: > * https://dist.apache.org/repos/dist/dev/iceberg/apache-iceberg-0.9.0-rc5/ > <https://dist.apache.org/repos/dist/dev/iceberg/apache-iceberg-0.9.0-rc5/> > > 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-1008/ > <https://repository.apache.org/content/repositories/orgapacheiceberg-1008/> > > This release includes support for Spark 3 and vectorized reads for flat > schemas in Spark. > > Please download, verify, and test. > > Please vote in the next 72 hours. > > [ ] +1 Release this as Apache Iceberg 0.9.0 > [ ] +0 > [ ] -1 Do not release this because... > > -- > Ryan Blue > Software Engineer > Netflix > > > -- > Ryan Blue > Software Engineer > Netflix > > > -- > Ryan Blue > Software Engineer > Netflix > > > -- > Ryan Blue > Software Engineer > Netflix