morningman commented on code in PR #16429: URL: https://github.com/apache/doris/pull/16429#discussion_r1097446752
########## docs/en/docs/lakehouse/multi-catalog/iceberg.md: ########## @@ -52,6 +54,50 @@ CREATE CATALOG iceberg PROPERTIES ( ); ``` +### Iceberg Native Catalog + +<version since="dev"> + +Access metadata with the iceberg API. The Hive, REST, Glue and other services can serve as the iceberg catalog. + Review Comment: missing the end tag of `<version>` Add here: `</version>` ########## docs/en/docs/lakehouse/multi-catalog/iceberg.md: ########## @@ -52,6 +54,50 @@ CREATE CATALOG iceberg PROPERTIES ( ); ``` +### Iceberg Native Catalog + +<version since="dev"> + +Access metadata with the iceberg API. The Hive, REST, Glue and other services can serve as the iceberg catalog. + +- Using Iceberg Hive Catalog + +```sql +CREATE CATALOG iceberg PROPERTIES ( + 'type'='iceberg', + 'iceberg.catalog.type'='hms', + 'hive.metastore.uris' = 'thrift://172.21.0.1:7004', + 'hadoop.username' = 'hive', + 'dfs.nameservices'='your-nameservice', + 'dfs.ha.namenodes.your-nameservice'='nn1,nn2', + 'dfs.namenode.rpc-address.your-nameservice.nn1'='172.21.0.2:4007', + 'dfs.namenode.rpc-address.your-nameservice.nn2'='172.21.0.3:4007', + 'dfs.client.failover.proxy.provider.your-nameservice'='org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider' +); +``` + +- Using Iceberg REST Catalog + +RESTful service as the server side. Implementing RESTCatalog interface of iceberg to obtain metadata. + +```sql +CREATE CATALOG iceberg PROPERTIES ( + 'type'='iceberg', + 'iceberg.catalog.type'='rest', + 'uri' = 'http://172.21.0.1:8181', +); +``` + +If you want to use S3 storage, the following properties need to be set. + +``` +"AWS_ACCESS_KEY" = "username" Review Comment: ```suggestion "AWS_ACCESS_KEY" = "ak" ``` ########## docs/zh-CN/docs/lakehouse/multi-catalog/iceberg.md: ########## @@ -54,6 +54,8 @@ CREATE CATALOG iceberg PROPERTIES ( ### 基于Iceberg API创建Catalog +<version since="dev"> Review Comment: ditto ########## fe/fe-core/src/main/java/org/apache/doris/catalog/S3Resource.java: ########## @@ -222,41 +226,36 @@ protected void getProcNodeData(BaseProcResult result) { public static Map<String, String> getS3HadoopProperties(Map<String, String> properties) { Map<String, String> s3Properties = Maps.newHashMap(); if (properties.containsKey(S3_ACCESS_KEY)) { - s3Properties.put("fs.s3a.access.key", properties.get(S3_ACCESS_KEY)); + s3Properties.put(Constants.ACCESS_KEY, properties.get(S3_ACCESS_KEY)); } if (properties.containsKey(S3Resource.S3_SECRET_KEY)) { - s3Properties.put("fs.s3a.secret.key", properties.get(S3_SECRET_KEY)); + s3Properties.put(Constants.SECRET_KEY, properties.get(S3_SECRET_KEY)); } if (properties.containsKey(S3Resource.S3_ENDPOINT)) { - s3Properties.put("fs.s3a.endpoint", properties.get(S3_ENDPOINT)); + s3Properties.put(Constants.ENDPOINT, properties.get(S3_ENDPOINT)); } if (properties.containsKey(S3Resource.S3_REGION)) { - s3Properties.put("fs.s3a.endpoint.region", properties.get(S3_REGION)); + s3Properties.put(Constants.AWS_REGION, properties.get(S3_REGION)); } if (properties.containsKey(S3Resource.S3_MAX_CONNECTIONS)) { - s3Properties.put("fs.s3a.connection.maximum", properties.get(S3_MAX_CONNECTIONS)); + s3Properties.put(Constants.MAXIMUM_CONNECTIONS, properties.get(S3_MAX_CONNECTIONS)); } if (properties.containsKey(S3Resource.S3_REQUEST_TIMEOUT_MS)) { - s3Properties.put("fs.s3a.connection.request.timeout", properties.get(S3_REQUEST_TIMEOUT_MS)); + s3Properties.put(Constants.REQUEST_TIMEOUT, properties.get(S3_REQUEST_TIMEOUT_MS)); } if (properties.containsKey(S3Resource.S3_CONNECTION_TIMEOUT_MS)) { - s3Properties.put("fs.s3a.connection.timeout", properties.get(S3_CONNECTION_TIMEOUT_MS)); + s3Properties.put(Constants.SOCKET_TIMEOUT, properties.get(S3_CONNECTION_TIMEOUT_MS)); } + s3Properties.put(Constants.MAX_ERROR_RETRIES, "2"); s3Properties.put("fs.s3.impl.disable.cache", "true"); - s3Properties.put("fs.s3.impl", "org.apache.hadoop.fs.s3a.S3AFileSystem"); - s3Properties.put("fs.s3a.attempts.maximum", "2"); + s3Properties.put("fs.s3.impl", S3AFileSystem.class.getName()); - if (Boolean.valueOf(properties.getOrDefault(S3Resource.USE_PATH_STYLE, "false")).booleanValue()) { - s3Properties.put("fs.s3a.path.style.access", "true"); - } else { - s3Properties.put("fs.s3a.path.style.access", "false"); - } + s3Properties.put(Constants.PATH_STYLE_ACCESS, properties.getOrDefault(S3Resource.USE_PATH_STYLE, "false")); + s3Properties.put(Constants.AWS_CREDENTIALS_PROVIDER, properties.getOrDefault(S3Resource.S3_CREDENTIALS_PROVIDER, + TemporaryAWSCredentialsProvider.class.getName())); Review Comment: default should be `com.amazonaws.glue.catalog.credentials.ConfigurationAWSCredentialsProviderFactory` ########## fe/fe-core/src/main/java/org/apache/doris/catalog/S3Resource.java: ########## @@ -222,41 +226,36 @@ protected void getProcNodeData(BaseProcResult result) { public static Map<String, String> getS3HadoopProperties(Map<String, String> properties) { Map<String, String> s3Properties = Maps.newHashMap(); if (properties.containsKey(S3_ACCESS_KEY)) { - s3Properties.put("fs.s3a.access.key", properties.get(S3_ACCESS_KEY)); + s3Properties.put(Constants.ACCESS_KEY, properties.get(S3_ACCESS_KEY)); } if (properties.containsKey(S3Resource.S3_SECRET_KEY)) { - s3Properties.put("fs.s3a.secret.key", properties.get(S3_SECRET_KEY)); + s3Properties.put(Constants.SECRET_KEY, properties.get(S3_SECRET_KEY)); } if (properties.containsKey(S3Resource.S3_ENDPOINT)) { - s3Properties.put("fs.s3a.endpoint", properties.get(S3_ENDPOINT)); + s3Properties.put(Constants.ENDPOINT, properties.get(S3_ENDPOINT)); } if (properties.containsKey(S3Resource.S3_REGION)) { - s3Properties.put("fs.s3a.endpoint.region", properties.get(S3_REGION)); + s3Properties.put(Constants.AWS_REGION, properties.get(S3_REGION)); } if (properties.containsKey(S3Resource.S3_MAX_CONNECTIONS)) { - s3Properties.put("fs.s3a.connection.maximum", properties.get(S3_MAX_CONNECTIONS)); + s3Properties.put(Constants.MAXIMUM_CONNECTIONS, properties.get(S3_MAX_CONNECTIONS)); } if (properties.containsKey(S3Resource.S3_REQUEST_TIMEOUT_MS)) { - s3Properties.put("fs.s3a.connection.request.timeout", properties.get(S3_REQUEST_TIMEOUT_MS)); + s3Properties.put(Constants.REQUEST_TIMEOUT, properties.get(S3_REQUEST_TIMEOUT_MS)); } if (properties.containsKey(S3Resource.S3_CONNECTION_TIMEOUT_MS)) { - s3Properties.put("fs.s3a.connection.timeout", properties.get(S3_CONNECTION_TIMEOUT_MS)); + s3Properties.put(Constants.SOCKET_TIMEOUT, properties.get(S3_CONNECTION_TIMEOUT_MS)); } + s3Properties.put(Constants.MAX_ERROR_RETRIES, "2"); s3Properties.put("fs.s3.impl.disable.cache", "true"); - s3Properties.put("fs.s3.impl", "org.apache.hadoop.fs.s3a.S3AFileSystem"); - s3Properties.put("fs.s3a.attempts.maximum", "2"); + s3Properties.put("fs.s3.impl", S3AFileSystem.class.getName()); - if (Boolean.valueOf(properties.getOrDefault(S3Resource.USE_PATH_STYLE, "false")).booleanValue()) { - s3Properties.put("fs.s3a.path.style.access", "true"); - } else { - s3Properties.put("fs.s3a.path.style.access", "false"); - } + s3Properties.put(Constants.PATH_STYLE_ACCESS, properties.getOrDefault(S3Resource.USE_PATH_STYLE, "false")); + s3Properties.put(Constants.AWS_CREDENTIALS_PROVIDER, properties.getOrDefault(S3Resource.S3_CREDENTIALS_PROVIDER, + TemporaryAWSCredentialsProvider.class.getName())); if (properties.containsKey(S3Resource.S3_TOKEN)) { - s3Properties.put("fs.s3a.session.token", properties.get(S3_TOKEN)); - s3Properties.put("fs.s3a.aws.credentials.provider", Review Comment: Add this back ########## docs/en/docs/lakehouse/multi-catalog/iceberg.md: ########## @@ -52,6 +54,50 @@ CREATE CATALOG iceberg PROPERTIES ( ); ``` +### Iceberg Native Catalog + +<version since="dev"> + +Access metadata with the iceberg API. The Hive, REST, Glue and other services can serve as the iceberg catalog. + +- Using Iceberg Hive Catalog + +```sql +CREATE CATALOG iceberg PROPERTIES ( + 'type'='iceberg', + 'iceberg.catalog.type'='hms', + 'hive.metastore.uris' = 'thrift://172.21.0.1:7004', + 'hadoop.username' = 'hive', + 'dfs.nameservices'='your-nameservice', + 'dfs.ha.namenodes.your-nameservice'='nn1,nn2', + 'dfs.namenode.rpc-address.your-nameservice.nn1'='172.21.0.2:4007', + 'dfs.namenode.rpc-address.your-nameservice.nn2'='172.21.0.3:4007', + 'dfs.client.failover.proxy.provider.your-nameservice'='org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider' +); +``` + +- Using Iceberg REST Catalog + +RESTful service as the server side. Implementing RESTCatalog interface of iceberg to obtain metadata. + +```sql +CREATE CATALOG iceberg PROPERTIES ( + 'type'='iceberg', + 'iceberg.catalog.type'='rest', + 'uri' = 'http://172.21.0.1:8181', +); +``` + +If you want to use S3 storage, the following properties need to be set. + +``` +"AWS_ACCESS_KEY" = "username" +"AWS_SECRET_KEY" = "password" Review Comment: ```suggestion "AWS_SECRET_KEY" = "sk" ``` ########## docs/en/docs/lakehouse/multi-catalog/iceberg.md: ########## @@ -52,6 +54,50 @@ CREATE CATALOG iceberg PROPERTIES ( ); ``` +### Iceberg Native Catalog + +<version since="dev"> + +Access metadata with the iceberg API. The Hive, REST, Glue and other services can serve as the iceberg catalog. + +- Using Iceberg Hive Catalog + +```sql +CREATE CATALOG iceberg PROPERTIES ( + 'type'='iceberg', + 'iceberg.catalog.type'='hms', + 'hive.metastore.uris' = 'thrift://172.21.0.1:7004', + 'hadoop.username' = 'hive', + 'dfs.nameservices'='your-nameservice', + 'dfs.ha.namenodes.your-nameservice'='nn1,nn2', + 'dfs.namenode.rpc-address.your-nameservice.nn1'='172.21.0.2:4007', + 'dfs.namenode.rpc-address.your-nameservice.nn2'='172.21.0.3:4007', + 'dfs.client.failover.proxy.provider.your-nameservice'='org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider' +); +``` + +- Using Iceberg REST Catalog + +RESTful service as the server side. Implementing RESTCatalog interface of iceberg to obtain metadata. + +```sql +CREATE CATALOG iceberg PROPERTIES ( + 'type'='iceberg', + 'iceberg.catalog.type'='rest', + 'uri' = 'http://172.21.0.1:8181', +); +``` + +If you want to use S3 storage, the following properties need to be set. + +``` +"AWS_ACCESS_KEY" = "username" +"AWS_SECRET_KEY" = "password" +"AWS_REGION" = "region-name" +"AWS_ENDPOINT" = "http://endpoint-uri" +"AWS_CREDENTIALS_PROVIDER" = "provider-class-name" // Optional. The default credentials class is based on BasicAWSCredentials. Review Comment: The default should be `com.amazonaws.glue.catalog.credentials.ConfigurationAWSCredentialsProviderFactory` -- 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: commits-unsubscr...@doris.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org