You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/06/08 02:41:28 UTC

[GitHub] [iceberg] wang-x-xia opened a new pull request, #4993: Dell: Add document.

wang-x-xia opened a new pull request, #4993:
URL: https://github.com/apache/iceberg/pull/4993

   The PR has 3 parts:
   1. Add a document of ECS how-to.
   2. Fix the bugs when I test the catalog in production.
   3. Add the `:iceberg:dell` module to the runtime package.
   


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r933581592


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,132 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integration
+
+## Dell ECS Integration
+
+Iceberg can be used with Dell's Enterprise Object Storage (ECS) by using the ECS catalog since 0.14.0.
+
+Dell ECS has many features that make Iceberg a highly compatible table format, such as append operations for file writers and content addressable storage (CAS) for table commits. See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When using Dell ECS with Iceberg, these configuration parameters are required:
+
+| Name                     | Description             |
+| ------------------------ | ----------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | ECS Username            |
+| ecs.s3.secret-access-key | S3 Secret Key           |
+
+An ECS catalog requires that you configure a warehouse location where all data and metadata will be created.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last `/` will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Runtime dependencies
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell EMC still recommends using the latest version of the client.
+
+### Spark
+
+For example, to use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```python
+from pyspark.sql import SparkSession
+
+spark = SparkSession.builder
+    .config("spark.jars.packages", "org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2")
+    .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")
+    .config("spark.sql.catalog.my_catalog", "org.apache.iceberg.spark.SparkCatalog")
+    .config("spark.sql.catalog.my_catalog.warehouse", "ecs://bucket-a/namespace-a")
+    .config("spark.sql.catalog.my_catalog.catalog-impl", "org.apache.iceberg.dell.ecs.EcsCatalog")
+    .config("spark.sql.catalog.my_catalog.ecs.s3.endpoint", "http://10.x.x.x:9020")
+    .config("spark.sql.catalog.my_catalog.ecs.s3.access-key-id", "<Your-ecs-s3-access-key>")
+    .config("spark.sql.catalog.my_catalog.ecs.s3.secret-access-key", "<Your-ecs-s3-secret-access-key>")
+    .getOrCreate()
+```
+
+Then, use `my_catalog` to access the data in ECS. You can use `SHOW NAMESPACES IN my_catalog` and `SHOW TABLES IN my_catalog` to fetch the namespaces and tables of the catalog.
+
+The related problems of catalog usage:
+
+1. The [pyspark.sql.SparkSession.catalog](https://spark.apache.org/docs/latest/api/python/reference/api/pyspark.sql.SparkSession.catalog.html#pyspark.sql.SparkSession.catalog) won't access the 3rd-party catalog of Spark, so please use DDL SQL to list all tables and namespaces.
+
+
+### Flink
+
+Use the Dell ECS catalog with Flink, you first must create a Flink environment.
+
+```python
+import requests
+from pyflink.datastream import StreamExecutionEnvironment
+from pyflink.table import StreamTableEnvironment
+
+# Set your workspace
+work_space = "<your_work_space>"
+
+jars = {
+    "iceberg-flink-runtime-1.14-0.14.0.jar" : "https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-flink-runtime-1.14/0.14.0/iceberg-flink-runtime-1.14-0.14.0.jar", 
+    "object-client-bundle-3.3.2.jar" : "https://repo1.maven.org/maven2/com/emc/ecs/object-client-bundle/3.3.2/object-client-bundle-3.3.2.jar",
+}
+
+# Download libraries
+for jar, link in jars.items():
+    with open(f"{work_space}/{jar}", "wb") as f:
+      f.write(requests.get(link).content)
+
+pipeline_jars = [f"file://{work_space}/{jar}" for jar in jars]
+
+# Setup Flink session
+env = StreamExecutionEnvironment.get_execution_environment()
+env.add_jars(*pipeline_jars)
+t_env = StreamTableEnvironment.create(env)
+```
+
+Then, use Flink SQL to create a catalog named `my_catalog`:
+
+```SQL
+CREATE CATALOG my_catalog WITH (
+    'type'='iceberg',
+    'warehouse' = 'ecs://bucket-a/namespace-a',
+    'catalog-impl'='org.apache.iceberg.dell.ecs.EcsCatalog',
+    'ecs.s3.endpoint' = 'http://10.x.x.x:9020',
+    'ecs.s3.access-key-id' = '<Your-ecs-s3-access-key>',
+    'ecs.s3.secret-access-key' = '<Your-ecs-s3-secret-access-key>')
+```
+
+Then, you can run `USE CATALOG my_catalog`, `SHOW DATABASES`, and `SHOW TABLES` to fetch the namespaces and tables of the catalog.
+
+### Limitations
+
+When you use the catalog with Dell ECS only, you should care about these limitations:
+
+1. `RENAME` statements are supported without other protections. When you try to rename a table, you need to guarantee all commits are finished in the original table.
+2. `RENAME` statements only rename the table without moving any data files. This can lead to a table's data being stored in a path outside of the configured warehouse path.

Review Comment:
   I don't think that table rename is supported. Wouldn't that break all metadata pointers?



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] wang-x-xia commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
wang-x-xia commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r896419349


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:
+
+| Name                     | Description            |
+| ------------------------ | ---------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | Username               |
+| ecs.s3.secret-access-key | S3 Secret Key          |
+
+As for the catalog, you should provide a warehouse location where will store all data and metadata later.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last / will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Dependencies of runtime
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell ECS still recommends using the latest version of the client.
+
+### Spark
+
+For example, to use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=ecs://bucket-a/namespace-a \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.dell.ecs.EcsCatalog \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.endpoint=http://10.x.x.x:9020 \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.access-key-id=test \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.secret-access-key=xxxxxxxxxxxxxxxx
+```
+
+```python
+from pyspark.sql import SparkSession
+
+spark = SparkSession.builder\
+    .config("spark.jars.packages", "org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2")\
+    .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")\
+    .config("spark.sql.catalog.my_catalog", "org.apache.iceberg.spark.SparkCatalog")\
+    .config("spark.sql.catalog.my_catalog.warehouse", "ecs://bucket-a/namespace-a")\
+    .config("spark.sql.catalog.my_catalog.catalog-impl", "org.apache.iceberg.dell.ecs.EcsCatalog")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.endpoint", "http://10.x.x.x:9020")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.access-key-id", "test")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.secret-access-key", "xxxxxxxxxxxxxxxx")\
+    .getOrCreate()
+```
+
+Then, use `my_catalog` to access the data in ECS. You can use `SHOW NAMESPACES IN my_catalog` and `SHOW TABLES IN my_catalog` to fetch the namespaces and tables of the catalog.
+
+The related problems of catalog usage:
+
+1. The [pyspark.sql.SparkSession.catalog](https://spark.apache.org/docs/latest/api/python/reference/api/pyspark.sql.SparkSession.catalog.html#pyspark.sql.SparkSession.catalog) won't access the 3rd-party catalog of Spark, so please use DDL SQL to list all tables and namespaces.
+
+
+### Flink
+
+For example, to use the Dell ECS catalog with Flink 1.14, you should create a Flink environment like:
+
+```sh
+wget https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-flink-runtime-1.14/0.14.0/iceberg-flink-runtime-1.14-0.14.0.jar
+wget https://repo1.maven.org/maven2/com/emc/ecs/object-client-bundle/3.3.2/object-client-bundle-3.3.2.jar
+
+sql-client.sh embedded -j iceberg-flink-runtime-1.14-0.14.0.jar -j object-client-bundle-3.3.2.jar shell
+```
+
+```python
+import requests
+from pyflink.datastream import StreamExecutionEnvironment
+from pyflink.table import StreamTableEnvironment
+
+# Set your workspace
+work_space = "/"
+
+# Download libraries
+maven_url="https://repo1.maven.org/maven2"
+with open(f"{work_space}/iceberg-flink-runtime-1.14-0.14.0.jar", "wb") as f:
+  f.write(requests.get(f"{maven_url}/org/apache/iceberg/iceberg-flink-runtime-1.14/0.14.0/iceberg-flink-runtime-1.14-0.14.0.jar").content)
+with open(f"{work_space}/object-client-bundle-3.3.2.jar", "wb") as f:
+  f.write(requests.get(f"{maven_url}/com/emc/ecs/object-client-bundle/3.3.2/object-client-bundle-3.3.2.jar").content)
+
+jars = ["iceberg-flink-runtime-1.14-0.14.0.jar", "object-client-bundle-3.3.2.jar"]
+pipeline_jars = [f"file://{work_space}/{jar}" for jar in jars]
+
+# Setup Flink session
+env = StreamExecutionEnvironment.get_execution_environment()
+env.add_jars(*pipeline_jars)
+t_env = StreamTableEnvironment.create(env)
+```
+
+Then, use Flink SQL to create a catalog named `my_catalog`:
+
+```SQL
+CREATE CATALOG my_catalog WITH (
+    'type'='iceberg',
+    'warehouse' = 'ecs://bucket-a/namespace-a',
+    'catalog-impl'='org.apache.iceberg.dell.ecs.EcsCatalog',
+    'ecs.s3.endpoint' = 'http://10.x.x.x:9020',
+    'ecs.s3.access-key-id' = 'test',
+    'ecs.s3.secret-access-key' = 'xxxxxxxxxxxxxxxx')
+```
+
+Then, `USE CATALOG my_catalog`, `SHOW DATABASES`, and `SHOW TABLES` to fetch the namespaces and tables of the catalog.
+
+### Limitations
+
+When you use the catalog with Dell ECS only, you should care about these limitations:
+
+1. The rename operation is supported without other protections. When you try to rename a table, you need to guarantee all commits are finished in the original table.
+2. The rename operation only renames the table without moving any data file. The renamed table maybe store data objects in different paths even not in the warehouse that is configured in the catalog.

Review Comment:
   Done.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] samredai commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
samredai commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r895354472


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:

Review Comment:
   There may be a better word than session to use here. I see that you mean a sort of reading session but I think that word is used in a lot of other places to mean specific things (like a Spark session). Maybe just say "Iceberg can be used with Dell's Enterprise Object Storage (ECS) by using the ECS catalog."



##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:
+
+| Name                     | Description            |
+| ------------------------ | ---------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | Username               |
+| ecs.s3.secret-access-key | S3 Secret Key          |
+
+As for the catalog, you should provide a warehouse location where will store all data and metadata later.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last / will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Dependencies of runtime
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell ECS still recommends using the latest version of the client.
+
+### Spark
+
+For example, to use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2 \

Review Comment:
   The different mechanisms to set the spark configuration feels like material more suited for a quickstart. Since the properties are listed in the `Connection parameters` section above--here I think it just needs to describe the added prefix to set these in Spark (`ecs.s3.endpoint` is set as `spark.sql.catalog.<catalog-name>.ecs.s3.endpoint`)
   
   In a quickstart, we can include this in a tabbed box with a spark-sql, spark-shell, and pyspark example to give something easy to copy & paste.



##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:
+
+| Name                     | Description            |
+| ------------------------ | ---------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | Username               |
+| ecs.s3.secret-access-key | S3 Secret Key          |
+
+As for the catalog, you should provide a warehouse location where will store all data and metadata later.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last / will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.

Review Comment:
   nit: Maybe add backticks around the `/`?



##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:
+
+| Name                     | Description            |
+| ------------------------ | ---------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | Username               |
+| ecs.s3.secret-access-key | S3 Secret Key          |
+
+As for the catalog, you should provide a warehouse location where will store all data and metadata later.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last / will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Dependencies of runtime
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell ECS still recommends using the latest version of the client.
+
+### Spark
+
+For example, to use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=ecs://bucket-a/namespace-a \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.dell.ecs.EcsCatalog \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.endpoint=http://10.x.x.x:9020 \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.access-key-id=test \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.secret-access-key=xxxxxxxxxxxxxxxx
+```
+
+```python
+from pyspark.sql import SparkSession
+
+spark = SparkSession.builder\
+    .config("spark.jars.packages", "org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2")\
+    .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")\
+    .config("spark.sql.catalog.my_catalog", "org.apache.iceberg.spark.SparkCatalog")\
+    .config("spark.sql.catalog.my_catalog.warehouse", "ecs://bucket-a/namespace-a")\
+    .config("spark.sql.catalog.my_catalog.catalog-impl", "org.apache.iceberg.dell.ecs.EcsCatalog")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.endpoint", "http://10.x.x.x:9020")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.access-key-id", "test")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.secret-access-key", "xxxxxxxxxxxxxxxx")\
+    .getOrCreate()
+```
+
+Then, use `my_catalog` to access the data in ECS. You can use `SHOW NAMESPACES IN my_catalog` and `SHOW TABLES IN my_catalog` to fetch the namespaces and tables of the catalog.
+
+The related problems of catalog usage:
+
+1. The [pyspark.sql.SparkSession.catalog](https://spark.apache.org/docs/latest/api/python/reference/api/pyspark.sql.SparkSession.catalog.html#pyspark.sql.SparkSession.catalog) won't access the 3rd-party catalog of Spark, so please use DDL SQL to list all tables and namespaces.
+
+
+### Flink
+
+For example, to use the Dell ECS catalog with Flink 1.14, you should create a Flink environment like:
+
+```sh
+wget https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-flink-runtime-1.14/0.14.0/iceberg-flink-runtime-1.14-0.14.0.jar
+wget https://repo1.maven.org/maven2/com/emc/ecs/object-client-bundle/3.3.2/object-client-bundle-3.3.2.jar
+
+sql-client.sh embedded -j iceberg-flink-runtime-1.14-0.14.0.jar -j object-client-bundle-3.3.2.jar shell
+```
+
+```python
+import requests
+from pyflink.datastream import StreamExecutionEnvironment
+from pyflink.table import StreamTableEnvironment
+
+# Set your workspace
+work_space = "/"
+
+# Download libraries
+maven_url="https://repo1.maven.org/maven2"
+with open(f"{work_space}/iceberg-flink-runtime-1.14-0.14.0.jar", "wb") as f:
+  f.write(requests.get(f"{maven_url}/org/apache/iceberg/iceberg-flink-runtime-1.14/0.14.0/iceberg-flink-runtime-1.14-0.14.0.jar").content)
+with open(f"{work_space}/object-client-bundle-3.3.2.jar", "wb") as f:
+  f.write(requests.get(f"{maven_url}/com/emc/ecs/object-client-bundle/3.3.2/object-client-bundle-3.3.2.jar").content)
+
+jars = ["iceberg-flink-runtime-1.14-0.14.0.jar", "object-client-bundle-3.3.2.jar"]
+pipeline_jars = [f"file://{work_space}/{jar}" for jar in jars]
+
+# Setup Flink session
+env = StreamExecutionEnvironment.get_execution_environment()
+env.add_jars(*pipeline_jars)
+t_env = StreamTableEnvironment.create(env)
+```
+
+Then, use Flink SQL to create a catalog named `my_catalog`:
+
+```SQL
+CREATE CATALOG my_catalog WITH (
+    'type'='iceberg',
+    'warehouse' = 'ecs://bucket-a/namespace-a',
+    'catalog-impl'='org.apache.iceberg.dell.ecs.EcsCatalog',
+    'ecs.s3.endpoint' = 'http://10.x.x.x:9020',
+    'ecs.s3.access-key-id' = 'test',
+    'ecs.s3.secret-access-key' = 'xxxxxxxxxxxxxxxx')
+```
+
+Then, `USE CATALOG my_catalog`, `SHOW DATABASES`, and `SHOW TABLES` to fetch the namespaces and tables of the catalog.
+
+### Limitations
+
+When you use the catalog with Dell ECS only, you should care about these limitations:
+
+1. The rename operation is supported without other protections. When you try to rename a table, you need to guarantee all commits are finished in the original table.
+2. The rename operation only renames the table without moving any data file. The renamed table maybe store data objects in different paths even not in the warehouse that is configured in the catalog.

Review Comment:
   This needs some rewording. How about:
   > `RENAME` statements only renames the table without moving any data files. This can lead to a table's data being stored in a path outside of the configured warehouse path.



##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations

Review Comment:
   This h2 title and the h1 title above it feels redundant. I would consolidate them into a single title such as `# Dell ECS` or `# Dell ECS Catalog`. Also I think the word "Integration" refers more to Iceberg's incorporation into compute/query engines such as Spark or Trino. I could be wrong but I think Dell ECS is more of an "Implementation".



##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:
+
+| Name                     | Description            |
+| ------------------------ | ---------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | Username               |

Review Comment:
   nit: Replace "Username" with "ECS Username"



##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.

Review Comment:
   The line above mentions "several features" but then only lists 2. Does it make more sense to provide a more comprehensive list of features or to reword the above sentence to say something like:
   
   > Dell ECS has many features that make Iceberg a highly compatible table format, such as append operations for file writers and content addressable storage (CAS) for table commits.
   
   ...then you can exclude this numbered list. What do you think?



##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:
+
+| Name                     | Description            |
+| ------------------------ | ---------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | Username               |
+| ecs.s3.secret-access-key | S3 Secret Key          |
+
+As for the catalog, you should provide a warehouse location where will store all data and metadata later.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last / will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Dependencies of runtime

Review Comment:
   "Runtime Dependencies" seems like the more common way this is described.



##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:
+
+| Name                     | Description            |
+| ------------------------ | ---------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | Username               |
+| ecs.s3.secret-access-key | S3 Secret Key          |
+
+As for the catalog, you should provide a warehouse location where will store all data and metadata later.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last / will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Dependencies of runtime
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell ECS still recommends using the latest version of the client.
+
+### Spark
+
+For example, to use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=ecs://bucket-a/namespace-a \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.dell.ecs.EcsCatalog \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.endpoint=http://10.x.x.x:9020 \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.access-key-id=test \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.secret-access-key=xxxxxxxxxxxxxxxx
+```
+
+```python
+from pyspark.sql import SparkSession
+
+spark = SparkSession.builder\
+    .config("spark.jars.packages", "org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2")\
+    .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")\
+    .config("spark.sql.catalog.my_catalog", "org.apache.iceberg.spark.SparkCatalog")\
+    .config("spark.sql.catalog.my_catalog.warehouse", "ecs://bucket-a/namespace-a")\
+    .config("spark.sql.catalog.my_catalog.catalog-impl", "org.apache.iceberg.dell.ecs.EcsCatalog")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.endpoint", "http://10.x.x.x:9020")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.access-key-id", "test")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.secret-access-key", "xxxxxxxxxxxxxxxx")\
+    .getOrCreate()
+```
+
+Then, use `my_catalog` to access the data in ECS. You can use `SHOW NAMESPACES IN my_catalog` and `SHOW TABLES IN my_catalog` to fetch the namespaces and tables of the catalog.
+
+The related problems of catalog usage:
+
+1. The [pyspark.sql.SparkSession.catalog](https://spark.apache.org/docs/latest/api/python/reference/api/pyspark.sql.SparkSession.catalog.html#pyspark.sql.SparkSession.catalog) won't access the 3rd-party catalog of Spark, so please use DDL SQL to list all tables and namespaces.
+
+
+### Flink
+
+For example, to use the Dell ECS catalog with Flink 1.14, you should create a Flink environment like:

Review Comment:
   I don't think "For example,` is needed here, I would just jump right in and say "Using the Dell ECS catalog with Flink, you first must create a Flink environment."



##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:

Review Comment:
   I could be overthinking this but `try` doesn't feel confident enough. I would just say "When using Dell ECS with Iceberg, these configuration parameters are required."



##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:
+
+| Name                     | Description            |
+| ------------------------ | ---------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | Username               |
+| ecs.s3.secret-access-key | S3 Secret Key          |
+
+As for the catalog, you should provide a warehouse location where will store all data and metadata later.

Review Comment:
   This looks like a typo or grammar issue. Maybe change this to:
   
   > An ECS catalog requires that you configure a warehouse location where all data and metadata will be created.



##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:
+
+| Name                     | Description            |
+| ------------------------ | ---------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | Username               |
+| ecs.s3.secret-access-key | S3 Secret Key          |
+
+As for the catalog, you should provide a warehouse location where will store all data and metadata later.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last / will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Dependencies of runtime
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell ECS still recommends using the latest version of the client.
+
+### Spark
+
+For example, to use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=ecs://bucket-a/namespace-a \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.dell.ecs.EcsCatalog \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.endpoint=http://10.x.x.x:9020 \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.access-key-id=test \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.secret-access-key=xxxxxxxxxxxxxxxx
+```
+
+```python
+from pyspark.sql import SparkSession
+
+spark = SparkSession.builder\
+    .config("spark.jars.packages", "org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2")\
+    .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")\
+    .config("spark.sql.catalog.my_catalog", "org.apache.iceberg.spark.SparkCatalog")\
+    .config("spark.sql.catalog.my_catalog.warehouse", "ecs://bucket-a/namespace-a")\
+    .config("spark.sql.catalog.my_catalog.catalog-impl", "org.apache.iceberg.dell.ecs.EcsCatalog")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.endpoint", "http://10.x.x.x:9020")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.access-key-id", "test")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.secret-access-key", "xxxxxxxxxxxxxxxx")\
+    .getOrCreate()
+```
+
+Then, use `my_catalog` to access the data in ECS. You can use `SHOW NAMESPACES IN my_catalog` and `SHOW TABLES IN my_catalog` to fetch the namespaces and tables of the catalog.
+
+The related problems of catalog usage:
+
+1. The [pyspark.sql.SparkSession.catalog](https://spark.apache.org/docs/latest/api/python/reference/api/pyspark.sql.SparkSession.catalog.html#pyspark.sql.SparkSession.catalog) won't access the 3rd-party catalog of Spark, so please use DDL SQL to list all tables and namespaces.
+
+
+### Flink
+
+For example, to use the Dell ECS catalog with Flink 1.14, you should create a Flink environment like:
+
+```sh
+wget https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-flink-runtime-1.14/0.14.0/iceberg-flink-runtime-1.14-0.14.0.jar
+wget https://repo1.maven.org/maven2/com/emc/ecs/object-client-bundle/3.3.2/object-client-bundle-3.3.2.jar
+
+sql-client.sh embedded -j iceberg-flink-runtime-1.14-0.14.0.jar -j object-client-bundle-3.3.2.jar shell
+```
+
+```python
+import requests
+from pyflink.datastream import StreamExecutionEnvironment
+from pyflink.table import StreamTableEnvironment
+
+# Set your workspace
+work_space = "/"
+
+# Download libraries
+maven_url="https://repo1.maven.org/maven2"
+with open(f"{work_space}/iceberg-flink-runtime-1.14-0.14.0.jar", "wb") as f:
+  f.write(requests.get(f"{maven_url}/org/apache/iceberg/iceberg-flink-runtime-1.14/0.14.0/iceberg-flink-runtime-1.14-0.14.0.jar").content)
+with open(f"{work_space}/object-client-bundle-3.3.2.jar", "wb") as f:
+  f.write(requests.get(f"{maven_url}/com/emc/ecs/object-client-bundle/3.3.2/object-client-bundle-3.3.2.jar").content)
+
+jars = ["iceberg-flink-runtime-1.14-0.14.0.jar", "object-client-bundle-3.3.2.jar"]
+pipeline_jars = [f"file://{work_space}/{jar}" for jar in jars]
+
+# Setup Flink session
+env = StreamExecutionEnvironment.get_execution_environment()
+env.add_jars(*pipeline_jars)
+t_env = StreamTableEnvironment.create(env)
+```
+
+Then, use Flink SQL to create a catalog named `my_catalog`:
+
+```SQL
+CREATE CATALOG my_catalog WITH (
+    'type'='iceberg',
+    'warehouse' = 'ecs://bucket-a/namespace-a',
+    'catalog-impl'='org.apache.iceberg.dell.ecs.EcsCatalog',
+    'ecs.s3.endpoint' = 'http://10.x.x.x:9020',
+    'ecs.s3.access-key-id' = 'test',
+    'ecs.s3.secret-access-key' = 'xxxxxxxxxxxxxxxx')
+```
+
+Then, `USE CATALOG my_catalog`, `SHOW DATABASES`, and `SHOW TABLES` to fetch the namespaces and tables of the catalog.

Review Comment:
   Then, **you can run** `USE CATALOG my_catalog`, `SHOW DATABASES`, and `SHOW TABLES` to fetch the namespaces and tables in the catalog.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] wang-x-xia commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
wang-x-xia commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r896417903


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:
+
+| Name                     | Description            |
+| ------------------------ | ---------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | Username               |
+| ecs.s3.secret-access-key | S3 Secret Key          |
+
+As for the catalog, you should provide a warehouse location where will store all data and metadata later.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last / will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Dependencies of runtime
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell ECS still recommends using the latest version of the client.
+
+### Spark
+
+For example, to use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=ecs://bucket-a/namespace-a \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.dell.ecs.EcsCatalog \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.endpoint=http://10.x.x.x:9020 \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.access-key-id=test \

Review Comment:
   Done.



##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:
+
+| Name                     | Description            |
+| ------------------------ | ---------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | Username               |
+| ecs.s3.secret-access-key | S3 Secret Key          |
+
+As for the catalog, you should provide a warehouse location where will store all data and metadata later.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last / will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Dependencies of runtime
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell ECS still recommends using the latest version of the client.
+
+### Spark
+
+For example, to use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=ecs://bucket-a/namespace-a \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.dell.ecs.EcsCatalog \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.endpoint=http://10.x.x.x:9020 \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.access-key-id=test \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.secret-access-key=xxxxxxxxxxxxxxxx

Review Comment:
   Done.



##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations

Review Comment:
   Done.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] wang-x-xia commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
wang-x-xia commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r934129368


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,132 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integration
+
+## Dell ECS Integration
+
+Iceberg can be used with Dell's Enterprise Object Storage (ECS) by using the ECS catalog since 0.14.0.
+
+Dell ECS has many features that make Iceberg a highly compatible table format, such as append operations for file writers and content addressable storage (CAS) for table commits. See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When using Dell ECS with Iceberg, these configuration parameters are required:
+
+| Name                     | Description             |
+| ------------------------ | ----------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | ECS Username            |
+| ecs.s3.secret-access-key | S3 Secret Key           |
+
+An ECS catalog requires that you configure a warehouse location where all data and metadata will be created.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last `/` will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Runtime dependencies
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell EMC still recommends using the latest version of the client.
+
+### Spark
+
+For example, to use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```python
+from pyspark.sql import SparkSession
+
+spark = SparkSession.builder
+    .config("spark.jars.packages", "org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2")

Review Comment:
   Added.
   When I wrote this document, I assume all PRs will be merged. 
   But now no PR be merged into 0.14.0. So I also fix the version to 0.15.0.
   😂



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] wang-x-xia commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
wang-x-xia commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r964344893


