Skip to content

Commit 7daf17b

Browse files
committed
HIVE-28965: tez.DagUtils: Failed to add credential supplier, ClassNotFoundException: org.apache.hadoop.hive.kafka.KafkaDagCredentialSupplier
1 parent f75c18b commit 7daf17b

File tree

3 files changed

+102
-10
lines changed

3 files changed

+102
-10
lines changed

ql/pom.xml

Lines changed: 15 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -861,6 +861,21 @@
861861
<artifactId>jersey-multipart</artifactId>
862862
<version>${jersey.version}</version>
863863
</dependency>
864+
<dependency>
865+
<groupId>org.apache.kafka</groupId>
866+
<artifactId>kafka-clients</artifactId>
867+
<version>${kafka.version}</version>
868+
<exclusions>
869+
<exclusion>
870+
<groupId>com.github.luben</groupId>
871+
<artifactId>zstd-jni</artifactId>
872+
</exclusion>
873+
<exclusion>
874+
<groupId>org.apache.shiro</groupId>
875+
<artifactId>shiro-crypto-cipher</artifactId>
876+
</exclusion>
877+
</exclusions>
878+
</dependency>
864879
<dependency>
865880
<groupId>com.esri.geometry</groupId>
866881
<artifactId>esri-geometry-api</artifactId>

ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java

