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 2020/10/12 14:07:39 UTC

[GitHub] [iceberg] openinx opened a new pull request #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

openinx opened a new pull request #1586:
URL: https://github.com/apache/iceberg/pull/1586


   This patch will address the comment: https://github.com/apache/iceberg/issues/1437#issuecomment-707112103. 


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

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 change in pull request #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1586:
URL: https://github.com/apache/iceberg/pull/1586#discussion_r509453461



##########
File path: flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java
##########
@@ -34,23 +40,53 @@
   private static TestHiveMetastore metastore = null;
   protected static HiveConf hiveConf = null;
   protected static HiveCatalog catalog = null;
-  protected static ConcurrentMap<String, Catalog> flinkCatalogs;
+
+  private volatile TableEnvironment tEnv = null;
 
   @BeforeClass
   public static void startMetastore() {
     FlinkTestBase.metastore = new TestHiveMetastore();
-    metastore.start();
+    metastore.start(15);

Review comment:
       Why did this change? Needing to increase the number of threads used by the Hive MetaStore is a red flag that this may be leaking Hive connections. In cases where all of the connections are handled by a `ClientPool`, we should not need to increase the number of handler threads. This helps us avoid leaks because we catch them in testing rather than at runtime.




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

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 change in pull request #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1586:
URL: https://github.com/apache/iceberg/pull/1586#discussion_r509462458



##########
File path: site/docs/flink.md
##########
@@ -97,7 +97,21 @@ CREATE CATALOG hive_catalog WITH (
   'catalog-type'='hive',
   'uri'='thrift://localhost:9083',
   'clients'='5',
-  'property-version'='1'
+  'property-version'='1',
+  'hive-conf-dir'='/opt/hive/conf'
+);
+```
+
+Alternatively one can instead set just the `warehouse` property (without specifying a Hive configuration directory) to initialize the Hive catalog:

Review comment:
       A small clarification: this implies that the use of `hive-site.xml` and `warehouse` are alternatives, but they aren't. You may want to use `warehouse` from the Hive site, or you may want to use other settings from `hive-site.xml` with a custom warehouse location. I think this should document the two options separately.
   
   For documenting options, I think we will want to add a table because copying the SQL block each time creates a long doc. I would have one simple example for each catalog type with the options we recommend for it (Hive: uri, warehouse, and property-version; Hadoop: warehouse, property-version) and a table of other options for the Hive catalog to document `clients` and `hive-conf-dir`.




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

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] JingsongLi commented on a change in pull request #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1586:
URL: https://github.com/apache/iceberg/pull/1586#discussion_r509852820



##########
File path: site/docs/flink.md
##########
@@ -97,15 +97,20 @@ CREATE CATALOG hive_catalog WITH (
   'catalog-type'='hive',
   'uri'='thrift://localhost:9083',
   'clients'='5',
-  'property-version'='1'
+  'property-version'='1',
+  'warehouse'='hdfs://nn:8020/warehouse/path'
 );
 ```
 
+We must set `hive-conf-dir` property to get the custom warehouse location by specifying the hive configuration directory where to load the `hive-site.xml`, if we don't provide the `warehouse` property explicitly.

Review comment:
       The document doesn't look clear enough. Actually, there are three options:
   -Configure URI and warehouse
   -Configure hive-conf-dir
   -The resource of class path contains hive conf
   
   Users can choose one of 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.

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 merged pull request #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

Posted by GitBox <gi...@apache.org>.
rdblue merged pull request #1586:
URL: https://github.com/apache/iceberg/pull/1586


   


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

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 removed a comment on pull request #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

Posted by GitBox <gi...@apache.org>.
openinx removed a comment on pull request #1586:
URL: https://github.com/apache/iceberg/pull/1586#issuecomment-709056128


   Run several times locally, never failed. Let's re-run travis test.


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

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] JingsongLi commented on a change in pull request #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1586:
URL: https://github.com/apache/iceberg/pull/1586#discussion_r509850848



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java
##########
@@ -71,12 +72,16 @@ protected CatalogLoader createCatalogLoader(String name, Map<String, String> pro
     String catalogType = properties.getOrDefault(ICEBERG_CATALOG_TYPE, "hive");
     switch (catalogType) {
       case "hive":
-        int clientPoolSize = Integer.parseInt(properties.getOrDefault(HIVE_CLIENT_POOL_SIZE, "2"));
+        // The values of properties 'uri', 'warehouse', 'hive-conf-dir' are allowed to be null, in that case it will
+        // fallback to parse those values from hadoop configuration which is loaded from classpath.
         String uri = properties.get(HIVE_URI);
-        return CatalogLoader.hive(name, hadoopConf, uri, clientPoolSize);
+        String warehouse = properties.get(WAREHOUSE_LOCATION);
+        int clientPoolSize = Integer.parseInt(properties.getOrDefault(HIVE_CLIENT_POOL_SIZE, "2"));
+        String hiveConfDir = properties.get(HIVE_CONF_DIR);
+        return CatalogLoader.hive(name, hadoopConf, uri, warehouse, clientPoolSize, hiveConfDir);

Review comment:
       Maybe I missed something. Why not load hive-conf here?




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

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] JingsongLi commented on a change in pull request #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1586:
URL: https://github.com/apache/iceberg/pull/1586#discussion_r509851860



##########
File path: site/docs/flink.md
##########
@@ -97,15 +97,20 @@ CREATE CATALOG hive_catalog WITH (
   'catalog-type'='hive',
   'uri'='thrift://localhost:9083',
   'clients'='5',
-  'property-version'='1'
+  'property-version'='1',

Review comment:
       (maybe it should be discussed elsewhere) we don't need to tell users configure this `property-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.

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 #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

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


   Run several times locally, never failed. Let's re-run travis test.


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

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] JingsongLi commented on a change in pull request #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1586:
URL: https://github.com/apache/iceberg/pull/1586#discussion_r509853494



##########
File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java
##########
@@ -107,7 +107,7 @@ protected StructLikeSet rowSet(String name, Table testTable, String... columns)
     RowType rowType = FlinkSchemaUtil.convert(projected);
     CatalogLoader hiveCatalogLoader = CatalogLoader.hive(catalog.name(),
         hiveConf,
-        hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname),
+        hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname), null,