##########
docs/dell.md:
##########
@@ -0,0 +1,135 @@
+---
+title: "Dell"
+url: dell
+menu:
+    main:
+        parent: Integrations
+        weight: 0
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integration
+
+## Dell ECS Integration
+
+Iceberg can be used with Dell's Enterprise Object Storage (ECS) by using the ECS catalog since 0.15.0.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Parameters
+
+When using Dell ECS with Iceberg, these configuration parameters are required:
+
+| Name                     | Description                       |
+| ------------------------ | --------------------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint           |
+| ecs.s3.access-key-id     | ECS Username                      |
+| ecs.s3.secret-access-key | S3 Secret Key                     |
+| warehouse                | The location of data and metadata |
+
+The warehouse should use the following formats:
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/            | Use the whole bucket as the data. The last `/` is ignored.      |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. You should pick the correct version.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell EMC still recommends using the latest version of the client.
+
+### Spark
+
+To use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```bash
+ICEBERG_VERSION=0.15.0
+SPARK_VERSION=3.2_2.12
+ECS_CLIENT_VERSION=3.3.2
+
+DEPENDENCIES="org.apache.iceberg:iceberg-spark-runtime-${SPARK_VERSION}:${ICEBERG_VERSION},\
+org.apache.iceberg:iceberg-dell:${ICEBERG_VERSION},\
+com.emc.ecs:object-client-bundle:${ECS_CLIENT_VERSION}"
+
+spark-sql --packages ${DEPENDENCIES} \
+    --conf spark.sql.extensions=org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=ecs://bucket-a/namespace-a \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.dell.ecs.EcsCatalog \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.endpoint=http://10.x.x.x:9020 \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.access-key-id=<Your-ecs-s3-access-key> \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.secret-access-key=<Your-ecs-s3-secret-access-key>
+```
+
+Then, use `my_catalog` to access the data in ECS. You can use `SHOW NAMESPACES IN my_catalog` and `SHOW TABLES IN my_catalog` to fetch the namespaces and tables of the catalog.
+
+The related problems of catalog usage:
+
+1. The `pyspark.sql.SparkSession.catalog` won't access the 3rd-party catalog of Spark, so please use DDL SQL to list all tables and namespaces.

