ayushtkn commented on code in PR #5657:
URL: https://github.com/apache/hive/pull/5657#discussion_r1972959590


##########
llap-common/src/java/org/apache/hadoop/hive/llap/security/SecretManager.java:
##########
@@ -45,6 +46,12 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static 
org.apache.hadoop.security.token.delegation.ZKDelegationTokenSecretManager.ZK_DTSM_ZK_SSL_ENABLED;
+import static 
org.apache.hadoop.security.token.delegation.ZKDelegationTokenSecretManager.ZK_DTSM_ZK_SSL_KEYSTORE_LOCATION;
+import static 
org.apache.hadoop.security.token.delegation.ZKDelegationTokenSecretManager.ZK_DTSM_ZK_SSL_KEYSTORE_PASSWORD;
+import static 
org.apache.hadoop.security.token.delegation.ZKDelegationTokenSecretManager.ZK_DTSM_ZK_SSL_TRUSTSTORE_LOCATION;
+import static 
org.apache.hadoop.security.token.delegation.ZKDelegationTokenSecretManager.ZK_DTSM_ZK_SSL_TRUSTSTORE_PASSWORD;
+

Review Comment:
   these are all unused, you have an import for 
``ZKDelegationTokenSecretManager`` above already



##########
llap-common/src/java/org/apache/hadoop/hive/llap/security/SecretManager.java:
##########
@@ -267,10 +287,28 @@ public void close() {
   private static void checkRootAcls(Configuration conf, String path, String 
user) {
     int stime = conf.getInt(ZK_DTSM_ZK_SESSION_TIMEOUT, 
ZK_DTSM_ZK_SESSION_TIMEOUT_DEFAULT),
         ctime = conf.getInt(ZK_DTSM_ZK_CONNECTION_TIMEOUT, 
ZK_DTSM_ZK_CONNECTION_TIMEOUT_DEFAULT);
-    CuratorFramework zkClient = 
CuratorFrameworkFactory.builder().namespace(null)
-        .retryPolicy(new 
RetryOneTime(10)).sessionTimeoutMs(stime).connectionTimeoutMs(ctime)
-        .ensembleProvider(new 
FixedEnsembleProvider(conf.get(ZK_DTSM_ZK_CONNECTION_STRING)))
-        .build();
+
+    CuratorFramework zkClient = null;
+    if (HiveConf.getBoolVar(conf, 
ConfVars.LLAP_ZKSM_ZK_CONNECTION_SSL_ENABLED)) {
+      zkClient = CuratorFrameworkFactory.builder().namespace(null)
+              .retryPolicy(new 
RetryOneTime(10)).sessionTimeoutMs(stime).connectionTimeoutMs(ctime)
+              .ensembleProvider(new 
FixedEnsembleProvider(conf.get(ZK_DTSM_ZK_CONNECTION_STRING)))
+              .zookeeperFactory(new SSLZookeeperFactory(
+                      HiveConf.getBoolVar(conf, 
ConfVars.LLAP_ZKSM_ZK_CONNECTION_SSL_ENABLED),
+                      HiveConf.getVar(conf, 
ConfVars.LLAP_ZKSM_ZK_CONNECTION_SSL_KEYSTORE_LOCATION),
+                      HiveConf.getVar(conf, 
ConfVars.LLAP_ZKSM_ZK_CONNECTION_SSL_KEYSTORE_PASSWORD),
+                      "",
+                      HiveConf.getVar(conf, 
ConfVars.LLAP_ZKSM_ZK_CONNECTION_SSL_TRUSTSTORE_LOCATION),
+                      HiveConf.getVar(conf, 
ConfVars.LLAP_ZKSM_ZK_CONNECTION_SSL_TRUSTSTORE_PASSWORD),
+                      "")
+              ).build();

Review Comment:
   can `ZookeeperHiveHelper` or `ZooKeeperHiveClientHelper` be used 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.

To unsubscribe, e-mail: gitbox-unsubscr...@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscr...@hive.apache.org
For additional commands, e-mail: gitbox-h...@hive.apache.org

Reply via email to