Review comment:
       NIT: use `hiveConf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)` instead of null.




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

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 change in pull request #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1586:
URL: https://github.com/apache/iceberg/pull/1586#discussion_r511162132



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -72,13 +72,19 @@ public HiveCatalog(Configuration conf) {
   }
 
   public HiveCatalog(String name, String uri, int clientPoolSize, Configuration conf) {
+    this(name, uri, null, clientPoolSize, conf);
+  }
+
+  public HiveCatalog(String name, String uri, String warehouse, int clientPoolSize, Configuration conf) {
     this.name = name;
     this.conf = new Configuration(conf);
     // before building the client pool, overwrite the configuration's URIs if the argument is non-null
     if (uri != null) {
       this.conf.set(HiveConf.ConfVars.METASTOREURIS.varname, uri);
     }
-
+    if (warehouse != null) {
+      this.conf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, warehouse);

Review comment:
       I think handling this the same way that we handle the URI makes sense. We set the URI because we need to pass it back into Hive to connect, which I don't think applies to the warehouse path, but handling both the same way is reasonable.




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

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] JingsongLi commented on a change in pull request #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1586:
URL: https://github.com/apache/iceberg/pull/1586#discussion_r509850848



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java
##########
@@ -71,12 +72,16 @@ protected CatalogLoader createCatalogLoader(String name, Map<String, String> pro
     String catalogType = properties.getOrDefault(ICEBERG_CATALOG_TYPE, "hive");
     switch (catalogType) {
       case "hive":
-        int clientPoolSize = Integer.parseInt(properties.getOrDefault(HIVE_CLIENT_POOL_SIZE, "2"));
+        // The values of properties 'uri', 'warehouse', 'hive-conf-dir' are allowed to be null, in that case it will
+        // fallback to parse those values from hadoop configuration which is loaded from classpath.
         String uri = properties.get(HIVE_URI);
-        return CatalogLoader.hive(name, hadoopConf, uri, clientPoolSize);
+        String warehouse = properties.get(WAREHOUSE_LOCATION);
+        int clientPoolSize = Integer.parseInt(properties.getOrDefault(HIVE_CLIENT_POOL_SIZE, "2"));
+        String hiveConfDir = properties.get(HIVE_CONF_DIR);
+        return CatalogLoader.hive(name, hadoopConf, uri, warehouse, clientPoolSize, hiveConfDir);

Review comment:
       Maybe I missed something. Why not load hive-conf into hadoopConf here?




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

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 closed pull request #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

Posted by GitBox <gi...@apache.org>.
openinx closed pull request #1586:
URL: https://github.com/apache/iceberg/pull/1586


   


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

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] massdosage commented on a change in pull request #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

Posted by GitBox <gi...@apache.org>.
massdosage commented on a change in pull request #1586:
URL: https://github.com/apache/iceberg/pull/1586#discussion_r508822234



##########
File path: site/docs/flink.md
##########
@@ -106,6 +120,8 @@ CREATE CATALOG hive_catalog WITH (
 * `uri`: The Hive metastore's thrift URI. (Required)
 * `clients`: The Hive metastore client pool size, default value is 2. (Optional)
 * `property-version`: Version number to describe the property version. This property can be used for backwards compatibility in case the property format changes. The currently property version is `1`. (Optional)
+* `warehouse`: The hive warehouse location, users would need to specify this path if them don't set the `hive-conf-dir` (The path where we load `hive-site.xml`).
+* `hive-conf-dir`: Directory path to load the `hive-site.xml`.

Review comment:
       ```suggestion
   * `hive-conf-dir`: Path to a directory containing a`hive-site.xml` configuration file which will be used to provide custom Hive configuration values.
   ```




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

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 change in pull request #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1586:
URL: https://github.com/apache/iceberg/pull/1586#discussion_r509864323



##########
File path: site/docs/flink.md
##########
@@ -97,15 +97,20 @@ CREATE CATALOG hive_catalog WITH (
   'catalog-type'='hive',
   'uri'='thrift://localhost:9083',
   'clients'='5',
-  'property-version'='1'
+  'property-version'='1',

Review comment:
       Seems we could remove it from the example,  it's not a necessary property ( The options description has mentioned that). 




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

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 change in pull request #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1586:
URL: https://github.com/apache/iceberg/pull/1586#discussion_r509456781



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -72,13 +72,19 @@ public HiveCatalog(Configuration conf) {
   }
 
   public HiveCatalog(String name, String uri, int clientPoolSize, Configuration conf) {
+    this(name, uri, null, clientPoolSize, conf);
+  }
+
+  public HiveCatalog(String name, String uri, String warehouse, int clientPoolSize, Configuration conf) {
     this.name = name;
     this.conf = new Configuration(conf);
     // before building the client pool, overwrite the configuration's URIs if the argument is non-null
     if (uri != null) {
       this.conf.set(HiveConf.ConfVars.METASTOREURIS.varname, uri);
     }
-
+    if (warehouse != null) {
+      this.conf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, warehouse);
+    }

Review comment:
       Style nit: we typically add an empty line after control flow statements. This removes one and doesn't add one after the new `if`.




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

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 change in pull request #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1586:
URL: https://github.com/apache/iceberg/pull/1586#discussion_r509863370



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java
##########
@@ -71,12 +72,16 @@ protected CatalogLoader createCatalogLoader(String name, Map<String, String> pro
     String catalogType = properties.getOrDefault(ICEBERG_CATALOG_TYPE, "hive");
     switch (catalogType) {
       case "hive":
-        int clientPoolSize = Integer.parseInt(properties.getOrDefault(HIVE_CLIENT_POOL_SIZE, "2"));
+        // The values of properties 'uri', 'warehouse', 'hive-conf-dir' are allowed to be null, in that case it will
+        // fallback to parse those values from hadoop configuration which is loaded from classpath.
         String uri = properties.get(HIVE_URI);
-        return CatalogLoader.hive(name, hadoopConf, uri, clientPoolSize);
+        String warehouse = properties.get(WAREHOUSE_LOCATION);
+        int clientPoolSize = Integer.parseInt(properties.getOrDefault(HIVE_CLIENT_POOL_SIZE, "2"));
+        String hiveConfDir = properties.get(HIVE_CONF_DIR);
+        return CatalogLoader.hive(name, hadoopConf, uri, warehouse, clientPoolSize, hiveConfDir);

Review comment:
       Oh, I almost forgot that the `loadCatalog` would be executed at task manager side for flink.   we should merge hive conf into hadoop conf before initializing the catalog loader, so that we won't miss to load a non-existed path in task manager node.




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

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] JingsongLi commented on a change in pull request #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1586:
URL: https://github.com/apache/iceberg/pull/1586#discussion_r509850427



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -72,13 +72,19 @@ public HiveCatalog(Configuration conf) {
   }
 
   public HiveCatalog(String name, String uri, int clientPoolSize, Configuration conf) {
+    this(name, uri, null, clientPoolSize, conf);
+  }
+
+  public HiveCatalog(String name, String uri, String warehouse, int clientPoolSize, Configuration conf) {
     this.name = name;
     this.conf = new Configuration(conf);
     // before building the client pool, overwrite the configuration's URIs if the argument is non-null
     if (uri != null) {
       this.conf.set(HiveConf.ConfVars.METASTOREURIS.varname, uri);
     }
-
+    if (warehouse != null) {
+      this.conf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, warehouse);

Review comment:
       Yes, It can be handled in the same way as URIs.




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

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 change in pull request #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1586:
URL: https://github.com/apache/iceberg/pull/1586#discussion_r503965409



##########
File path: flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java
##########
@@ -38,8 +40,13 @@ static CatalogLoader hadoop(String name, Configuration hadoopConf, String wareho
     return new HadoopCatalogLoader(name, hadoopConf, warehouseLocation);
   }
 
-  static CatalogLoader hive(String name, Configuration hadoopConf, String uri, int clientPoolSize) {
-    return new HiveCatalogLoader(name, hadoopConf, uri, clientPoolSize);
+  static CatalogLoader hive(String name, Configuration hadoopConf, String uri, String warehouse, int clientPoolSize) {

Review comment:
       The user can choose to set `warehouse` property or load `hive-site.xml` when creating the iceberg catalog.  If use the former,  the warehouse path may be different with the hive metastore, that means it will create files under the user specified directory.  
   
   If use the `hive-site.xml`,  then it should be the same configuration as the metastore. 




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

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] massdosage commented on a change in pull request #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

Posted by GitBox <gi...@apache.org>.
massdosage commented on a change in pull request #1586:
URL: https://github.com/apache/iceberg/pull/1586#discussion_r508821242



##########
File path: site/docs/flink.md
##########
@@ -106,6 +120,8 @@ CREATE CATALOG hive_catalog WITH (
 * `uri`: The Hive metastore's thrift URI. (Required)
 * `clients`: The Hive metastore client pool size, default value is 2. (Optional)
 * `property-version`: Version number to describe the property version. This property can be used for backwards compatibility in case the property format changes. The currently property version is `1`. (Optional)
+* `warehouse`: The hive warehouse location, users would need to specify this path if them don't set the `hive-conf-dir` (The path where we load `hive-site.xml`).

Review comment:
       ```suggestion
   * `warehouse`: The Hive warehouse location, users shouldy specify this path if they don't set the `hive-conf-dir` to a location containing a `hive-site.xml` configuration file.
   ```




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

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 change in pull request #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1586:
URL: https://github.com/apache/iceberg/pull/1586#discussion_r508947022



##########
File path: site/docs/flink.md
##########
@@ -97,7 +97,21 @@ CREATE CATALOG hive_catalog WITH (
   'catalog-type'='hive',
   'uri'='thrift://localhost:9083',
   'clients'='5',
-  'property-version'='1'
+  'property-version'='1',
+  'hive-conf-dir'='/opt/hive/conf'
+);
+```
+
+We could also just specify the `warehouse` property (without specifying any hive configuration path) to initialize the hive catalog: 

Review comment:
       Thanks for the change,  I will accept it. @massdosage  ! 




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

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] massdosage commented on a change in pull request #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

Posted by GitBox <gi...@apache.org>.
massdosage commented on a change in pull request #1586:
URL: https://github.com/apache/iceberg/pull/1586#discussion_r508823744



##########
File path: site/docs/flink.md
##########
@@ -97,7 +97,21 @@ CREATE CATALOG hive_catalog WITH (
   'catalog-type'='hive',
   'uri'='thrift://localhost:9083',
   'clients'='5',
-  'property-version'='1'
+  'property-version'='1',
+  'hive-conf-dir'='/opt/hive/conf'
+);
+```
+
+We could also just specify the `warehouse` property (without specifying any hive configuration path) to initialize the hive catalog: 

Review comment:
       ```suggestion
   Alternatively one can instead set just the `warehouse` property (without specifying a Hive configuration directory) to initialize the Hive 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.

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 #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

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


   Ping @rdblue , Mind to take a look for this when you have time ?


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

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 #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

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


   Thanks, @openinx! Mostly minor comments, except for the number of threads for the Hive MetaStore. We're very careful about changing that value so I'd like to understand your reason for changing it.
   
   I think the implementation here addresses my concerns from the discussion on #1558. I understand not being able to add files to the classpath easily, and I like that this makes it simple to add a config directory. That's a reasonable thing to support, although it would be nice if users didn't need to specify it each time they create a 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.

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 change in pull request #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1586:
URL: https://github.com/apache/iceberg/pull/1586#discussion_r509463904



##########
File path: site/docs/flink.md
##########
@@ -293,4 +309,4 @@ There are some features that we do not yet support in the current flink iceberg
 * Don't support creating iceberg table with computed column.
 * Don't support creating iceberg table with watermark.
 * Don't support adding columns, removing columns, renaming columns, changing columns. [FLINK-19062](https://issues.apache.org/jira/browse/FLINK-19062) is tracking this.
-* Don't support flink read iceberg table in batch or streaming mode. [#1346](https://github.com/apache/iceberg/pull/1346) and [#1293](https://github.com/apache/iceberg/pull/1293) are tracking this. 
\ No newline at end of file
+* Don't support flink read iceberg table in batch or streaming mode. [#1346](https://github.com/apache/iceberg/pull/1346) and [#1293](https://github.com/apache/iceberg/pull/1293) are tracking this.

Review comment:
       I think we support reads in batch mode, right?




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

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 #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

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


   Looks like all the issues have been addressed, so I'll merge this. Thanks @openinx!


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

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 change in pull request #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1586:
URL: https://github.com/apache/iceberg/pull/1586#discussion_r509457751



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -72,13 +72,19 @@ public HiveCatalog(Configuration conf) {
   }
 
   public HiveCatalog(String name, String uri, int clientPoolSize, Configuration conf) {
+    this(name, uri, null, clientPoolSize, conf);
+  }
+
+  public HiveCatalog(String name, String uri, String warehouse, int clientPoolSize, Configuration conf) {
     this.name = name;
     this.conf = new Configuration(conf);
     // before building the client pool, overwrite the configuration's URIs if the argument is non-null
     if (uri != null) {
       this.conf.set(HiveConf.ConfVars.METASTOREURIS.varname, uri);
     }
-
+    if (warehouse != null) {
+      this.conf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, warehouse);

Review comment:
       Why not keep this location as a field in the `HiveCatalog` instead of storing it in `conf`? Then the default logic would only need to be run once.




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

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 change in pull request #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1586:
URL: https://github.com/apache/iceberg/pull/1586#discussion_r509839531



##########
File path: site/docs/flink.md
##########
@@ -97,7 +97,21 @@ CREATE CATALOG hive_catalog WITH (
   'catalog-type'='hive',
   'uri'='thrift://localhost:9083',
   'clients'='5',
-  'property-version'='1'
+  'property-version'='1',
+  'hive-conf-dir'='/opt/hive/conf'
+);
+```
+
+Alternatively one can instead set just the `warehouse` property (without specifying a Hive configuration directory) to initialize the Hive catalog:

Review comment:
       Well, let me make this more clear. 

##########
File path: site/docs/flink.md
##########
@@ -293,4 +309,4 @@ There are some features that we do not yet support in the current flink iceberg
 * Don't support creating iceberg table with computed column.
 * Don't support creating iceberg table with watermark.
 * Don't support adding columns, removing columns, renaming columns, changing columns. [FLINK-19062](https://issues.apache.org/jira/browse/FLINK-19062) is tracking this.
-* Don't support flink read iceberg table in batch or streaming mode. [#1346](https://github.com/apache/iceberg/pull/1346) and [#1293](https://github.com/apache/iceberg/pull/1293) are tracking this. 
\ No newline at end of file
+* Don't support flink read iceberg table in batch or streaming mode. [#1346](https://github.com/apache/iceberg/pull/1346) and [#1293](https://github.com/apache/iceberg/pull/1293) are tracking this.

Review comment:
       Yes, that's right. 

##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -72,13 +72,19 @@ public HiveCatalog(Configuration conf) {
   }
 
   public HiveCatalog(String name, String uri, int clientPoolSize, Configuration conf) {
+    this(name, uri, null, clientPoolSize, conf);
+  }
+
+  public HiveCatalog(String name, String uri, String warehouse, int clientPoolSize, Configuration conf) {
     this.name = name;
     this.conf = new Configuration(conf);
     // before building the client pool, overwrite the configuration's URIs if the argument is non-null
     if (uri != null) {
       this.conf.set(HiveConf.ConfVars.METASTOREURIS.varname, uri);
     }
-
+    if (warehouse != null) {
+      this.conf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, warehouse);

Review comment:
       As we discussed in this [comment](https://github.com/apache/iceberg/pull/1586/files#r509462458),   we could set `warehouse` string or `hive-conf-dir` to get the hive warehouse.   If use a local field,  then in getWarehouseLocation,  we would check the local field first and then read the hiveConf ?  I want to unify the parse path so I  put this key-value into hiveConf (It's a deep-cloned conf, so should not affect the origin conf).




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

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] JingsongLi commented on a change in pull request #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1586:
URL: https://github.com/apache/iceberg/pull/1586#discussion_r509850427



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -72,13 +72,19 @@ public HiveCatalog(Configuration conf) {
   }
 
   public HiveCatalog(String name, String uri, int clientPoolSize, Configuration conf) {
+    this(name, uri, null, clientPoolSize, conf);
+  }
+
+  public HiveCatalog(String name, String uri, String warehouse, int clientPoolSize, Configuration conf) {
     this.name = name;
     this.conf = new Configuration(conf);
     // before building the client pool, overwrite the configuration's URIs if the argument is non-null
     if (uri != null) {
       this.conf.set(HiveConf.ConfVars.METASTOREURIS.varname, uri);
     }
-
+    if (warehouse != null) {
+      this.conf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, warehouse);

Review comment:
       Yes, It can be handled in the same way as URIs.
   
   BTW, can you unify `hive.metastore.warehouse.dir` and `HiveConf.ConfVars.METASTOREWAREHOUSE.varname`?




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

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 #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

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


   The broken unit test is : 
   
   ```
   org.apache.iceberg.flink.TestFlinkCatalogDatabase > testListNamespace[catalogName = testhadoop baseNamespace = [Ljava.lang.String;@155b8b06] FAILED
       java.lang.AssertionError: Should have 2 database expected:<2> but was:<3>
           at org.junit.Assert.fail(Assert.java:88)
           at org.junit.Assert.failNotEquals(Assert.java:834)
           at org.junit.Assert.assertEquals(Assert.java:645)
           at org.apache.iceberg.flink.TestFlinkCatalogDatabase.testListNamespace(TestFlinkCatalogDatabase.java:150)
   ```
   
   Let we find out why. 


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

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 change in pull request #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1586:
URL: https://github.com/apache/iceberg/pull/1586#discussion_r504450286



##########
File path: flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java
##########
@@ -53,4 +64,36 @@ public static void stopMetastore() {
     flinkCatalogs.values().forEach(Catalog::close);
     flinkCatalogs.clear();
   }
+
+  protected TableEnvironment getTableEnv() {

Review comment:
       Moving those tow method inside this super class, because the newly introduced unit test `TestFlinkHiveCatalog`  would use 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.

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 change in pull request #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1586:
URL: https://github.com/apache/iceberg/pull/1586#discussion_r509854267



##########
File path: flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java
##########
@@ -34,23 +40,53 @@
   private static TestHiveMetastore metastore = null;
   protected static HiveConf hiveConf = null;
   protected static HiveCatalog catalog = null;
-  protected static ConcurrentMap<String, Catalog> flinkCatalogs;
+
+  private volatile TableEnvironment tEnv = null;
 
   @BeforeClass
   public static void startMetastore() {
     FlinkTestBase.metastore = new TestHiveMetastore();
-    metastore.start();
+    metastore.start(15);

Review comment:
        I thought the hive handlers were not enough in the `TestFlinkCatalogDatabase`  test suite, but today I checked the logs carefully, it's indeed a connection leak issue.   The catalog connection would be closed only when unregistering the catalog, means executing the sql `DROP CATALOG test_catalog`.  I found we did not release the connection after a test case finished.   No need to increase the handler number here, just need to provide `Before/After` to create/release connection, pls see https://github.com/apache/iceberg/pull/1586/commits/f2740c6cf4f7aa5e090a4b3df60e853aaa140e38.




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

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 change in pull request #1586: Flink: support specifying user-provided hive-site.xml for hive catalog.

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1586:
URL: https://github.com/apache/iceberg/pull/1586#discussion_r509462458



##########
File path: site/docs/flink.md
##########
@@ -97,7 +97,21 @@ CREATE CATALOG hive_catalog WITH (
   'catalog-type'='hive',
   'uri'='thrift://localhost:9083',
   'clients'='5',
-  'property-version'='1'
+  'property-version'='1',
+  'hive-conf-dir'='/opt/hive/conf'
+);
+```
+
+Alternatively one can instead set just the `warehouse` property (without specifying a Hive configuration directory) to initialize the Hive catalog:

Review comment:
       A small clarification: this implies that the use of `hive-site.xml` and `warehouse` are alternatives, but they aren't. You may want to use `warehouse` from the Hive site, or you may want to use other settings from `hive-site.xml` with a custom warehouse location. I think this should document the two options separately.
   
   For documenting options, I think we will want to rely more on the option list below because copying the SQL block each time creates a long doc. I would have one simple example for each catalog type with the options we recommend for it (Hive: uri, warehouse, and property-version; Hadoop: warehouse, property-version) and a refer to the options list to document `clients` and `hive-conf-dir`.




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

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