Review Comment:
   All languages are the same. I have fixed this statement.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] wang-x-xia commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
wang-x-xia commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r896342176


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations

Review Comment:
   In previous PRs, the reviewers let me use `dell` as the module name.
   I also used `dell` as the document title and used the h2 title for ECS. Should I change this document to ECS only?
   
   And the "Integration" will change to "Implementation".



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r898256091


##########
dell/src/main/java/org/apache/iceberg/dell/DellClientFactory.java:
##########
@@ -20,9 +20,10 @@
 package org.apache.iceberg.dell;
 
 import com.emc.object.s3.S3Client;
+import java.io.Serializable;
 import java.util.Map;
 
-public interface DellClientFactory {
+public interface DellClientFactory extends Serializable {

Review Comment:
   Can you move code changes out of a documentation PR? These should be in a code PR for people that cherry-pick commits.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
rdblue commented on PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#issuecomment-1153321341

   @samredai, can you take a look?


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
openinx commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r895284023


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:
+
+| Name                     | Description            |
+| ------------------------ | ---------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | Username               |
+| ecs.s3.secret-access-key | S3 Secret Key          |
+
+As for the catalog, you should provide a warehouse location where will store all data and metadata later.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last / will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Dependencies of runtime
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell ECS still recommends using the latest version of the client.
+
+### Spark
+
+For example, to use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=ecs://bucket-a/namespace-a \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.dell.ecs.EcsCatalog \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.endpoint=http://10.x.x.x:9020 \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.access-key-id=test \

Review Comment:
   Nit:  `test` -> `<Your-ecs-s3-access-key>`.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] wang-x-xia commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
wang-x-xia commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r934128513


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,132 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integration
+
+## Dell ECS Integration
+
+Iceberg can be used with Dell's Enterprise Object Storage (ECS) by using the ECS catalog since 0.14.0.
+
+Dell ECS has many features that make Iceberg a highly compatible table format, such as append operations for file writers and content addressable storage (CAS) for table commits. See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.

Review Comment:
   Removed!



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
openinx commented on PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#issuecomment-1236492683

   Sorry for the later response, @wang-x-xia. I remember @wang-x-xia ping me in other IM tools, but I missed to track this document. so sorry.  Let me take a look and get this merged if it's ready.


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] wang-x-xia commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
wang-x-xia commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r964346292


##########
docs/dell.md:
##########
@@ -0,0 +1,135 @@
+---
+title: "Dell"
+url: dell
+menu:
+    main:
+        parent: Integrations
+        weight: 0
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integration
+
+## Dell ECS Integration
+
+Iceberg can be used with Dell's Enterprise Object Storage (ECS) by using the ECS catalog since 0.15.0.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Parameters
+
+When using Dell ECS with Iceberg, these configuration parameters are required:
+
+| Name                     | Description                       |
+| ------------------------ | --------------------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint           |
+| ecs.s3.access-key-id     | ECS Username                      |
+| ecs.s3.secret-access-key | S3 Secret Key                     |
+| warehouse                | The location of data and metadata |
+
+The warehouse should use the following formats:
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/            | Use the whole bucket as the data. The last `/` is ignored.      |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. You should pick the correct version.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell EMC still recommends using the latest version of the client.
+
+### Spark
+
+To use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```bash
+ICEBERG_VERSION=0.15.0
+SPARK_VERSION=3.2_2.12
+ECS_CLIENT_VERSION=3.3.2
+
+DEPENDENCIES="org.apache.iceberg:iceberg-spark-runtime-${SPARK_VERSION}:${ICEBERG_VERSION},\
+org.apache.iceberg:iceberg-dell:${ICEBERG_VERSION},\
+com.emc.ecs:object-client-bundle:${ECS_CLIENT_VERSION}"
+
+spark-sql --packages ${DEPENDENCIES} \
+    --conf spark.sql.extensions=org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=ecs://bucket-a/namespace-a \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.dell.ecs.EcsCatalog \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.endpoint=http://10.x.x.x:9020 \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.access-key-id=<Your-ecs-s3-access-key> \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.secret-access-key=<Your-ecs-s3-secret-access-key>
+```
+
+Then, use `my_catalog` to access the data in ECS. You can use `SHOW NAMESPACES IN my_catalog` and `SHOW TABLES IN my_catalog` to fetch the namespaces and tables of the catalog.
+
+The related problems of catalog usage:
+
+1. The `pyspark.sql.SparkSession.catalog` won't access the 3rd-party catalog of Spark, so please use DDL SQL to list all tables and namespaces.
+
+
+### Flink
+
+Use the Dell ECS catalog with Flink, you first must create a Flink environment.
+
+```bash
+# HADOOP_HOME is your hadoop root directory after unpack the binary package.
+export HADOOP_CLASSPATH=`$HADOOP_HOME/bin/hadoop classpath`
+
+# download Iceberg dependency
+MAVEN_URL=https://repo1.maven.org/maven2
+ICEBERG_VERSION=0.15.0
+FLINK_VERSION=1.14
+wget ${MAVEN_URL}/org/apache/iceberg/iceberg-flink-runtime-${FLINK_VERSION}/${ICEBERG_VERSION}/iceberg-flink-runtime-${FLINK_VERSION}-${ICEBERG_VERSION}.jar
+wget ${MAVEN_URL}/org/apache/iceberg/iceberg-dell/${ICEBERG_VERSION}/iceberg-dell-${ICEBERG_VERSION}.jar
+
+# download ECS object client
+ECS_CLIENT_VERSION=3.3.2
+wget ${MAVEN_URL}/com/emc/ecs/object-client-bundle/${ECS_CLIENT_VERSION}/object-client-bundle-${ECS_CLIENT_VERSION}.jar
+
+# open the SQL client.
+/path/to/bin/sql-client.sh embedded \
+    -j iceberg-flink-runtime-${FLINK_VERSION}-${ICEBERG_VERSION}.jar \
+    -j iceberg-dell-${ICEBERG_VERSION}.jar \
+    -j object-client-bundle-${ECS_CLIENT_VERSION}.jar \
+    shell
+```
+
+Then, use Flink SQL to create a catalog named `my_catalog`:
+
+```SQL
+CREATE CATALOG my_catalog WITH (
+    'type'='iceberg',
+    'warehouse' = 'ecs://bucket-a/namespace-a',
+    'catalog-impl'='org.apache.iceberg.dell.ecs.EcsCatalog',
+    'ecs.s3.endpoint' = 'http://10.x.x.x:9020',
+    'ecs.s3.access-key-id' = '<Your-ecs-s3-access-key>',
+    'ecs.s3.secret-access-key' = '<Your-ecs-s3-secret-access-key>')
+```
+
+Then, you can run `USE CATALOG my_catalog`, `SHOW DATABASES`, and `SHOW TABLES` to fetch the namespaces and tables of the catalog.
+

Review Comment:
   It's a good idea. Relevant tests are being prepared. And I wish to merge this document firstly. 



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
openinx commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r895287304


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:
+
+| Name                     | Description            |
+| ------------------------ | ---------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | Username               |
+| ecs.s3.secret-access-key | S3 Secret Key          |
+
+As for the catalog, you should provide a warehouse location where will store all data and metadata later.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last / will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Dependencies of runtime
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell ECS still recommends using the latest version of the client.
+
+### Spark
+
+For example, to use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=ecs://bucket-a/namespace-a \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.dell.ecs.EcsCatalog \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.endpoint=http://10.x.x.x:9020 \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.access-key-id=test \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.secret-access-key=xxxxxxxxxxxxxxxx
+```
+
+```python
+from pyspark.sql import SparkSession
+
+spark = SparkSession.builder\
+    .config("spark.jars.packages", "org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2")\
+    .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")\
+    .config("spark.sql.catalog.my_catalog", "org.apache.iceberg.spark.SparkCatalog")\
+    .config("spark.sql.catalog.my_catalog.warehouse", "ecs://bucket-a/namespace-a")\
+    .config("spark.sql.catalog.my_catalog.catalog-impl", "org.apache.iceberg.dell.ecs.EcsCatalog")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.endpoint", "http://10.x.x.x:9020")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.access-key-id", "test")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.secret-access-key", "xxxxxxxxxxxxxxxx")\
+    .getOrCreate()
+```
+
+Then, use `my_catalog` to access the data in ECS. You can use `SHOW NAMESPACES IN my_catalog` and `SHOW TABLES IN my_catalog` to fetch the namespaces and tables of the catalog.
+
+The related problems of catalog usage:
+
+1. The [pyspark.sql.SparkSession.catalog](https://spark.apache.org/docs/latest/api/python/reference/api/pyspark.sql.SparkSession.catalog.html#pyspark.sql.SparkSession.catalog) won't access the 3rd-party catalog of Spark, so please use DDL SQL to list all tables and namespaces.
+
+
+### Flink
+
+For example, to use the Dell ECS catalog with Flink 1.14, you should create a Flink environment like:
+
+```sh
+wget https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-flink-runtime-1.14/0.14.0/iceberg-flink-runtime-1.14-0.14.0.jar
+wget https://repo1.maven.org/maven2/com/emc/ecs/object-client-bundle/3.3.2/object-client-bundle-3.3.2.jar
+
+sql-client.sh embedded -j iceberg-flink-runtime-1.14-0.14.0.jar -j object-client-bundle-3.3.2.jar shell
+```
+
+```python
+import requests
+from pyflink.datastream import StreamExecutionEnvironment
+from pyflink.table import StreamTableEnvironment
+
+# Set your workspace
+work_space = "/"
+
+# Download libraries
+maven_url="https://repo1.maven.org/maven2"
+with open(f"{work_space}/iceberg-flink-runtime-1.14-0.14.0.jar", "wb") as f:
+  f.write(requests.get(f"{maven_url}/org/apache/iceberg/iceberg-flink-runtime-1.14/0.14.0/iceberg-flink-runtime-1.14-0.14.0.jar").content)
+with open(f"{work_space}/object-client-bundle-3.3.2.jar", "wb") as f:
+  f.write(requests.get(f"{maven_url}/com/emc/ecs/object-client-bundle/3.3.2/object-client-bundle-3.3.2.jar").content)

Review Comment:
   Let's put the maven urls and file name into a map so that we don't need to attach the const concated strings in the download code piece ? 



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
openinx commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r962439955


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,132 @@
+---

Review Comment:
   Is this document path still valid ?  I saw the current docs directory don't have any integration now ?  would you mind to update this PR ? 



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] wang-x-xia commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
wang-x-xia commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r962683766


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,132 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integration
+
+## Dell ECS Integration
+
+Iceberg can be used with Dell's Enterprise Object Storage (ECS) by using the ECS catalog since 0.14.0.
+
+Dell ECS has many features that make Iceberg a highly compatible table format, such as append operations for file writers and content addressable storage (CAS) for table commits. See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.

Review Comment:
   This part is already removed in the latest version.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
