HADOOP-18870. CURATOR-599 change broke functionality introduced in HADOOP-18139 and HADOOP-18709. Contributed by Ferenc Erdelyi

This commit is contained in:
Szilard Nemeth 2023-09-06 21:32:36 -04:00
parent 07c8b69f7c
commit 9342ecf6cc
2 changed files with 46 additions and 1 deletions

View File

@ -549,7 +549,17 @@ public HadoopZookeeperFactory(String zkPrincipal, String kerberosPrincipal,
public ZooKeeper newZooKeeper(String connectString, int sessionTimeout,
Watcher watcher, boolean canBeReadOnly
) throws Exception {
ZKClientConfig zkClientConfig = new ZKClientConfig();
return this.newZooKeeper(connectString, sessionTimeout,
watcher, canBeReadOnly, new ZKClientConfig());
}
@Override
public ZooKeeper newZooKeeper(String connectString, int sessionTimeout,
Watcher watcher, boolean canBeReadOnly, ZKClientConfig zkClientConfig
) throws Exception {
if (zkClientConfig == null) {
zkClientConfig = new ZKClientConfig();
}
if (zkPrincipal != null) {
LOG.info("Configuring zookeeper to use {} as the server principal",
zkPrincipal);

View File

@ -22,10 +22,15 @@
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import javax.security.auth.login.AppConfigurationEntry;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.CuratorFrameworkFactory;
import org.apache.curator.retry.RetryNTimes;
import org.apache.curator.test.TestingServer;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
@ -193,6 +198,36 @@ public void testJaasConfiguration() throws Exception {
}
}
@Test
public void testCuratorFrameworkFactory() throws Exception{
// By not explicitly calling the NewZooKeeper method validate that the Curator override works.
ZKClientConfig zkClientConfig = new ZKClientConfig();
Configuration conf = new Configuration();
conf.set(CommonConfigurationKeys.ZK_ADDRESS, this.server.getConnectString());
int numRetries = conf.getInt(CommonConfigurationKeys.ZK_NUM_RETRIES,
CommonConfigurationKeys.ZK_NUM_RETRIES_DEFAULT);
int zkSessionTimeout = conf.getInt(CommonConfigurationKeys.ZK_TIMEOUT_MS,
CommonConfigurationKeys.ZK_TIMEOUT_MS_DEFAULT);
int zkRetryInterval = conf.getInt(
CommonConfigurationKeys.ZK_RETRY_INTERVAL_MS,
CommonConfigurationKeys.ZK_RETRY_INTERVAL_MS_DEFAULT);
RetryNTimes retryPolicy = new RetryNTimes(numRetries, zkRetryInterval);
CuratorFramework client = CuratorFrameworkFactory.builder()
.connectString(conf.get(CommonConfigurationKeys.ZK_ADDRESS))
.zkClientConfig(zkClientConfig)
.sessionTimeoutMs(zkSessionTimeout).retryPolicy(retryPolicy)
.authorization(new ArrayList<>())
.zookeeperFactory(new ZKCuratorManager.HadoopZookeeperFactory(
"foo1", "bar1", "bar1.keytab", false,
new ZKCuratorManager.TruststoreKeystore(conf))
).build();
client.start();
validateJaasConfiguration(ZKCuratorManager.HadoopZookeeperFactory.JAAS_CLIENT_ENTRY,
"bar1", "bar1.keytab", client.getZookeeperClient().getZooKeeper());
}
private void validateJaasConfiguration(String clientConfig, String principal, String keytab,
ZooKeeper zk) {
assertEquals("Validate that expected clientConfig is set in ZK config", clientConfig,