The config you used specified a catalog named "hive_prod", so to reference it you need to either "use hive_prod" or refer to the table with the catalog identifier "CREATE TABLE hive_prod.default.mytable"
On Mon, Aug 9, 2021 at 6:15 PM Lian Jiang <jiangok2...@gmail.com> wrote: > Thanks Ryan. > > Using this command (uri is omitted because the uri is in hive-site.xml): > spark-shell --conf > spark.sql.catalog.hive_prod=org.apache.iceberg.spark.SparkCatalog --conf > spark.sql.catalog.hive_prod.type=hive > > This statement: > spark.sql("CREATE TABLE default.mytable (uuid string) USING iceberg") > > caused warning: > WARN HiveExternalCatalog: Couldn't find corresponding Hive SerDe for data > source provider iceberg. > > I tried: > * the solution (put iceberg-hive-runtime.jar and > iceberg-spark3-runtime.jar to spark/jars) mentioned in > https://github.com/apache/iceberg/issues/2260 > * use --packages > org.apache.iceberg:iceberg-hive-runtime:0.11.1,org.apache.iceberg:iceberg-spark3-runtime:0.11.1 > > but they did not help. This warning blocks inserting any data into this > table. Any ideas are appreciated! > > On Mon, Aug 9, 2021 at 10:15 AM Ryan Blue <b...@tabular.io> wrote: > >> Lian, >> >> I think we should improve the docs for catalogs since it isn’t clear. We >> have a few configuration pages that are helpful, but it looks like they >> assume you know what your options are already. Take a look at the Spark >> docs for catalogs, which is the closest we have right now: >> https://iceberg.apache.org/spark-configuration/#catalog-configuration >> >> What you’ll want to do is to configure a catalog like the first example: >> >> spark.sql.catalog.hive_prod = org.apache.iceberg.spark.SparkCatalog >> spark.sql.catalog.hive_prod.type = hive >> spark.sql.catalog.hive_prod.uri = thrift://metastore-host:port >> # omit uri to use the same URI as Spark: hive.metastore.uris in hive-site.xml >> >> For MERGE INTO, the DataFrame API is not present in Spark, which is why >> it can’t be used by SQL. This is something that should probably be added to >> Spark and not Iceberg since it is just a different way to build the same >> underlying Spark plan. >> >> To your question about dataframes vs SQL, I highly recommend SQL over >> DataFrames so that you don’t end up needing to use Jars produced by >> compiling Scala code. I think it’s easier to just use SQL. But Iceberg >> should support both because DataFrames are useful for customization in some >> cases. It really should be up to you and what you want to use, as far as >> Iceberg is concerned. >> >> Ryan >> >> On Mon, Aug 9, 2021 at 9:31 AM Lian Jiang <jiangok2...@gmail.com> wrote: >> >>> Thanks Eduard and Ryan. >>> >>> I use spark on a K8S cluster to write parquet on s3 and then add an >>> external table in hive metastore for this parquet. In the future, when >>> using iceberg, I prefer hive metadata store since it is my >>> centralized metastore for batch and streaming datasets. I don't see that >>> hive metastore is supported in iceberg AWS integration on >>> https://iceberg.apache.org/aws/. Is there another link for that? >>> >>> Most of the examples use spark sql to write/read iceberg. For example, >>> there is no "sql merge into" like support for spark API. Is spark sql >>> preferred over spark dataframe/dataset API in Iceberg? If so, could you >>> clarify the rationale behind? I personally feel spark API is more dev >>> friendly and scalable. Thanks very much! >>> >>> >>> On Mon, Aug 9, 2021 at 8:53 AM Ryan Blue <b...@tabular.io> wrote: >>> >>>> Lian, >>>> >>>> Iceberg tables work great in S3. When creating the table, just pass the >>>> `LOCATION` clause with an S3 path, or set your catalog's warehouse location >>>> to S3 so tables are automatically created there. >>>> >>>> The only restriction for S3 is that you need a metastore to track the >>>> table metadata location because S3 doesn't have a way to implement a >>>> metadata commit. For a metastore, there are implementations backed by the >>>> Hive MetaStore, Glue/DynamoDB, and Nessie. And the upcoming release adds >>>> support for DynamoDB without Glue and JDBC. >>>> >>>> Ryan >>>> >>>> On Mon, Aug 9, 2021 at 2:24 AM Eduard Tudenhoefner <edu...@dremio.com> >>>> wrote: >>>> >>>>> Lian you can have a look at https://iceberg.apache.org/aws/. It >>>>> should contain all the info that you need. The codebase contains a >>>>> *S3FileIO >>>>> *class, which is an implementation that is backed by S3. >>>>> >>>>> On Mon, Aug 9, 2021 at 7:37 AM Lian Jiang <jiangok2...@gmail.com> >>>>> wrote: >>>>> >>>>>> I am reading https://iceberg.apache.org/spark-writes/#spark-writes >>>>>> and wondering if it is possible to create an iceberg table on S3. This >>>>>> guide seems to say only write to a hive table (backed up by HDFS if I >>>>>> understand correctly). Hudi and Delta can write to s3 with a specified S3 >>>>>> path. How can I do it using iceberg? Thanks for any clarification. >>>>>> >>>>>> >>>>>> >>>> >>>> -- >>>> Ryan Blue >>>> Tabular >>>> >>> >>> >>> -- >>> >>> Create your own email signature >>> <https://www.wisestamp.com/signature-in-email/?utm_source=promotion&utm_medium=signature&utm_campaign=create_your_own&srcid=5234462839406592> >>> >> >> >> -- >> Ryan Blue >> Tabular >> > > > -- > > Create your own email signature > <https://www.wisestamp.com/signature-in-email/?utm_source=promotion&utm_medium=signature&utm_campaign=create_your_own&srcid=5234462839406592> >