openinx commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r962446066


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,132 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integration
+
+## Dell ECS Integration
+
+Iceberg can be used with Dell's Enterprise Object Storage (ECS) by using the ECS catalog since 0.14.0.
+
+Dell ECS has many features that make Iceberg a highly compatible table format, such as append operations for file writers and content addressable storage (CAS) for table commits. See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When using Dell ECS with Iceberg, these configuration parameters are required:
+
+| Name                     | Description             |
+| ------------------------ | ----------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | ECS Username            |
+| ecs.s3.secret-access-key | S3 Secret Key           |
+
+An ECS catalog requires that you configure a warehouse location where all data and metadata will be created.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last `/` will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Runtime dependencies
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell EMC still recommends using the latest version of the client.
+
+### Spark
+
+For example, to use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```python
+from pyspark.sql import SparkSession
+
+spark = SparkSession.builder
+    .config("spark.jars.packages", "org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2")
+    .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")
+    .config("spark.sql.catalog.my_catalog", "org.apache.iceberg.spark.SparkCatalog")
+    .config("spark.sql.catalog.my_catalog.warehouse", "ecs://bucket-a/namespace-a")
+    .config("spark.sql.catalog.my_catalog.catalog-impl", "org.apache.iceberg.dell.ecs.EcsCatalog")
+    .config("spark.sql.catalog.my_catalog.ecs.s3.endpoint", "http://10.x.x.x:9020")
+    .config("spark.sql.catalog.my_catalog.ecs.s3.access-key-id", "<Your-ecs-s3-access-key>")
+    .config("spark.sql.catalog.my_catalog.ecs.s3.secret-access-key", "<Your-ecs-s3-secret-access-key>")
+    .getOrCreate()
+```
+
+Then, use `my_catalog` to access the data in ECS. You can use `SHOW NAMESPACES IN my_catalog` and `SHOW TABLES IN my_catalog` to fetch the namespaces and tables of the catalog.
+
+The related problems of catalog usage:
+
+1. The [pyspark.sql.SparkSession.catalog](https://spark.apache.org/docs/latest/api/python/reference/api/pyspark.sql.SparkSession.catalog.html#pyspark.sql.SparkSession.catalog) won't access the 3rd-party catalog of Spark, so please use DDL SQL to list all tables and namespaces.

Review Comment:
   The link is 404 now.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r933579465


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,132 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integration
+
+## Dell ECS Integration
+
+Iceberg can be used with Dell's Enterprise Object Storage (ECS) by using the ECS catalog since 0.14.0.
+
+Dell ECS has many features that make Iceberg a highly compatible table format, such as append operations for file writers and content addressable storage (CAS) for table commits. See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.

Review Comment:
   I'm not sure what it means to be a "highly compatible table format". I think this should just have the second sentence, "See Dell ECS for more information."



##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,132 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integration
+
+## Dell ECS Integration
+
+Iceberg can be used with Dell's Enterprise Object Storage (ECS) by using the ECS catalog since 0.14.0.
+
+Dell ECS has many features that make Iceberg a highly compatible table format, such as append operations for file writers and content addressable storage (CAS) for table commits. See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When using Dell ECS with Iceberg, these configuration parameters are required:

Review Comment:
   Where?



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] wang-x-xia commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
wang-x-xia commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r896421524


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:

Review Comment:
   Done.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] wang-x-xia commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
