abstractdog commented on code in PR #4385: URL: https://github.com/apache/hive/pull/4385#discussion_r1219295973
########## kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaDagCredentialSupplier.java: ########## @@ -0,0 +1,136 @@ +/* + * 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. + */ +package org.apache.hadoop.hive.kafka; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.exec.tez.DagCredentialSupplier; +import org.apache.hadoop.hive.ql.plan.BaseWork; +import org.apache.hadoop.hive.ql.plan.MapWork; +import org.apache.hadoop.hive.ql.plan.PartitionDesc; +import org.apache.hadoop.hive.ql.plan.TableDesc; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.security.token.Token; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.AdminClientConfig; +import org.apache.kafka.clients.admin.CreateDelegationTokenOptions; +import org.apache.kafka.clients.admin.CreateDelegationTokenResult; +import org.apache.kafka.common.config.SaslConfigs; +import org.apache.kafka.common.security.token.delegation.DelegationToken; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.ExecutionException; + +public class KafkaDagCredentialSupplier implements DagCredentialSupplier { + private static final Logger LOG = LoggerFactory.getLogger(KafkaDagCredentialSupplier.class); + private static final Text KAFKA_DELEGATION_TOKEN_KEY = new Text("KAFKA_DELEGATION_TOKEN"); + + @Override + public Token<?> obtainToken(BaseWork work, Set<TableDesc> fileSinkTableDescs, Configuration conf) { + if(!(work instanceof MapWork)){ + return null; + } + Map<String, PartitionDesc> partitions = ((MapWork) work).getAliasToPartnInfo(); + + // We don't need to iterate on all partitions, and check the same TableDesc. + PartitionDesc partition = partitions.values().stream().findFirst().orElse(null); + if (partition != null) { + TableDesc tableDesc = partition.getTableDesc(); + if (collectKafkaDelegationTokenForTableDesc(tableDesc)) { + // don't collect delegation token again, if it was already successful + return getKafkaDelegationTokenForBrokers(conf, tableDesc); + } + } + + for (TableDesc tableDesc : fileSinkTableDescs) { + if (collectKafkaDelegationTokenForTableDesc(tableDesc)) { + // don't collect delegation token again, if it was already successful + return getKafkaDelegationTokenForBrokers(conf, tableDesc); + } + } + return null; + } + + @Override + public Text getTokenAlias() { + return KAFKA_DELEGATION_TOKEN_KEY; + } + + /** + * Returns whether we should collect delegation tokens for kafka in the scope of a TableDesc. + * If "security.protocol" is set to "PLAINTEXT", we don't need to collect delegation token at all. + * @param tableDesc + * @return true if we should collect a token for the specified table and false otherwise. + */ + private boolean collectKafkaDelegationTokenForTableDesc(TableDesc tableDesc) { + String kafkaBrokers = (String) tableDesc.getProperties().get("kafka.bootstrap.servers"); //FIXME: KafkaTableProperties + String consumerSecurityProtocol = (String) tableDesc.getProperties().get( + "kafka.consumer." + CommonClientConfigs.SECURITY_PROTOCOL_CONFIG); + String producerSecurityProtocol = (String) tableDesc.getProperties().get( + "kafka.producer." + CommonClientConfigs.SECURITY_PROTOCOL_CONFIG); + return kafkaBrokers != null && !kafkaBrokers.isEmpty() + && !CommonClientConfigs.DEFAULT_SECURITY_PROTOCOL.equalsIgnoreCase(consumerSecurityProtocol) + && !CommonClientConfigs.DEFAULT_SECURITY_PROTOCOL.equalsIgnoreCase(producerSecurityProtocol); + } + + private Token<?> getKafkaDelegationTokenForBrokers(Configuration conf, TableDesc tableDesc) { + String kafkaBrokers = (String) tableDesc.getProperties().get("kafka.bootstrap.servers"); + LOG.info("Getting kafka credentials for brokers: {}", kafkaBrokers); + + String keytab = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB); + String principal = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL); + try { + principal = SecurityUtil.getServerPrincipal(principal, "0.0.0.0"); Review Comment: also, SecurityUtil.getServerPrincipal(principal, "0.0.0.0") is introduced by an earlier patch, not in this refactor, I believe if it's not okay (but it is), we need to address it in another ticket instead of hiding it in a refactor -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
