think to use MinIO you need to use a custom client factory to set
> your S3 endpoint as that MinIO endpoint.
>
> -Jack
>
> On Tue, Aug 17, 2021 at 11:36 AM Lian Jiang wrote:
>
>> Hi Ryan,
>>
>> S3FileIO need canned ACL according to:
>>
>> /**
>&g
gt;
> Ryan
>
> On Mon, Aug 16, 2021 at 7:57 PM Jack Ye wrote:
>
>> Talked with Lian on Slack, the user is using a hadoop 3.2.1 + hive
>> (postgres) + spark + minio docker installation. There might be some S3A
>> related dependencies missing on the Hive server side base
gt; https://mvnrepository.com/artifact/org.apache.hadoop/hadoop-aws
> -Jack
>
> On Mon, Aug 16, 2021 at 7:09 PM Lian Jiang wrote:
>
>> Jack,
>>
>> You are right. S3FileIO will not work on minio since minio does not
>> support ACL: https://docs.min.io/docs/minio-se
gt; --conf
> spark.sql.catalog.hive_test.io-impl=org.apache.iceberg.aws.s3.S3FileIO \
> --conf spark.sql.catalog.hive_test.warehouse=s3://bucket
>
> Best,
> Jack Ye
>
>
>
> On Sun, Aug 15, 2021 at 2:53 PM Lian Jiang wrote:
>
>> Thanks. I prefer S3FileIO
>
> -Dan
>
> On Fri, Aug 13, 2021, 4:48 PM Lian Jiang wrote:
>
>> Thanks Daniel.
>>
>> After modifying the script to,
>>
>> export AWS_REGION=us-east-1
>> export AWS_ACCESS_KEY_ID=minio
>> export AWS_SECRET_ACCESS_KEY=minio123
>>
&
t for the S3AFileSystem (which would
> not be used with S3FileIO).
>
> You might try just removing that line since it should use the HadoopFileIO
> at that point and may work.
>
> Hope that's helpful,
> -Dan
>
> On Fri, Aug 13, 2021 at 3:50 PM Lian Jiang wrote:
&
Hi,
I try to create an iceberg table on minio s3 and hive.
*This is how I launch spark-shell:*
# add Iceberg dependency
export AWS_REGION=us-east-1
export AWS_ACCESS_KEY_ID=minio
export AWS_SECRET_ACCESS_KEY=minio123
ICEBERG_VERSION=0.11.1
DEPENDENCIES="org.apache.iceberg:iceberg-spark3-runtime
tables,
> not HDFS tables by running `SET iceberg.mr.catalog=hive`.
>
> On Wed, Aug 11, 2021 at 3:51 PM Lian Jiang wrote:
>
>> hive> describe formatted mytable3;
>> OK
>> # col_name data_type comment
>> value
n you run `DESCRIBE FORMATTED` for the table? Then we can see if there
> is a storage handler set up for it.
>
> On Wed, Aug 11, 2021 at 1:46 PM Lian Jiang wrote:
>
>> Thanks guys. tableProperty("location", ...) works.
>>
>> I have trouble making hive query an iceberg
uild possible of Spark. There is a bug in 3.0 of Spark which
>> ignores options passed to the V2 api sometimes,
>> https://issues.apache.org/jira/browse/SPARK-32592 . Which is fixed in 3.1
>>
>> On Aug 11, 2021, at 11:00 AM, Lian Jiang wrote:
>>
>> A
Any help is highly appreciated!
On Tue, Aug 10, 2021 at 11:06 AM Lian Jiang wrote:
> Thanks Russell.
>
> I tried:
>
> /spark/bin/spark-shell --packages
> org.apache.iceberg:iceberg-hive-runtime:0.11.1,org.apache.iceberg:iceberg-spark3-runtime:0.11.1
> --conf spark.
quot;location" when creating the table. Just add a
> ".option("location", "path")"
>
> On Aug 10, 2021, at 11:15 AM, Lian Jiang wrote:
>
> Thanks Russell. This helps a lot.
>
> I want to specify a HDFS location when creating an iceberg datase
ot;ts"))
.createOrReplace()
On Mon, Aug 9, 2021 at 4:22 PM Russell Spitzer
wrote:
> 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 "C
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 wrote:
>
>> Thanks Ed
ote:
>
>> 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 w
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 d
Hi,
I want to dump events in a kafka topic into datalake as a type 1 snapshot
in iceberg. Type 1 means a record having a key will overwrite the previous
record having the same key. Each key will have only one record in the
snapshot.
Note that I want to simplify the long path:
kafka -> (streaming
Hi,
I am new to Iceberg. I just built master branch and ran the jars on EMR
cluster spark-shell 2.3.0.
spark-shell --jars
gradle-wrapper.jar,iceberg-api-66fa048.jar,iceberg-common-66fa048.jar,iceberg-core-66fa048.jar,iceberg-data-66fa048.jar,iceberg-hive-66fa048.jar,iceberg-hive-66fa048-tests.
18 matches
Mail list logo