wang-x-xia commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r962684103


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,132 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integration
+
+## Dell ECS Integration
+
+Iceberg can be used with Dell's Enterprise Object Storage (ECS) by using the ECS catalog since 0.14.0.
+
+Dell ECS has many features that make Iceberg a highly compatible table format, such as append operations for file writers and content addressable storage (CAS) for table commits. See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When using Dell ECS with Iceberg, these configuration parameters are required:
+
+| Name                     | Description             |
+| ------------------------ | ----------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | ECS Username            |
+| ecs.s3.secret-access-key | S3 Secret Key           |
+
+An ECS catalog requires that you configure a warehouse location where all data and metadata will be created.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last `/` will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Runtime dependencies
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell EMC still recommends using the latest version of the client.
+
+### Spark
+
+For example, to use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```python
+from pyspark.sql import SparkSession
+
+spark = SparkSession.builder
+    .config("spark.jars.packages", "org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2")
+    .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")
+    .config("spark.sql.catalog.my_catalog", "org.apache.iceberg.spark.SparkCatalog")
+    .config("spark.sql.catalog.my_catalog.warehouse", "ecs://bucket-a/namespace-a")
+    .config("spark.sql.catalog.my_catalog.catalog-impl", "org.apache.iceberg.dell.ecs.EcsCatalog")
+    .config("spark.sql.catalog.my_catalog.ecs.s3.endpoint", "http://10.x.x.x:9020")
+    .config("spark.sql.catalog.my_catalog.ecs.s3.access-key-id", "<Your-ecs-s3-access-key>")
+    .config("spark.sql.catalog.my_catalog.ecs.s3.secret-access-key", "<Your-ecs-s3-secret-access-key>")
+    .getOrCreate()
+```
+
+Then, use `my_catalog` to access the data in ECS. You can use `SHOW NAMESPACES IN my_catalog` and `SHOW TABLES IN my_catalog` to fetch the namespaces and tables of the catalog.
+
+The related problems of catalog usage:
+
+1. The [pyspark.sql.SparkSession.catalog](https://spark.apache.org/docs/latest/api/python/reference/api/pyspark.sql.SparkSession.catalog.html#pyspark.sql.SparkSession.catalog) won't access the 3rd-party catalog of Spark, so please use DDL SQL to list all tables and namespaces.
+
+
+### Flink
+
+Use the Dell ECS catalog with Flink, you first must create a Flink environment.
+
+```python
+import requests

Review Comment:
   Now the scripts are bash-script.



##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,132 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integration
+
+## Dell ECS Integration
+
+Iceberg can be used with Dell's Enterprise Object Storage (ECS) by using the ECS catalog since 0.14.0.
+
+Dell ECS has many features that make Iceberg a highly compatible table format, such as append operations for file writers and content addressable storage (CAS) for table commits. See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When using Dell ECS with Iceberg, these configuration parameters are required:
+
+| Name                     | Description             |
+| ------------------------ | ----------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | ECS Username            |
+| ecs.s3.secret-access-key | S3 Secret Key           |
+
+An ECS catalog requires that you configure a warehouse location where all data and metadata will be created.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last `/` will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Runtime dependencies
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell EMC still recommends using the latest version of the client.
+
+### Spark
+
+For example, to use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```python
+from pyspark.sql import SparkSession
+
+spark = SparkSession.builder
+    .config("spark.jars.packages", "org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2")
+    .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")
+    .config("spark.sql.catalog.my_catalog", "org.apache.iceberg.spark.SparkCatalog")
+    .config("spark.sql.catalog.my_catalog.warehouse", "ecs://bucket-a/namespace-a")
+    .config("spark.sql.catalog.my_catalog.catalog-impl", "org.apache.iceberg.dell.ecs.EcsCatalog")
+    .config("spark.sql.catalog.my_catalog.ecs.s3.endpoint", "http://10.x.x.x:9020")
+    .config("spark.sql.catalog.my_catalog.ecs.s3.access-key-id", "<Your-ecs-s3-access-key>")
+    .config("spark.sql.catalog.my_catalog.ecs.s3.secret-access-key", "<Your-ecs-s3-secret-access-key>")
+    .getOrCreate()
+```
+
+Then, use `my_catalog` to access the data in ECS. You can use `SHOW NAMESPACES IN my_catalog` and `SHOW TABLES IN my_catalog` to fetch the namespaces and tables of the catalog.
+
+The related problems of catalog usage:
+
+1. The [pyspark.sql.SparkSession.catalog](https://spark.apache.org/docs/latest/api/python/reference/api/pyspark.sql.SparkSession.catalog.html#pyspark.sql.SparkSession.catalog) won't access the 3rd-party catalog of Spark, so please use DDL SQL to list all tables and namespaces.

Review Comment:
   The link is removed.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
openinx commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r962870328


##########
docs/dell.md:
##########
@@ -0,0 +1,135 @@
+---
+title: "Dell"
+url: dell
+menu:
+    main:
+        parent: Integrations
+        weight: 0
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integration
+
+## Dell ECS Integration
+
+Iceberg can be used with Dell's Enterprise Object Storage (ECS) by using the ECS catalog since 0.15.0.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Parameters
+
+When using Dell ECS with Iceberg, these configuration parameters are required:
+
+| Name                     | Description                       |
+| ------------------------ | --------------------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint           |
+| ecs.s3.access-key-id     | ECS Username                      |
+| ecs.s3.secret-access-key | S3 Secret Key                     |
+| warehouse                | The location of data and metadata |
+
+The warehouse should use the following formats:
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/            | Use the whole bucket as the data. The last `/` is ignored.      |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. You should pick the correct version.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell EMC still recommends using the latest version of the client.
+
+### Spark
+
+To use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```bash
+ICEBERG_VERSION=0.15.0
+SPARK_VERSION=3.2_2.12
+ECS_CLIENT_VERSION=3.3.2
+
+DEPENDENCIES="org.apache.iceberg:iceberg-spark-runtime-${SPARK_VERSION}:${ICEBERG_VERSION},\
+org.apache.iceberg:iceberg-dell:${ICEBERG_VERSION},\
+com.emc.ecs:object-client-bundle:${ECS_CLIENT_VERSION}"
+
+spark-sql --packages ${DEPENDENCIES} \
+    --conf spark.sql.extensions=org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=ecs://bucket-a/namespace-a \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.dell.ecs.EcsCatalog \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.endpoint=http://10.x.x.x:9020 \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.access-key-id=<Your-ecs-s3-access-key> \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.secret-access-key=<Your-ecs-s3-secret-access-key>
+```
+
+Then, use `my_catalog` to access the data in ECS. You can use `SHOW NAMESPACES IN my_catalog` and `SHOW TABLES IN my_catalog` to fetch the namespaces and tables of the catalog.
+
+The related problems of catalog usage:
+
+1. The `pyspark.sql.SparkSession.catalog` won't access the 3rd-party catalog of Spark, so please use DDL SQL to list all tables and namespaces.
+
+
+### Flink
+
+Use the Dell ECS catalog with Flink, you first must create a Flink environment.
+
+```bash
+# HADOOP_HOME is your hadoop root directory after unpack the binary package.
+export HADOOP_CLASSPATH=`$HADOOP_HOME/bin/hadoop classpath`
+
+# download Iceberg dependency
+MAVEN_URL=https://repo1.maven.org/maven2
+ICEBERG_VERSION=0.15.0
+FLINK_VERSION=1.14
+wget ${MAVEN_URL}/org/apache/iceberg/iceberg-flink-runtime-${FLINK_VERSION}/${ICEBERG_VERSION}/iceberg-flink-runtime-${FLINK_VERSION}-${ICEBERG_VERSION}.jar
+wget ${MAVEN_URL}/org/apache/iceberg/iceberg-dell/${ICEBERG_VERSION}/iceberg-dell-${ICEBERG_VERSION}.jar
+
+# download ECS object client
+ECS_CLIENT_VERSION=3.3.2
+wget ${MAVEN_URL}/com/emc/ecs/object-client-bundle/${ECS_CLIENT_VERSION}/object-client-bundle-${ECS_CLIENT_VERSION}.jar
+
+# open the SQL client.
+/path/to/bin/sql-client.sh embedded \
+    -j iceberg-flink-runtime-${FLINK_VERSION}-${ICEBERG_VERSION}.jar \
+    -j iceberg-dell-${ICEBERG_VERSION}.jar \
+    -j object-client-bundle-${ECS_CLIENT_VERSION}.jar \
+    shell
+```
+
+Then, use Flink SQL to create a catalog named `my_catalog`:
+
+```SQL
+CREATE CATALOG my_catalog WITH (
+    'type'='iceberg',
+    'warehouse' = 'ecs://bucket-a/namespace-a',
+    'catalog-impl'='org.apache.iceberg.dell.ecs.EcsCatalog',
+    'ecs.s3.endpoint' = 'http://10.x.x.x:9020',
+    'ecs.s3.access-key-id' = '<Your-ecs-s3-access-key>',
+    'ecs.s3.secret-access-key' = '<Your-ecs-s3-secret-access-key>')
+```
+
+Then, you can run `USE CATALOG my_catalog`, `SHOW DATABASES`, and `SHOW TABLES` to fetch the namespaces and tables of the catalog.
+

Review Comment:
   Open question: Should we also add hive section ? I guess many people still use hive tez to process their data, it's okay if you think we don't need to. Thanks.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
rdblue commented on PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#issuecomment-1170239730

   Since this PR is replaced by newer ones, I'm going to close it.


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue closed pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
rdblue closed pull request #4993: Dell: Add document.
URL: https://github.com/apache/iceberg/pull/4993


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] wang-x-xia commented on pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
wang-x-xia commented on PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#issuecomment-1170702862

   @rdblue 
   This PR is for documents only. I moved other changes to new PRs except the document.
   Should I create a new PR for the ducoment?


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r898256984


##########
spark/v3.2/build.gradle:
##########
@@ -213,6 +213,7 @@ project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersio
       exclude group: 'commons-logging', module: 'commons-logging'
     }
     implementation project(':iceberg-hive-metastore')
+    implementation project(':iceberg-dell')

Review Comment:
   Adding this module to the Spark runtime should be discussed and done in a separate PR, not in a documentation PR. Can you bring this up on the dev list and summarize what it pulls in and changes?
   
   You'll also need to update the LICENSE and NOTICE files.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
openinx commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r895287525


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:
+
+| Name                     | Description            |
+| ------------------------ | ---------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | Username               |
+| ecs.s3.secret-access-key | S3 Secret Key          |
+
+As for the catalog, you should provide a warehouse location where will store all data and metadata later.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last / will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Dependencies of runtime
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell ECS still recommends using the latest version of the client.
+
+### Spark
+
+For example, to use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=ecs://bucket-a/namespace-a \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.dell.ecs.EcsCatalog \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.endpoint=http://10.x.x.x:9020 \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.access-key-id=test \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.secret-access-key=xxxxxxxxxxxxxxxx
+```
+
+```python
+from pyspark.sql import SparkSession
+
+spark = SparkSession.builder\
+    .config("spark.jars.packages", "org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2")\
+    .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")\
+    .config("spark.sql.catalog.my_catalog", "org.apache.iceberg.spark.SparkCatalog")\
+    .config("spark.sql.catalog.my_catalog.warehouse", "ecs://bucket-a/namespace-a")\
+    .config("spark.sql.catalog.my_catalog.catalog-impl", "org.apache.iceberg.dell.ecs.EcsCatalog")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.endpoint", "http://10.x.x.x:9020")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.access-key-id", "test")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.secret-access-key", "xxxxxxxxxxxxxxxx")\
+    .getOrCreate()
+```
+
+Then, use `my_catalog` to access the data in ECS. You can use `SHOW NAMESPACES IN my_catalog` and `SHOW TABLES IN my_catalog` to fetch the namespaces and tables of the catalog.
+
+The related problems of catalog usage:
+
+1. The [pyspark.sql.SparkSession.catalog](https://spark.apache.org/docs/latest/api/python/reference/api/pyspark.sql.SparkSession.catalog.html#pyspark.sql.SparkSession.catalog) won't access the 3rd-party catalog of Spark, so please use DDL SQL to list all tables and namespaces.
+
+
+### Flink
+
+For example, to use the Dell ECS catalog with Flink 1.14, you should create a Flink environment like:
+
+```sh
+wget https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-flink-runtime-1.14/0.14.0/iceberg-flink-runtime-1.14-0.14.0.jar
+wget https://repo1.maven.org/maven2/com/emc/ecs/object-client-bundle/3.3.2/object-client-bundle-3.3.2.jar
+
+sql-client.sh embedded -j iceberg-flink-runtime-1.14-0.14.0.jar -j object-client-bundle-3.3.2.jar shell
+```
+
+```python
+import requests
+from pyflink.datastream import StreamExecutionEnvironment
+from pyflink.table import StreamTableEnvironment
+
+# Set your workspace
+work_space = "/"
+
+# Download libraries
+maven_url="https://repo1.maven.org/maven2"
+with open(f"{work_space}/iceberg-flink-runtime-1.14-0.14.0.jar", "wb") as f:

Review Comment:
   Will we put this artifact in `//iceberg-flink-runtime-1.14-0.14.0.jar` ? Is it correct ? 



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] wang-x-xia commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
wang-x-xia commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r896422366


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:
+
+| Name                     | Description            |
+| ------------------------ | ---------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | Username               |
+| ecs.s3.secret-access-key | S3 Secret Key          |
+
+As for the catalog, you should provide a warehouse location where will store all data and metadata later.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last / will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Dependencies of runtime
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell ECS still recommends using the latest version of the client.
+
+### Spark
+
+For example, to use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=ecs://bucket-a/namespace-a \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.dell.ecs.EcsCatalog \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.endpoint=http://10.x.x.x:9020 \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.access-key-id=test \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.secret-access-key=xxxxxxxxxxxxxxxx
+```
+
+```python
+from pyspark.sql import SparkSession
+
+spark = SparkSession.builder\
+    .config("spark.jars.packages", "org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2")\
+    .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")\
+    .config("spark.sql.catalog.my_catalog", "org.apache.iceberg.spark.SparkCatalog")\
+    .config("spark.sql.catalog.my_catalog.warehouse", "ecs://bucket-a/namespace-a")\
+    .config("spark.sql.catalog.my_catalog.catalog-impl", "org.apache.iceberg.dell.ecs.EcsCatalog")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.endpoint", "http://10.x.x.x:9020")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.access-key-id", "test")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.secret-access-key", "xxxxxxxxxxxxxxxx")\
+    .getOrCreate()
+```
+
+Then, use `my_catalog` to access the data in ECS. You can use `SHOW NAMESPACES IN my_catalog` and `SHOW TABLES IN my_catalog` to fetch the namespaces and tables of the catalog.
+
+The related problems of catalog usage:
+
+1. The [pyspark.sql.SparkSession.catalog](https://spark.apache.org/docs/latest/api/python/reference/api/pyspark.sql.SparkSession.catalog.html#pyspark.sql.SparkSession.catalog) won't access the 3rd-party catalog of Spark, so please use DDL SQL to list all tables and namespaces.
+
+
+### Flink
+
+For example, to use the Dell ECS catalog with Flink 1.14, you should create a Flink environment like:

Review Comment:
   Done.



##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:
+
+| Name                     | Description            |
+| ------------------------ | ---------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | Username               |
+| ecs.s3.secret-access-key | S3 Secret Key          |
+
+As for the catalog, you should provide a warehouse location where will store all data and metadata later.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last / will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Dependencies of runtime
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell ECS still recommends using the latest version of the client.
+
+### Spark
+
+For example, to use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=ecs://bucket-a/namespace-a \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.dell.ecs.EcsCatalog \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.endpoint=http://10.x.x.x:9020 \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.access-key-id=test \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.secret-access-key=xxxxxxxxxxxxxxxx
+```
+
+```python
+from pyspark.sql import SparkSession
+
+spark = SparkSession.builder\
+    .config("spark.jars.packages", "org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2")\
+    .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")\
+    .config("spark.sql.catalog.my_catalog", "org.apache.iceberg.spark.SparkCatalog")\
+    .config("spark.sql.catalog.my_catalog.warehouse", "ecs://bucket-a/namespace-a")\
+    .config("spark.sql.catalog.my_catalog.catalog-impl", "org.apache.iceberg.dell.ecs.EcsCatalog")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.endpoint", "http://10.x.x.x:9020")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.access-key-id", "test")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.secret-access-key", "xxxxxxxxxxxxxxxx")\
+    .getOrCreate()
+```
+
+Then, use `my_catalog` to access the data in ECS. You can use `SHOW NAMESPACES IN my_catalog` and `SHOW TABLES IN my_catalog` to fetch the namespaces and tables of the catalog.
+
+The related problems of catalog usage:
+
+1. The [pyspark.sql.SparkSession.catalog](https://spark.apache.org/docs/latest/api/python/reference/api/pyspark.sql.SparkSession.catalog.html#pyspark.sql.SparkSession.catalog) won't access the 3rd-party catalog of Spark, so please use DDL SQL to list all tables and namespaces.
+
+
+### Flink
+
+For example, to use the Dell ECS catalog with Flink 1.14, you should create a Flink environment like:
+
+```sh
+wget https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-flink-runtime-1.14/0.14.0/iceberg-flink-runtime-1.14-0.14.0.jar
+wget https://repo1.maven.org/maven2/com/emc/ecs/object-client-bundle/3.3.2/object-client-bundle-3.3.2.jar
+
+sql-client.sh embedded -j iceberg-flink-runtime-1.14-0.14.0.jar -j object-client-bundle-3.3.2.jar shell
+```
+
+```python
+import requests
+from pyflink.datastream import StreamExecutionEnvironment
+from pyflink.table import StreamTableEnvironment
+
+# Set your workspace
+work_space = "/"
+
+# Download libraries
+maven_url="https://repo1.maven.org/maven2"
+with open(f"{work_space}/iceberg-flink-runtime-1.14-0.14.0.jar", "wb") as f:
+  f.write(requests.get(f"{maven_url}/org/apache/iceberg/iceberg-flink-runtime-1.14/0.14.0/iceberg-flink-runtime-1.14-0.14.0.jar").content)
+with open(f"{work_space}/object-client-bundle-3.3.2.jar", "wb") as f:
+  f.write(requests.get(f"{maven_url}/com/emc/ecs/object-client-bundle/3.3.2/object-client-bundle-3.3.2.jar").content)
+
+jars = ["iceberg-flink-runtime-1.14-0.14.0.jar", "object-client-bundle-3.3.2.jar"]
+pipeline_jars = [f"file://{work_space}/{jar}" for jar in jars]
+
+# Setup Flink session
+env = StreamExecutionEnvironment.get_execution_environment()
+env.add_jars(*pipeline_jars)
+t_env = StreamTableEnvironment.create(env)
+```
+
+Then, use Flink SQL to create a catalog named `my_catalog`:
+
+```SQL
+CREATE CATALOG my_catalog WITH (
+    'type'='iceberg',
+    'warehouse' = 'ecs://bucket-a/namespace-a',
+    'catalog-impl'='org.apache.iceberg.dell.ecs.EcsCatalog',
+    'ecs.s3.endpoint' = 'http://10.x.x.x:9020',
+    'ecs.s3.access-key-id' = 'test',
+    'ecs.s3.secret-access-key' = 'xxxxxxxxxxxxxxxx')
+```
+
+Then, `USE CATALOG my_catalog`, `SHOW DATABASES`, and `SHOW TABLES` to fetch the namespaces and tables of the catalog.

Review Comment:
   Done.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] wang-x-xia commented on pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
wang-x-xia commented on PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#issuecomment-1154771852

   Thanks for @samredai 's help. My English is bad. If you have other suggestions, I'll fix them to make this document more readable.


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx merged pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
openinx merged PR #4993:
URL: https://github.com/apache/iceberg/pull/4993


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r933580482


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,132 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integration
+
+## Dell ECS Integration
+
+Iceberg can be used with Dell's Enterprise Object Storage (ECS) by using the ECS catalog since 0.14.0.
+
+Dell ECS has many features that make Iceberg a highly compatible table format, such as append operations for file writers and content addressable storage (CAS) for table commits. See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When using Dell ECS with Iceberg, these configuration parameters are required:
+
+| Name                     | Description             |
+| ------------------------ | ----------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | ECS Username            |
+| ecs.s3.secret-access-key | S3 Secret Key           |
+
+An ECS catalog requires that you configure a warehouse location where all data and metadata will be created.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last `/` will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Runtime dependencies
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.

Review Comment:
   This shouldn't start a section. It's probably a good note (if your version is different, make sure you have the right runtime Jar!).



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] wang-x-xia commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
wang-x-xia commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r934130772


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,132 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integration
+
+## Dell ECS Integration
+
+Iceberg can be used with Dell's Enterprise Object Storage (ECS) by using the ECS catalog since 0.14.0.
+
+Dell ECS has many features that make Iceberg a highly compatible table format, such as append operations for file writers and content addressable storage (CAS) for table commits. See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When using Dell ECS with Iceberg, these configuration parameters are required:
+
+| Name                     | Description             |
+| ------------------------ | ----------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | ECS Username            |
+| ecs.s3.secret-access-key | S3 Secret Key           |
+
+An ECS catalog requires that you configure a warehouse location where all data and metadata will be created.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last `/` will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Runtime dependencies
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell EMC still recommends using the latest version of the client.
+
+### Spark
+
+For example, to use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```python
+from pyspark.sql import SparkSession
+
+spark = SparkSession.builder
+    .config("spark.jars.packages", "org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2")
+    .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")
+    .config("spark.sql.catalog.my_catalog", "org.apache.iceberg.spark.SparkCatalog")
+    .config("spark.sql.catalog.my_catalog.warehouse", "ecs://bucket-a/namespace-a")
+    .config("spark.sql.catalog.my_catalog.catalog-impl", "org.apache.iceberg.dell.ecs.EcsCatalog")
+    .config("spark.sql.catalog.my_catalog.ecs.s3.endpoint", "http://10.x.x.x:9020")
+    .config("spark.sql.catalog.my_catalog.ecs.s3.access-key-id", "<Your-ecs-s3-access-key>")
+    .config("spark.sql.catalog.my_catalog.ecs.s3.secret-access-key", "<Your-ecs-s3-secret-access-key>")
+    .getOrCreate()
+```
+
+Then, use `my_catalog` to access the data in ECS. You can use `SHOW NAMESPACES IN my_catalog` and `SHOW TABLES IN my_catalog` to fetch the namespaces and tables of the catalog.
+
+The related problems of catalog usage:
+
+1. The [pyspark.sql.SparkSession.catalog](https://spark.apache.org/docs/latest/api/python/reference/api/pyspark.sql.SparkSession.catalog.html#pyspark.sql.SparkSession.catalog) won't access the 3rd-party catalog of Spark, so please use DDL SQL to list all tables and namespaces.
+
+
+### Flink
+
+Use the Dell ECS catalog with Flink, you first must create a Flink environment.
+
+```python
+import requests
+from pyflink.datastream import StreamExecutionEnvironment
+from pyflink.table import StreamTableEnvironment
+
+# Set your workspace
+work_space = "<your_work_space>"
+
+jars = {
+    "iceberg-flink-runtime-1.14-0.14.0.jar" : "https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-flink-runtime-1.14/0.14.0/iceberg-flink-runtime-1.14-0.14.0.jar", 
+    "object-client-bundle-3.3.2.jar" : "https://repo1.maven.org/maven2/com/emc/ecs/object-client-bundle/3.3.2/object-client-bundle-3.3.2.jar",
+}
+
+# Download libraries
+for jar, link in jars.items():
+    with open(f"{work_space}/{jar}", "wb") as f:
+      f.write(requests.get(link).content)
+
+pipeline_jars = [f"file://{work_space}/{jar}" for jar in jars]
+
+# Setup Flink session
+env = StreamExecutionEnvironment.get_execution_environment()
+env.add_jars(*pipeline_jars)
+t_env = StreamTableEnvironment.create(env)
+```
+
+Then, use Flink SQL to create a catalog named `my_catalog`:
+
+```SQL
+CREATE CATALOG my_catalog WITH (
+    'type'='iceberg',
+    'warehouse' = 'ecs://bucket-a/namespace-a',
+    'catalog-impl'='org.apache.iceberg.dell.ecs.EcsCatalog',
+    'ecs.s3.endpoint' = 'http://10.x.x.x:9020',
+    'ecs.s3.access-key-id' = '<Your-ecs-s3-access-key>',
+    'ecs.s3.secret-access-key' = '<Your-ecs-s3-secret-access-key>')
+```
+
+Then, you can run `USE CATALOG my_catalog`, `SHOW DATABASES`, and `SHOW TABLES` to fetch the namespaces and tables of the catalog.
+
+### Limitations
+
+When you use the catalog with Dell ECS only, you should care about these limitations:
+
+1. `RENAME` statements are supported without other protections. When you try to rename a table, you need to guarantee all commits are finished in the original table.
+2. `RENAME` statements only rename the table without moving any data files. This can lead to a table's data being stored in a path outside of the configured warehouse path.

Review Comment:
   It's just a rename for the catalog layer.
   Only the entry object which store the latest version will be renamed, but not changed.
   So the data controlled by Iceberg won't be changed. Any pointers won't be broken.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] wang-x-xia commented on pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