Lines changed: 83 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -41,6 +41,7 @@
4141
import java.util.LinkedHashMap;
4242
import java.util.List;
4343
import java.util.Map;
44+
import java.util.Properties;
4445
import java.util.Set;
4546
import java.util.Stack;
4647
import java.util.concurrent.TimeUnit;
@@ -105,6 +106,7 @@
105106
import org.apache.hadoop.hive.ql.plan.MapWork;
106107
import org.apache.hadoop.hive.ql.plan.MergeJoinWork;
107108
import org.apache.hadoop.hive.ql.plan.OperatorDesc;
109+
import org.apache.hadoop.hive.ql.plan.PartitionDesc;
108110
import org.apache.hadoop.hive.ql.plan.ReduceWork;
109111
import org.apache.hadoop.hive.ql.plan.TableDesc;
110112
import org.apache.hadoop.hive.ql.plan.TezEdgeProperty;
@@ -133,6 +135,8 @@
133135
import org.apache.hadoop.yarn.api.records.URL;
134136
import org.apache.hadoop.yarn.util.ConverterUtils;
135137
import org.apache.hadoop.yarn.util.Records;
138+
import org.apache.kafka.clients.CommonClientConfigs;
139+
import org.apache.kafka.common.security.auth.SecurityProtocol;
136140
import org.apache.tez.common.TezUtils;
137141
import org.apache.tez.dag.api.DAG;
138142
import org.apache.tez.dag.api.DataSinkDescriptor;
@@ -179,13 +183,29 @@ public class DagUtils {
179183
public static final String TEZ_TMP_DIR_KEY = "_hive_tez_tmp_dir";
180184
private static final Logger LOG = LoggerFactory.getLogger(DagUtils.class.getName());
181185
private static final String TEZ_DIR = "_tez_scratch_dir";
182-
private static final DagUtils instance = new DagUtils(defaultCredentialSuppliers());
186+
private static final DagUtils instance = new DagUtils();
183187
// The merge file being currently processed.
184188
public static final String TEZ_MERGE_CURRENT_MERGE_FILE_PREFIX =
185189
"hive.tez.current.merge.file.prefix";
186190
// A comma separated list of work names used as prefix.
187191
public static final String TEZ_MERGE_WORK_FILE_PREFIXES = "hive.tez.merge.file.prefixes";
188-
private final List<DagCredentialSupplier> credentialSuppliers;
192+
private static final List<DagCredentialSupplier> credentialSuppliers = new ArrayList<>();
193+
194+
/**
195+
* MANDATORY Table property indicating kafka broker(s) connection string.
196+
*/
197+
private static final String HIVE_KAFKA_BOOTSTRAP_SERVERS = "kafka.bootstrap.servers";
198+
199+
/**
200+
* Table property prefix used to inject kafka consumer properties, e.g "kafka.consumer.max.poll.records" = "5000"
201+
* this will lead to inject max.poll.records=5000 to the Kafka Consumer. NOT MANDATORY defaults to nothing
202+
*/
203+
private static final String CONSUMER_CONFIGURATION_PREFIX = "kafka.consumer";
204+
205+
/**
206+
* Table property prefix used to inject kafka producer properties, e.g "kafka.producer.lingers.ms" = "100".
207+
*/
208+
private static final String PRODUCER_CONFIGURATION_PREFIX = "kafka.producer";
189209
/**
190210
* Notifiers to synchronize resource localization across threads. If one thread is localizing
191211
* a file, other threads can wait on the corresponding notifier object instead of just sleeping
@@ -286,6 +306,23 @@ private void getCredentialsFromSuppliers(BaseWork work, Set<TableDesc> tables, D
286306
if (!UserGroupInformation.isSecurityEnabled()){
287307
return;
288308
}
309+
Map<String, PartitionDesc> partitions = ((MapWork) work).getAliasToPartnInfo();
310+
311+
// We don't need to iterate on all partitions, and check the same TableDesc.
312+
PartitionDesc partition = partitions.values().stream().findFirst().orElse(null);
313+
if (partition != null) {
314+
TableDesc tableDesc = partition.getTableDesc();
315+
if (isTokenRequired(tableDesc)) {
316+
addCredentialSuppliers(Collections.singletonList("org.apache.hadoop.hive.kafka.KafkaDagCredentialSupplier"));
317+
}
318+
}
319+
320+
for (TableDesc tableDesc : tables) {
321+
if (isTokenRequired(tableDesc)) {
322+
addCredentialSuppliers(Collections.singletonList("org.apache.hadoop.hive.kafka.KafkaDagCredentialSupplier"));
323+
break;
324+
}
325+
}
289326
for (DagCredentialSupplier supplier : credentialSuppliers) {
290327
Text alias = supplier.getTokenAlias();
291328
Token<?> t = dag.getCredentials().getToken(alias);
@@ -300,11 +337,10 @@ private void getCredentialsFromSuppliers(BaseWork work, Set<TableDesc> tables, D
300337
}
301338
}
302339

303-
private static List<DagCredentialSupplier> defaultCredentialSuppliers() {
340+
@VisibleForTesting
341+
static void addCredentialSuppliers(List<String> supplierClassNames) {
304342
// Class names of credential providers that should be used when adding credentials to the dag.
305343
// Use plain strings instead of {@link Class#getName()} to avoid compile scope dependencies to other modules.
306-
List<String> supplierClassNames =
307-
Collections.singletonList("org.apache.hadoop.hive.kafka.KafkaDagCredentialSupplier");
308344
List<DagCredentialSupplier> dagSuppliers = new ArrayList<>();
309345
for (String s : supplierClassNames) {
310346
try {
@@ -314,7 +350,46 @@ private static List<DagCredentialSupplier> defaultCredentialSuppliers() {
314350
LOG.error("Failed to add credential supplier", e);
315351
}
316352
}
317-
return dagSuppliers;
353+
credentialSuppliers.addAll(dagSuppliers);
354+
}
355+
356+
/**
357+
* Returns the security protocol if one is defined in the properties and null otherwise.
358+
* <p>The following properties are examined to determine the protocol:</p>
359+
* <ol>
360+
* <li>security.protocol</li>
361+
* <li>kafka.consumer.security.protocol</li>
362+
* <li>kafka.producer.security.protocol</li>
363+
* </ol>
364+
* <p>and the first non null/not empty is returned.</p>
365+
* <p>Defining multiple security protocols at the same time is invalid but this method is lenient and tries to pick
366+
* the most reasonable option.</p>
367+
* @param props the properties from which to obtain the protocol.
368+
* @return the security protocol if one is defined in the properties and null otherwise.
369+
*/
370+
static SecurityProtocol getSecurityProtocol(Properties props) {
371+
String[] securityProtocolConfigs = new String[] { CommonClientConfigs.SECURITY_PROTOCOL_CONFIG,
372+
CONSUMER_CONFIGURATION_PREFIX + "." + CommonClientConfigs.SECURITY_PROTOCOL_CONFIG,
373+
PRODUCER_CONFIGURATION_PREFIX + "." + CommonClientConfigs.SECURITY_PROTOCOL_CONFIG };
374+
for (String c : securityProtocolConfigs) {
375+
String v = props.getProperty(c);
376+
if (v != null && !v.isEmpty()) {
377+
return SecurityProtocol.forName(v);
378+
}
379+
}
380+
return null;
381+
}
382+
383+
/**
384+
* Returns whether a Kafka token is required for performing operations on the specified table.
385+
* If "security.protocol" is set to "PLAINTEXT", we don't need to collect delegation token at all.
386+
*
387+
* @return true if a Kafka token is required for performing operations on the specified table and false otherwise.
388+
*/
389+
private boolean isTokenRequired(TableDesc tableDesc) {
390+
String kafkaBrokers = tableDesc.getProperties().getProperty(HIVE_KAFKA_BOOTSTRAP_SERVERS);
391+
SecurityProtocol protocol = getSecurityProtocol(tableDesc.getProperties());
392+
return !StringUtils.isEmpty(kafkaBrokers) && SecurityProtocol.PLAINTEXT != protocol;
318393
}
319394

320395
private void collectNeededFileSinkData(BaseWork work, Set<URI> fileSinkUris, Set<TableDesc> fileSinkTableDescs) {
@@ -1697,8 +1772,8 @@ public static String getUserSpecifiedDagName(Configuration conf) {
16971772
}
16981773

16991774
@VisibleForTesting
1700-
DagUtils(List<DagCredentialSupplier> suppliers) {
1701-
this.credentialSuppliers = suppliers;
1775+
DagUtils() {
1776+
// don't instantiate
17021777
}
17031778

17041779
/**

ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestDagUtilsSecurityEnabled.java

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -59,7 +59,8 @@ public static void clear() {
5959
@Test
6060
public void testAddCredentialsWithCredentialSupplierNewTokenAdded() {
6161
IncrementalIntDagCredentialSupplier supplier = new IncrementalIntDagCredentialSupplier();
62-
DagUtils dagUtils = new DagUtils(Collections.singletonList(supplier));
62+
DagUtils dagUtils = new DagUtils();
63+
DagUtils.addCredentialSuppliers(Collections.singletonList("org.apache.hadoop.hive.ql.exec.tez.IncrementalIntDagCredentialSupplier"));
6364
DAG dag = DAG.create("test_credentials_dag");
6465

6566
dagUtils.addCredentials(mock(MapWork.class), dag, null);
@@ -70,7 +71,8 @@ public void testAddCredentialsWithCredentialSupplierNewTokenAdded() {
7071
@Test
7172
public void testAddCredentialsWithCredentialSupplierTokenExistsNothingAdded() {
7273
IncrementalIntDagCredentialSupplier supplier = new IncrementalIntDagCredentialSupplier();
73-
DagUtils dagUtils = new DagUtils(Collections.singletonList(supplier));
74+
DagUtils dagUtils = new DagUtils();
75+
DagUtils.addCredentialSuppliers(Collections.singletonList("org.apache.hadoop.hive.ql.exec.tez.IncrementalIntDagCredentialSupplier"));
7476
DAG dag = DAG.create("test_credentials_dag");
7577
Token<TokenIdentifier> oldToken = new Token<>();
7678
// Add explicitly the token in the DAG before calling addCredentials simulating the use-case where the DAG has already the token

0 commit comments

Comments
 (0)