wang-x-xia commented on PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#issuecomment-1174544970

   @rdblue If you have some comments about the document, please let me know. This PR needs your approval now.😂


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] wang-x-xia commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
wang-x-xia commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r896421080


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.

Review Comment:
   I've changed it.



##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:
+
+| Name                     | Description            |
+| ------------------------ | ---------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | Username               |
+| ecs.s3.secret-access-key | S3 Secret Key          |
+
+As for the catalog, you should provide a warehouse location where will store all data and metadata later.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last / will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.

Review Comment:
   Done.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
openinx commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r895288199


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:
+
+| Name                     | Description            |
+| ------------------------ | ---------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | Username               |
+| ecs.s3.secret-access-key | S3 Secret Key          |
+
+As for the catalog, you should provide a warehouse location where will store all data and metadata later.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last / will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Dependencies of runtime
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell ECS still recommends using the latest version of the client.
+
+### Spark
+
+For example, to use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=ecs://bucket-a/namespace-a \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.dell.ecs.EcsCatalog \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.endpoint=http://10.x.x.x:9020 \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.access-key-id=test \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.secret-access-key=xxxxxxxxxxxxxxxx
+```
+
+```python
+from pyspark.sql import SparkSession
+
+spark = SparkSession.builder\
+    .config("spark.jars.packages", "org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2")\
+    .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")\
+    .config("spark.sql.catalog.my_catalog", "org.apache.iceberg.spark.SparkCatalog")\
+    .config("spark.sql.catalog.my_catalog.warehouse", "ecs://bucket-a/namespace-a")\
+    .config("spark.sql.catalog.my_catalog.catalog-impl", "org.apache.iceberg.dell.ecs.EcsCatalog")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.endpoint", "http://10.x.x.x:9020")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.access-key-id", "test")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.secret-access-key", "xxxxxxxxxxxxxxxx")\
+    .getOrCreate()
+```
+
+Then, use `my_catalog` to access the data in ECS. You can use `SHOW NAMESPACES IN my_catalog` and `SHOW TABLES IN my_catalog` to fetch the namespaces and tables of the catalog.
+
+The related problems of catalog usage:
+
+1. The [pyspark.sql.SparkSession.catalog](https://spark.apache.org/docs/latest/api/python/reference/api/pyspark.sql.SparkSession.catalog.html#pyspark.sql.SparkSession.catalog) won't access the 3rd-party catalog of Spark, so please use DDL SQL to list all tables and namespaces.
+
+
+### Flink
+
+For example, to use the Dell ECS catalog with Flink 1.14, you should create a Flink environment like:
+
+```sh
+wget https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-flink-runtime-1.14/0.14.0/iceberg-flink-runtime-1.14-0.14.0.jar
+wget https://repo1.maven.org/maven2/com/emc/ecs/object-client-bundle/3.3.2/object-client-bundle-3.3.2.jar
+
+sql-client.sh embedded -j iceberg-flink-runtime-1.14-0.14.0.jar -j object-client-bundle-3.3.2.jar shell
+```
+
+```python
+import requests
+from pyflink.datastream import StreamExecutionEnvironment
+from pyflink.table import StreamTableEnvironment
+
+# Set your workspace
+work_space = "/"
+
+# Download libraries
+maven_url="https://repo1.maven.org/maven2"
+with open(f"{work_space}/iceberg-flink-runtime-1.14-0.14.0.jar", "wb") as f:
+  f.write(requests.get(f"{maven_url}/org/apache/iceberg/iceberg-flink-runtime-1.14/0.14.0/iceberg-flink-runtime-1.14-0.14.0.jar").content)
+with open(f"{work_space}/object-client-bundle-3.3.2.jar", "wb") as f:
+  f.write(requests.get(f"{maven_url}/com/emc/ecs/object-client-bundle/3.3.2/object-client-bundle-3.3.2.jar").content)
+
+jars = ["iceberg-flink-runtime-1.14-0.14.0.jar", "object-client-bundle-3.3.2.jar"]
+pipeline_jars = [f"file://{work_space}/{jar}" for jar in jars]
+
+# Setup Flink session
+env = StreamExecutionEnvironment.get_execution_environment()
+env.add_jars(*pipeline_jars)
+t_env = StreamTableEnvironment.create(env)
+```
+
+Then, use Flink SQL to create a catalog named `my_catalog`:
+
+```SQL
+CREATE CATALOG my_catalog WITH (
+    'type'='iceberg',
+    'warehouse' = 'ecs://bucket-a/namespace-a',
+    'catalog-impl'='org.apache.iceberg.dell.ecs.EcsCatalog',
+    'ecs.s3.endpoint' = 'http://10.x.x.x:9020',
+    'ecs.s3.access-key-id' = 'test',
+    'ecs.s3.secret-access-key' = 'xxxxxxxxxxxxxxxx')
+```
+
+Then, `USE CATALOG my_catalog`, `SHOW DATABASES`, and `SHOW TABLES` to fetch the namespaces and tables of the catalog.

Review Comment:
   Is possible to provide a section for the HiveCatalog + Dell ECS ?  In my real experience,  there are many people whose just want to replace the HDFS storage with Object Storage Service while still maintain their tables in HiveCatalog.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] wang-x-xia commented on pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
wang-x-xia commented on PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#issuecomment-1157256538

   @rdblue
   The changes are reverted,
   I've created a new PR for bugfix:
   https://github.com/apache/iceberg/pull/5059
   And a new PR for runtime JAR:
   https://github.com/apache/iceberg/pull/5060


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] wang-x-xia commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
wang-x-xia commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r934128768


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,132 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integration
+
+## Dell ECS Integration
+
+Iceberg can be used with Dell's Enterprise Object Storage (ECS) by using the ECS catalog since 0.14.0.
+
+Dell ECS has many features that make Iceberg a highly compatible table format, such as append operations for file writers and content addressable storage (CAS) for table commits. See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When using Dell ECS with Iceberg, these configuration parameters are required:
+
+| Name                     | Description             |
+| ------------------------ | ----------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | ECS Username            |
+| ecs.s3.secret-access-key | S3 Secret Key           |
+
+An ECS catalog requires that you configure a warehouse location where all data and metadata will be created.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last `/` will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.

Review Comment:
   The following examples will show how to config these properties.



##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,132 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integration
+
+## Dell ECS Integration
+
+Iceberg can be used with Dell's Enterprise Object Storage (ECS) by using the ECS catalog since 0.14.0.
+
+Dell ECS has many features that make Iceberg a highly compatible table format, such as append operations for file writers and content addressable storage (CAS) for table commits. See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When using Dell ECS with Iceberg, these configuration parameters are required:
+
+| Name                     | Description             |
+| ------------------------ | ----------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | ECS Username            |
+| ecs.s3.secret-access-key | S3 Secret Key           |
+
+An ECS catalog requires that you configure a warehouse location where all data and metadata will be created.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last `/` will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Runtime dependencies
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.

Review Comment:
   Removed!



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] wang-x-xia commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
wang-x-xia commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r962682973


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,132 @@
+---

Review Comment:
   The document is moved and fixed the header~



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
openinx commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r962442435


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,132 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integration
+
+## Dell ECS Integration
+
+Iceberg can be used with Dell's Enterprise Object Storage (ECS) by using the ECS catalog since 0.14.0.
+
+Dell ECS has many features that make Iceberg a highly compatible table format, such as append operations for file writers and content addressable storage (CAS) for table commits. See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.

Review Comment:
   Would you mind to attach the ECS CAS link for more detailed information ? 



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
rdblue commented on PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#issuecomment-1171370827

   No, if this is only docs, then we can review it. Thanks, @wang-x-xia.


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
openinx commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r895282564


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.

Review Comment:
   Would you mind to provide more description or dell doc link for those two features ? I think we need to let others to know more about the difference between Dell ECS and other vendor Object Storage Service. 



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
openinx commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r895285113


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations

Review Comment:
   Please add a sentence to say that the `Dell ECS Integration` is available since apache iceberg 0.14.0.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] wang-x-xia commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
wang-x-xia commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r896365206


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:
+
+| Name                     | Description            |
+| ------------------------ | ---------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | Username               |
+| ecs.s3.secret-access-key | S3 Secret Key          |
+
+As for the catalog, you should provide a warehouse location where will store all data and metadata later.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last / will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Dependencies of runtime
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell ECS still recommends using the latest version of the client.
+
+### Spark
+
+For example, to use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=ecs://bucket-a/namespace-a \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.dell.ecs.EcsCatalog \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.endpoint=http://10.x.x.x:9020 \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.access-key-id=test \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.secret-access-key=xxxxxxxxxxxxxxxx
+```
+
+```python
+from pyspark.sql import SparkSession
+
+spark = SparkSession.builder\
+    .config("spark.jars.packages", "org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2")\
+    .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")\
+    .config("spark.sql.catalog.my_catalog", "org.apache.iceberg.spark.SparkCatalog")\
+    .config("spark.sql.catalog.my_catalog.warehouse", "ecs://bucket-a/namespace-a")\
+    .config("spark.sql.catalog.my_catalog.catalog-impl", "org.apache.iceberg.dell.ecs.EcsCatalog")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.endpoint", "http://10.x.x.x:9020")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.access-key-id", "test")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.secret-access-key", "xxxxxxxxxxxxxxxx")\
+    .getOrCreate()
+```
+
+Then, use `my_catalog` to access the data in ECS. You can use `SHOW NAMESPACES IN my_catalog` and `SHOW TABLES IN my_catalog` to fetch the namespaces and tables of the catalog.
+
+The related problems of catalog usage:
+
+1. The [pyspark.sql.SparkSession.catalog](https://spark.apache.org/docs/latest/api/python/reference/api/pyspark.sql.SparkSession.catalog.html#pyspark.sql.SparkSession.catalog) won't access the 3rd-party catalog of Spark, so please use DDL SQL to list all tables and namespaces.
+
+
+### Flink
+
+For example, to use the Dell ECS catalog with Flink 1.14, you should create a Flink environment like:
+
+```sh
+wget https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-flink-runtime-1.14/0.14.0/iceberg-flink-runtime-1.14-0.14.0.jar
+wget https://repo1.maven.org/maven2/com/emc/ecs/object-client-bundle/3.3.2/object-client-bundle-3.3.2.jar
+
+sql-client.sh embedded -j iceberg-flink-runtime-1.14-0.14.0.jar -j object-client-bundle-3.3.2.jar shell
+```
+
+```python
+import requests
+from pyflink.datastream import StreamExecutionEnvironment
+from pyflink.table import StreamTableEnvironment
+
+# Set your workspace
+work_space = "/"
+
+# Download libraries
+maven_url="https://repo1.maven.org/maven2"
+with open(f"{work_space}/iceberg-flink-runtime-1.14-0.14.0.jar", "wb") as f:
+  f.write(requests.get(f"{maven_url}/org/apache/iceberg/iceberg-flink-runtime-1.14/0.14.0/iceberg-flink-runtime-1.14-0.14.0.jar").content)
+with open(f"{work_space}/object-client-bundle-3.3.2.jar", "wb") as f:
+  f.write(requests.get(f"{maven_url}/com/emc/ecs/object-client-bundle/3.3.2/object-client-bundle-3.3.2.jar").content)
+
+jars = ["iceberg-flink-runtime-1.14-0.14.0.jar", "object-client-bundle-3.3.2.jar"]
+pipeline_jars = [f"file://{work_space}/{jar}" for jar in jars]
+
+# Setup Flink session
+env = StreamExecutionEnvironment.get_execution_environment()
+env.add_jars(*pipeline_jars)
+t_env = StreamTableEnvironment.create(env)
+```
+
+Then, use Flink SQL to create a catalog named `my_catalog`:
+
+```SQL
+CREATE CATALOG my_catalog WITH (
+    'type'='iceberg',
+    'warehouse' = 'ecs://bucket-a/namespace-a',
+    'catalog-impl'='org.apache.iceberg.dell.ecs.EcsCatalog',
+    'ecs.s3.endpoint' = 'http://10.x.x.x:9020',
+    'ecs.s3.access-key-id' = 'test',
+    'ecs.s3.secret-access-key' = 'xxxxxxxxxxxxxxxx')
+```
+
+Then, `USE CATALOG my_catalog`, `SHOW DATABASES`, and `SHOW TABLES` to fetch the namespaces and tables of the catalog.
+
+### Limitations
+
+When you use the catalog with Dell ECS only, you should care about these limitations:

Review Comment:
   Yes. We wish to provide an all-in-one product.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
openinx commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r895289268


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:
+
+| Name                     | Description            |
+| ------------------------ | ---------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | Username               |
+| ecs.s3.secret-access-key | S3 Secret Key          |
+
+As for the catalog, you should provide a warehouse location where will store all data and metadata later.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last / will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Dependencies of runtime
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell ECS still recommends using the latest version of the client.
+
+### Spark
+
+For example, to use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=ecs://bucket-a/namespace-a \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.dell.ecs.EcsCatalog \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.endpoint=http://10.x.x.x:9020 \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.access-key-id=test \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.secret-access-key=xxxxxxxxxxxxxxxx
+```
+
+```python
+from pyspark.sql import SparkSession
+
+spark = SparkSession.builder\
+    .config("spark.jars.packages", "org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2")\
+    .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")\
+    .config("spark.sql.catalog.my_catalog", "org.apache.iceberg.spark.SparkCatalog")\
+    .config("spark.sql.catalog.my_catalog.warehouse", "ecs://bucket-a/namespace-a")\
+    .config("spark.sql.catalog.my_catalog.catalog-impl", "org.apache.iceberg.dell.ecs.EcsCatalog")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.endpoint", "http://10.x.x.x:9020")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.access-key-id", "test")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.secret-access-key", "xxxxxxxxxxxxxxxx")\
+    .getOrCreate()
+```
+
+Then, use `my_catalog` to access the data in ECS. You can use `SHOW NAMESPACES IN my_catalog` and `SHOW TABLES IN my_catalog` to fetch the namespaces and tables of the catalog.
+
+The related problems of catalog usage:
+
+1. The [pyspark.sql.SparkSession.catalog](https://spark.apache.org/docs/latest/api/python/reference/api/pyspark.sql.SparkSession.catalog.html#pyspark.sql.SparkSession.catalog) won't access the 3rd-party catalog of Spark, so please use DDL SQL to list all tables and namespaces.
+
+
+### Flink
+
+For example, to use the Dell ECS catalog with Flink 1.14, you should create a Flink environment like:
+
+```sh
+wget https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-flink-runtime-1.14/0.14.0/iceberg-flink-runtime-1.14-0.14.0.jar
+wget https://repo1.maven.org/maven2/com/emc/ecs/object-client-bundle/3.3.2/object-client-bundle-3.3.2.jar
+
+sql-client.sh embedded -j iceberg-flink-runtime-1.14-0.14.0.jar -j object-client-bundle-3.3.2.jar shell
+```
+
+```python
+import requests
+from pyflink.datastream import StreamExecutionEnvironment
+from pyflink.table import StreamTableEnvironment
+
+# Set your workspace
+work_space = "/"
+
+# Download libraries
+maven_url="https://repo1.maven.org/maven2"
+with open(f"{work_space}/iceberg-flink-runtime-1.14-0.14.0.jar", "wb") as f:
+  f.write(requests.get(f"{maven_url}/org/apache/iceberg/iceberg-flink-runtime-1.14/0.14.0/iceberg-flink-runtime-1.14-0.14.0.jar").content)
+with open(f"{work_space}/object-client-bundle-3.3.2.jar", "wb") as f:
+  f.write(requests.get(f"{maven_url}/com/emc/ecs/object-client-bundle/3.3.2/object-client-bundle-3.3.2.jar").content)
+
+jars = ["iceberg-flink-runtime-1.14-0.14.0.jar", "object-client-bundle-3.3.2.jar"]
+pipeline_jars = [f"file://{work_space}/{jar}" for jar in jars]
+
+# Setup Flink session
+env = StreamExecutionEnvironment.get_execution_environment()
+env.add_jars(*pipeline_jars)
+t_env = StreamTableEnvironment.create(env)
+```
+
+Then, use Flink SQL to create a catalog named `my_catalog`:
+
+```SQL
+CREATE CATALOG my_catalog WITH (
+    'type'='iceberg',
+    'warehouse' = 'ecs://bucket-a/namespace-a',
+    'catalog-impl'='org.apache.iceberg.dell.ecs.EcsCatalog',
+    'ecs.s3.endpoint' = 'http://10.x.x.x:9020',
+    'ecs.s3.access-key-id' = 'test',
+    'ecs.s3.secret-access-key' = 'xxxxxxxxxxxxxxxx')
+```
+
+Then, `USE CATALOG my_catalog`, `SHOW DATABASES`, and `SHOW TABLES` to fetch the namespaces and tables of the catalog.
+
+### Limitations
+
+When you use the catalog with Dell ECS only, you should care about these limitations:

Review Comment:
   Is the DELL ECS catalog the expected catalog that you Dell ECS guys recommend to ?  In real cases,  I see people will prefer to use HiveCatalog or other centralized metastore to collect all the table so that they can track table transforms in the global view.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] wang-x-xia commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
wang-x-xia commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r896418116


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:
+
+| Name                     | Description            |
+| ------------------------ | ---------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | Username               |
+| ecs.s3.secret-access-key | S3 Secret Key          |
+
+As for the catalog, you should provide a warehouse location where will store all data and metadata later.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last / will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Dependencies of runtime
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell ECS still recommends using the latest version of the client.
+
+### Spark
+
+For example, to use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=ecs://bucket-a/namespace-a \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.dell.ecs.EcsCatalog \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.endpoint=http://10.x.x.x:9020 \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.access-key-id=test \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.secret-access-key=xxxxxxxxxxxxxxxx
+```
+
+```python
+from pyspark.sql import SparkSession
+
+spark = SparkSession.builder\
+    .config("spark.jars.packages", "org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2")\
+    .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")\
+    .config("spark.sql.catalog.my_catalog", "org.apache.iceberg.spark.SparkCatalog")\
+    .config("spark.sql.catalog.my_catalog.warehouse", "ecs://bucket-a/namespace-a")\
+    .config("spark.sql.catalog.my_catalog.catalog-impl", "org.apache.iceberg.dell.ecs.EcsCatalog")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.endpoint", "http://10.x.x.x:9020")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.access-key-id", "test")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.secret-access-key", "xxxxxxxxxxxxxxxx")\
+    .getOrCreate()
+```
+
+Then, use `my_catalog` to access the data in ECS. You can use `SHOW NAMESPACES IN my_catalog` and `SHOW TABLES IN my_catalog` to fetch the namespaces and tables of the catalog.
+
+The related problems of catalog usage:
+
+1. The [pyspark.sql.SparkSession.catalog](https://spark.apache.org/docs/latest/api/python/reference/api/pyspark.sql.SparkSession.catalog.html#pyspark.sql.SparkSession.catalog) won't access the 3rd-party catalog of Spark, so please use DDL SQL to list all tables and namespaces.
+
+
+### Flink
+
+For example, to use the Dell ECS catalog with Flink 1.14, you should create a Flink environment like:
+
+```sh
+wget https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-flink-runtime-1.14/0.14.0/iceberg-flink-runtime-1.14-0.14.0.jar
+wget https://repo1.maven.org/maven2/com/emc/ecs/object-client-bundle/3.3.2/object-client-bundle-3.3.2.jar
+
+sql-client.sh embedded -j iceberg-flink-runtime-1.14-0.14.0.jar -j object-client-bundle-3.3.2.jar shell
+```
+
+```python
+import requests
+from pyflink.datastream import StreamExecutionEnvironment
+from pyflink.table import StreamTableEnvironment
+
+# Set your workspace
+work_space = "/"
+
+# Download libraries
+maven_url="https://repo1.maven.org/maven2"
+with open(f"{work_space}/iceberg-flink-runtime-1.14-0.14.0.jar", "wb") as f:
+  f.write(requests.get(f"{maven_url}/org/apache/iceberg/iceberg-flink-runtime-1.14/0.14.0/iceberg-flink-runtime-1.14-0.14.0.jar").content)
+with open(f"{work_space}/object-client-bundle-3.3.2.jar", "wb") as f:
+  f.write(requests.get(f"{maven_url}/com/emc/ecs/object-client-bundle/3.3.2/object-client-bundle-3.3.2.jar").content)

Review Comment:
   Done.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] wang-x-xia commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
wang-x-xia commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r896420805


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:
+
+| Name                     | Description            |
+| ------------------------ | ---------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | Username               |
+| ecs.s3.secret-access-key | S3 Secret Key          |
+
+As for the catalog, you should provide a warehouse location where will store all data and metadata later.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last / will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Dependencies of runtime
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell ECS still recommends using the latest version of the client.
+
+### Spark
+
+For example, to use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2 \

Review Comment:
   I've removed the example of `spark-sql` and Flink `sql-client.sh` to avoid redundancy.



##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:

Review Comment:
   Done.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] wang-x-xia commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
wang-x-xia commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r896421242


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:
+
+| Name                     | Description            |
+| ------------------------ | ---------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | Username               |

Review Comment:
   Done.



##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:
+
+| Name                     | Description            |
+| ------------------------ | ---------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | Username               |
+| ecs.s3.secret-access-key | S3 Secret Key          |
+
+As for the catalog, you should provide a warehouse location where will store all data and metadata later.

Review Comment:
   Done.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r933580121


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,132 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integration
+
+## Dell ECS Integration
+
+Iceberg can be used with Dell's Enterprise Object Storage (ECS) by using the ECS catalog since 0.14.0.
+
+Dell ECS has many features that make Iceberg a highly compatible table format, such as append operations for file writers and content addressable storage (CAS) for table commits. See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When using Dell ECS with Iceberg, these configuration parameters are required:
+
+| Name                     | Description             |
+| ------------------------ | ----------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | ECS Username            |
+| ecs.s3.secret-access-key | S3 Secret Key           |
+
+An ECS catalog requires that you configure a warehouse location where all data and metadata will be created.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last `/` will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.

Review Comment:
   Can you specifically state all of the required properties and where to set them? I think these are catalog properties.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r933580883


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,132 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integration
+
+## Dell ECS Integration
+
+Iceberg can be used with Dell's Enterprise Object Storage (ECS) by using the ECS catalog since 0.14.0.
+
+Dell ECS has many features that make Iceberg a highly compatible table format, such as append operations for file writers and content addressable storage (CAS) for table commits. See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When using Dell ECS with Iceberg, these configuration parameters are required:
+
+| Name                     | Description             |
+| ------------------------ | ----------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | ECS Username            |
+| ecs.s3.secret-access-key | S3 Secret Key           |
+
+An ECS catalog requires that you configure a warehouse location where all data and metadata will be created.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last `/` will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Runtime dependencies
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell EMC still recommends using the latest version of the client.
+
+### Spark
+
+For example, to use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```python
+from pyspark.sql import SparkSession
+
+spark = SparkSession.builder
+    .config("spark.jars.packages", "org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2")

Review Comment:
   Isn't the `iceberg-dell` Jar required as well?



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] wang-x-xia commented on pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
wang-x-xia commented on PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#issuecomment-1233665207

   @rdblue 
   Could you please confirm whether the next release is 0.14.1 or 0.15?
   If the next release is 0.14.1, I'll change the Jar version to 0.14.1.
   And if the document has any other problems, I can also fix them! 😂


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] wang-x-xia commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
wang-x-xia commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r962684398


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,132 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integration
+
+## Dell ECS Integration
+
+Iceberg can be used with Dell's Enterprise Object Storage (ECS) by using the ECS catalog since 0.14.0.
+
+Dell ECS has many features that make Iceberg a highly compatible table format, such as append operations for file writers and content addressable storage (CAS) for table commits. See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When using Dell ECS with Iceberg, these configuration parameters are required:
+
+| Name                     | Description             |
+| ------------------------ | ----------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | ECS Username            |
+| ecs.s3.secret-access-key | S3 Secret Key           |
+
+An ECS catalog requires that you configure a warehouse location where all data and metadata will be created.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last `/` will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Runtime dependencies
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell EMC still recommends using the latest version of the client.
+
+### Spark
+
+For example, to use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```python

Review Comment:
   Now the scripts are bash-script.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
openinx commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r962864368


##########
docs/dell.md:
##########
@@ -0,0 +1,135 @@
+---
+title: "Dell"
+url: dell
+menu:
+    main:
+        parent: Integrations
+        weight: 0
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integration
+
+## Dell ECS Integration
+
+Iceberg can be used with Dell's Enterprise Object Storage (ECS) by using the ECS catalog since 0.15.0.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Parameters
+
+When using Dell ECS with Iceberg, these configuration parameters are required:
+
+| Name                     | Description                       |
+| ------------------------ | --------------------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint           |
+| ecs.s3.access-key-id     | ECS Username                      |
+| ecs.s3.secret-access-key | S3 Secret Key                     |
+| warehouse                | The location of data and metadata |
+
+The warehouse should use the following formats:
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/            | Use the whole bucket as the data. The last `/` is ignored.      |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. You should pick the correct version.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell EMC still recommends using the latest version of the client.
+
+### Spark
+
+To use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```bash
+ICEBERG_VERSION=0.15.0
+SPARK_VERSION=3.2_2.12
+ECS_CLIENT_VERSION=3.3.2
+
+DEPENDENCIES="org.apache.iceberg:iceberg-spark-runtime-${SPARK_VERSION}:${ICEBERG_VERSION},\
+org.apache.iceberg:iceberg-dell:${ICEBERG_VERSION},\
+com.emc.ecs:object-client-bundle:${ECS_CLIENT_VERSION}"
+
+spark-sql --packages ${DEPENDENCIES} \
+    --conf spark.sql.extensions=org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=ecs://bucket-a/namespace-a \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.dell.ecs.EcsCatalog \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.endpoint=http://10.x.x.x:9020 \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.access-key-id=<Your-ecs-s3-access-key> \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.secret-access-key=<Your-ecs-s3-secret-access-key>
+```
+
+Then, use `my_catalog` to access the data in ECS. You can use `SHOW NAMESPACES IN my_catalog` and `SHOW TABLES IN my_catalog` to fetch the namespaces and tables of the catalog.
+
+The related problems of catalog usage:
+
+1. The `pyspark.sql.SparkSession.catalog` won't access the 3rd-party catalog of Spark, so please use DDL SQL to list all tables and namespaces.

Review Comment:
   Only the `pyspark.sql.SparkSession.catalog` has this problems ?  How about the java & scala ?



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
openinx commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r962446368


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,132 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integration
+
+## Dell ECS Integration
+
+Iceberg can be used with Dell's Enterprise Object Storage (ECS) by using the ECS catalog since 0.14.0.
+
+Dell ECS has many features that make Iceberg a highly compatible table format, such as append operations for file writers and content addressable storage (CAS) for table commits. See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When using Dell ECS with Iceberg, these configuration parameters are required:
+
+| Name                     | Description             |
+| ------------------------ | ----------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | ECS Username            |
+| ecs.s3.secret-access-key | S3 Secret Key           |
+
+An ECS catalog requires that you configure a warehouse location where all data and metadata will be created.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last `/` will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Runtime dependencies
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell EMC still recommends using the latest version of the client.
+
+### Spark
+
+For example, to use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```python

Review Comment:
   Let's just use java/scalar or bash example for keep consistentence as other pages.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
openinx commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r895284149


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:
+
+| Name                     | Description            |
+| ------------------------ | ---------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | Username               |
+| ecs.s3.secret-access-key | S3 Secret Key          |
+
+As for the catalog, you should provide a warehouse location where will store all data and metadata later.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last / will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Dependencies of runtime
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell ECS still recommends using the latest version of the client.
+
+### Spark
+
+For example, to use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=ecs://bucket-a/namespace-a \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.dell.ecs.EcsCatalog \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.endpoint=http://10.x.x.x:9020 \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.access-key-id=test \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.secret-access-key=xxxxxxxxxxxxxxxx

Review Comment:
   Nit: `xxxxxxxxxxxxxxxx` -> `Your-ecs-s3-secret-access-key`.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
openinx commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r895286318


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:
+
+| Name                     | Description            |
+| ------------------------ | ---------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | Username               |
+| ecs.s3.secret-access-key | S3 Secret Key          |
+
+As for the catalog, you should provide a warehouse location where will store all data and metadata later.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last / will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Dependencies of runtime
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell ECS still recommends using the latest version of the client.
+
+### Spark
+
+For example, to use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=ecs://bucket-a/namespace-a \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.dell.ecs.EcsCatalog \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.endpoint=http://10.x.x.x:9020 \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.access-key-id=test \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.secret-access-key=xxxxxxxxxxxxxxxx
+```
+
+```python
+from pyspark.sql import SparkSession
+
+spark = SparkSession.builder\
+    .config("spark.jars.packages", "org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2")\
+    .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")\
+    .config("spark.sql.catalog.my_catalog", "org.apache.iceberg.spark.SparkCatalog")\
+    .config("spark.sql.catalog.my_catalog.warehouse", "ecs://bucket-a/namespace-a")\
+    .config("spark.sql.catalog.my_catalog.catalog-impl", "org.apache.iceberg.dell.ecs.EcsCatalog")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.endpoint", "http://10.x.x.x:9020")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.access-key-id", "test")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.secret-access-key", "xxxxxxxxxxxxxxxx")\
+    .getOrCreate()
+```
+
+Then, use `my_catalog` to access the data in ECS. You can use `SHOW NAMESPACES IN my_catalog` and `SHOW TABLES IN my_catalog` to fetch the namespaces and tables of the catalog.
+
+The related problems of catalog usage:
+
+1. The [pyspark.sql.SparkSession.catalog](https://spark.apache.org/docs/latest/api/python/reference/api/pyspark.sql.SparkSession.catalog.html#pyspark.sql.SparkSession.catalog) won't access the 3rd-party catalog of Spark, so please use DDL SQL to list all tables and namespaces.
+
+
+### Flink
+
+For example, to use the Dell ECS catalog with Flink 1.14, you should create a Flink environment like:
+
+```sh
+wget https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-flink-runtime-1.14/0.14.0/iceberg-flink-runtime-1.14-0.14.0.jar

Review Comment:
   Since we still don't release the iceberg 0.14.0, so we can not verify & experience this Dell ECS integration work.  It's possible to `wget` the latest snapshot artifact from the maven repo ?



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] wang-x-xia commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
wang-x-xia commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r896419191


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:
+
+| Name                     | Description            |
+| ------------------------ | ---------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | Username               |
+| ecs.s3.secret-access-key | S3 Secret Key          |
+
+As for the catalog, you should provide a warehouse location where will store all data and metadata later.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last / will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Dependencies of runtime
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell ECS still recommends using the latest version of the client.
+
+### Spark
+
+For example, to use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=ecs://bucket-a/namespace-a \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.dell.ecs.EcsCatalog \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.endpoint=http://10.x.x.x:9020 \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.access-key-id=test \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.secret-access-key=xxxxxxxxxxxxxxxx
+```
+
+```python
+from pyspark.sql import SparkSession
+
+spark = SparkSession.builder\
+    .config("spark.jars.packages", "org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2")\
+    .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")\
+    .config("spark.sql.catalog.my_catalog", "org.apache.iceberg.spark.SparkCatalog")\
+    .config("spark.sql.catalog.my_catalog.warehouse", "ecs://bucket-a/namespace-a")\
+    .config("spark.sql.catalog.my_catalog.catalog-impl", "org.apache.iceberg.dell.ecs.EcsCatalog")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.endpoint", "http://10.x.x.x:9020")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.access-key-id", "test")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.secret-access-key", "xxxxxxxxxxxxxxxx")\
+    .getOrCreate()
+```
+
+Then, use `my_catalog` to access the data in ECS. You can use `SHOW NAMESPACES IN my_catalog` and `SHOW TABLES IN my_catalog` to fetch the namespaces and tables of the catalog.
+
+The related problems of catalog usage:
+
+1. The [pyspark.sql.SparkSession.catalog](https://spark.apache.org/docs/latest/api/python/reference/api/pyspark.sql.SparkSession.catalog.html#pyspark.sql.SparkSession.catalog) won't access the 3rd-party catalog of Spark, so please use DDL SQL to list all tables and namespaces.
+
+
+### Flink
+
+For example, to use the Dell ECS catalog with Flink 1.14, you should create a Flink environment like:
+
+```sh
+wget https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-flink-runtime-1.14/0.14.0/iceberg-flink-runtime-1.14-0.14.0.jar
+wget https://repo1.maven.org/maven2/com/emc/ecs/object-client-bundle/3.3.2/object-client-bundle-3.3.2.jar
+
+sql-client.sh embedded -j iceberg-flink-runtime-1.14-0.14.0.jar -j object-client-bundle-3.3.2.jar shell
+```
+
+```python
+import requests
+from pyflink.datastream import StreamExecutionEnvironment
+from pyflink.table import StreamTableEnvironment
+
+# Set your workspace
+work_space = "/"
+
+# Download libraries
+maven_url="https://repo1.maven.org/maven2"
+with open(f"{work_space}/iceberg-flink-runtime-1.14-0.14.0.jar", "wb") as f:

Review Comment:
   The `work_space=/` is an example. I replaced it.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] samredai commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
samredai commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r897410006


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations

Review Comment:
   On second thought, I think integration makes sense here. I think I'm getting caught up in the fact that this is an implementation of the catalog interface, but you're right--integration is more consistent with how we've been using this elsewhere and overall makes sense (this integrates Iceberg and ECS).



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] wang-x-xia commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
wang-x-xia commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r896404797


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:
+
+| Name                     | Description            |
+| ------------------------ | ---------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | Username               |
+| ecs.s3.secret-access-key | S3 Secret Key          |
+
+As for the catalog, you should provide a warehouse location where will store all data and metadata later.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last / will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Dependencies of runtime
+
+The Iceberg `runtime` jar supports different versions of Spark and Flink. If the version was not matched in the example, please check the related document of Spark and Flink.
+
+Even though the [Dell ECS client](https://github.com/EMCECS/ecs-object-client-java) jar is backward compatible, Dell ECS still recommends using the latest version of the client.
+
+### Spark
+
+For example, to use the Dell ECS catalog with Spark 3.2.1, you should create a Spark session like:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=ecs://bucket-a/namespace-a \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.dell.ecs.EcsCatalog \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.endpoint=http://10.x.x.x:9020 \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.access-key-id=test \
+    --conf spark.sql.catalog.my_catalog.ecs.s3.secret-access-key=xxxxxxxxxxxxxxxx
+```
+
+```python
+from pyspark.sql import SparkSession
+
+spark = SparkSession.builder\
+    .config("spark.jars.packages", "org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.14.0,com.emc.ecs:object-client-bundle:3.3.2")\
+    .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")\
+    .config("spark.sql.catalog.my_catalog", "org.apache.iceberg.spark.SparkCatalog")\
+    .config("spark.sql.catalog.my_catalog.warehouse", "ecs://bucket-a/namespace-a")\
+    .config("spark.sql.catalog.my_catalog.catalog-impl", "org.apache.iceberg.dell.ecs.EcsCatalog")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.endpoint", "http://10.x.x.x:9020")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.access-key-id", "test")\
+    .config("spark.sql.catalog.my_catalog.ecs.s3.secret-access-key", "xxxxxxxxxxxxxxxx")\
+    .getOrCreate()
+```
+
+Then, use `my_catalog` to access the data in ECS. You can use `SHOW NAMESPACES IN my_catalog` and `SHOW TABLES IN my_catalog` to fetch the namespaces and tables of the catalog.
+
+The related problems of catalog usage:
+
+1. The [pyspark.sql.SparkSession.catalog](https://spark.apache.org/docs/latest/api/python/reference/api/pyspark.sql.SparkSession.catalog.html#pyspark.sql.SparkSession.catalog) won't access the 3rd-party catalog of Spark, so please use DDL SQL to list all tables and namespaces.
+
+
+### Flink
+
+For example, to use the Dell ECS catalog with Flink 1.14, you should create a Flink environment like:
+
+```sh
+wget https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-flink-runtime-1.14/0.14.0/iceberg-flink-runtime-1.14-0.14.0.jar

Review Comment:
   Because the dependencies are also added in this commit. I don't know which version should I use?



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] wang-x-xia commented on a diff in pull request #4993: Dell: Add document.

Posted by GitBox <gi...@apache.org>.
wang-x-xia commented on code in PR #4993:
URL: https://github.com/apache/iceberg/pull/4993#discussion_r896421664


##########
docs/integrations/dell.md:
##########
@@ -0,0 +1,150 @@
+---
+title: "Dell"
+url: dell
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+
+# Iceberg Dell Integrations
+
+## Dell ECS Integrations
+
+This session will show you how to use Iceberg with Dell ECS. Dell ECS provides several features that are more appropriate for Iceberg:
+
+1. Append operation for file writer.
+2. CAS operation for Table commit.
+
+See [Dell ECS](https://www.dell.com/en-us/dt/storage/ecs/index.htm) for more information on Dell ECS.
+
+### Connection parameters
+
+When you try to connect Dell ECS with Iceberg, these connection parameters should be prepared:
+
+| Name                     | Description            |
+| ------------------------ | ---------------------- |
+| ecs.s3.endpoint          | ECS S3 service endpoint |
+| ecs.s3.access-key-id     | Username               |
+| ecs.s3.secret-access-key | S3 Secret Key          |
+
+As for the catalog, you should provide a warehouse location where will store all data and metadata later.
+
+| Example                    | Description                                                     |
+| -------------------------- | --------------------------------------------------------------- |
+| ecs://bucket-a             | Use the whole bucket as the data                                |
+| ecs://bucket-a/namespace-a | Use a prefix to access the data only in this specific namespace |
+
+When you provide the `warehouse`, the last / will be ignored. The `ecs://bucket-a` is same with `ecs://bucket-a/`.
+
+### Dependencies of runtime

Review Comment:
   Done.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org