diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index bb0ca3acf79426df5d34d44da8595ac78edf8c76..285caa3f761595a61ca337fdb0b669d54e6c3b25 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -1206,6 +1206,8 @@ private static void populateLlapDaemonVarsSet(Set llapDaemonVarsSetLocal "to use dictionary or not will be retained thereafter."), HIVE_ORC_DEFAULT_BUFFER_SIZE("hive.exec.orc.default.buffer.size", 256 * 1024, "Define the default ORC buffer size, in bytes."), + HIVE_ORC_BASE_DELTA_RATIO("hive.exec.orc.base.delta.ratio", 8, "The ratio of base writer and\n" + + "delta writer in terms of STRIPE_SIZE and BUFFER_SIZE."), HIVE_ORC_DEFAULT_BLOCK_PADDING("hive.exec.orc.default.block.padding", true, "Define the default block padding, which pads stripes to the HDFS block boundaries."), HIVE_ORC_BLOCK_PADDING_TOLERANCE("hive.exec.orc.block.padding.tolerance", 0.05f, @@ -2886,6 +2888,11 @@ private static void populateLlapDaemonVarsSet(Set llapDaemonVarsSetLocal LLAP_ENABLE_GRACE_JOIN_IN_LLAP("hive.llap.enable.grace.join.in.llap", false, "Override if grace join should be allowed to run in llap."), + LLAP_HS2_ENABLE_COORDINATOR("hive.llap.hs2.coordinator.enabled", true, + "Whether to create the LLAP coordinator; since execution engine and container vs llap\n" + + "settings are both coming from job configs, we don't know at start whether this should\n" + + "be created. Default true."), + SPARK_CLIENT_FUTURE_TIMEOUT("hive.spark.client.future.timeout", "60s", new TimeValidator(TimeUnit.SECONDS), "Timeout for requests from Hive client to remote Spark driver."), diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/llap/ext/TestLlapInputSplit.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/llap/ext/TestLlapInputSplit.java index 1de8aa6686a5b5db697fb4dee4b4155b9dc14ef2..7491222ef3485d0dab7179cab55096446952148d 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/llap/ext/TestLlapInputSplit.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/llap/ext/TestLlapInputSplit.java @@ -59,13 +59,8 @@ public void testWritable() throws Exception { colDescs.add(new FieldDesc("col2", new TypeDesc(TypeDesc.Type.INT))); Schema schema = new Schema(colDescs); - LlapInputSplit split1 = new LlapInputSplit( - splitNum, - planBytes, - fragmentBytes, - locations, - schema, - "hive"); + LlapInputSplit split1 = new LlapInputSplit(splitNum, planBytes, fragmentBytes, null, + locations, schema, "hive"); ByteArrayOutputStream byteOutStream = new ByteArrayOutputStream(); DataOutputStream dataOut = new DataOutputStream(byteOutStream); split1.write(dataOut); diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java b/llap-client/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java index 02aedfd8929038e763ed413a3545f68117e08638..ab11926e90d410cca8f1eafd5b1892932d936c8a 100644 --- a/llap-client/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java +++ b/llap-client/src/java/org/apache/hadoop/hive/llap/LlapInputSplit.java @@ -23,24 +23,26 @@ import org.apache.hadoop.hive.llap.Schema; import org.apache.hadoop.mapred.InputSplitWithLocationInfo; import org.apache.hadoop.mapred.SplitLocationInfo; -import org.apache.thrift.TDeserializer; -import org.apache.thrift.TSerializer; public class LlapInputSplit implements InputSplitWithLocationInfo { - int splitNum; - byte[] planBytes; - byte[] fragmentBytes; - SplitLocationInfo[] locations; - Schema schema; - String llapUser; + private int splitNum; + private byte[] planBytes; + private byte[] fragmentBytes; + private SplitLocationInfo[] locations; + private Schema schema; + private String llapUser; + private byte[] fragmentBytesSignature; public LlapInputSplit() { } - public LlapInputSplit(int splitNum, byte[] planBytes, byte[] fragmentBytes, SplitLocationInfo[] locations, Schema schema, String llapUser) { + public LlapInputSplit(int splitNum, byte[] planBytes, byte[] fragmentBytes, + byte[] fragmentBytesSignature, SplitLocationInfo[] locations, Schema schema, + String llapUser) { this.planBytes = planBytes; this.fragmentBytes = fragmentBytes; + this.fragmentBytesSignature = fragmentBytesSignature; this.locations = locations; this.schema = schema; this.splitNum = splitNum; @@ -77,7 +79,9 @@ public int getSplitNum() { return fragmentBytes; } - + public byte[] getFragmentBytesSignature() { + return fragmentBytesSignature; + } @Override public void write(DataOutput out) throws IOException { @@ -87,6 +91,12 @@ public void write(DataOutput out) throws IOException { out.writeInt(fragmentBytes.length); out.write(fragmentBytes); + if (fragmentBytesSignature != null) { + out.writeInt(fragmentBytesSignature.length); + out.write(fragmentBytesSignature); + } else { + out.writeInt(0); + } out.writeInt(locations.length); for (int i = 0; i < locations.length; ++i) { @@ -107,6 +117,11 @@ public void readFields(DataInput in) throws IOException { length = in.readInt(); fragmentBytes = new byte[length]; in.readFully(fragmentBytes); + length = in.readInt(); + if (length > 0) { + fragmentBytesSignature = new byte[length]; + in.readFully(fragmentBytesSignature); + } length = in.readInt(); locations = new SplitLocationInfo[length]; diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java b/llap-client/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java index 670429412923566536dc933463a64daabc5adb45..95b0ffcea5958d385c4933f065f5a2a35d3bce22 100644 --- a/llap-client/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java +++ b/llap-client/src/java/org/apache/hadoop/hive/llap/SubmitWorkInfo.java @@ -28,33 +28,32 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.tez.common.security.JobTokenIdentifier; import org.apache.tez.common.security.JobTokenSecretManager; -import org.apache.tez.runtime.api.impl.TaskSpec; public class SubmitWorkInfo implements Writable { - private TaskSpec taskSpec; private ApplicationId fakeAppId; private long creationTime; + private byte[] vertexSpec, vertexSpecSignature; // This is used to communicate over the LlapUmbilicalProtocol. Not related to tokens used to // talk to LLAP daemons itself via the securit work. private Token token; + private int vertexParallelism; - public SubmitWorkInfo(TaskSpec taskSpec, ApplicationId fakeAppId, long creationTime) { - this.taskSpec = taskSpec; + public SubmitWorkInfo(ApplicationId fakeAppId, long creationTime, + int vertexParallelism, byte[] vertexSpec, byte[] vertexSpecSignature) { this.fakeAppId = fakeAppId; this.token = createJobToken(); this.creationTime = creationTime; + this.vertexSpec = vertexSpec; + this.vertexSpecSignature = vertexSpecSignature; + this.vertexParallelism = vertexParallelism; } // Empty constructor for writable etc. public SubmitWorkInfo() { } - public TaskSpec getTaskSpec() { - return taskSpec; - } - public ApplicationId getFakeAppId() { return fakeAppId; } @@ -73,23 +72,44 @@ public long getCreationTime() { @Override public void write(DataOutput out) throws IOException { - taskSpec.write(out); out.writeLong(fakeAppId.getClusterTimestamp()); out.writeInt(fakeAppId.getId()); token.write(out); out.writeLong(creationTime); + out.writeInt(vertexParallelism); + if (vertexSpec != null) { + out.writeInt(vertexSpec.length); + out.write(vertexSpec); + } else { + out.writeInt(0); + } + if (vertexSpecSignature != null) { + out.writeInt(vertexSpecSignature.length); + out.write(vertexSpecSignature); + } else { + out.writeInt(0); + } } @Override public void readFields(DataInput in) throws IOException { - taskSpec = new TaskSpec(); - taskSpec.readFields(in); long appIdTs = in.readLong(); int appIdId = in.readInt(); fakeAppId = ApplicationId.newInstance(appIdTs, appIdId); token = new Token<>(); token.readFields(in); creationTime = in.readLong(); + vertexParallelism = in.readInt(); + int vertexSpecBytes = in.readInt(); + if (vertexSpecBytes > 0) { + vertexSpec = new byte[vertexSpecBytes]; + in.readFully(vertexSpec); + } + int vertexSpecSignBytes = in.readInt(); + if (vertexSpecSignBytes > 0) { + vertexSpecSignature = new byte[vertexSpecSignBytes]; + in.readFully(vertexSpecSignature); + } } public static byte[] toBytes(SubmitWorkInfo submitWorkInfo) throws IOException { @@ -116,4 +136,16 @@ public static SubmitWorkInfo fromBytes(byte[] submitWorkInfoBytes) throws IOExce sessionToken.setService(identifier.getJobId()); return sessionToken; } + + public byte[] getVertexBinary() { + return vertexSpec; + } + + public byte[] getVertexSignature() { + return vertexSpecSignature; + } + + public int getVertexParallelism() { + return vertexParallelism; + } } diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/coordinator/LlapCoordinator.java b/llap-client/src/java/org/apache/hadoop/hive/llap/coordinator/LlapCoordinator.java new file mode 100644 index 0000000000000000000000000000000000000000..f55779b9f4e95c828e42061230d64f8bf5253fb4 --- /dev/null +++ b/llap-client/src/java/org/apache/hadoop/hive/llap/coordinator/LlapCoordinator.java @@ -0,0 +1,159 @@ +/** + * 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.llap.coordinator; + +import java.io.IOException; +import java.util.Random; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.llap.DaemonId; +import org.apache.hadoop.hive.llap.LlapUtil; +import org.apache.hadoop.hive.llap.coordinator.LlapCoordinator; +import org.apache.hadoop.hive.llap.security.LlapSigner; +import org.apache.hadoop.hive.llap.security.LlapSignerImpl; +import org.apache.hadoop.hive.llap.security.LlapTokenLocalClient; +import org.apache.hadoop.hive.llap.security.LlapTokenLocalClientImpl; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.RemovalListener; +import com.google.common.cache.RemovalNotification; + +/** + * The class containing facilities for LLAP interactions in HS2. + * This may eventually evolve into a central LLAP manager hosted by HS2 or elsewhere. + * Refactor as needed. + */ +public class LlapCoordinator { + private static final Logger LOG = LoggerFactory.getLogger(LlapCoordinator.class); + + /** We'll keep signers per cluster around for some time, for reuse. */ + private final Cache signers = CacheBuilder.newBuilder().removalListener( + new RemovalListener() { + @Override + public void onRemoval(RemovalNotification notification) { + if (notification.getValue() != null) { + notification.getValue().close(); + } + } + }).expireAfterAccess(10, TimeUnit.MINUTES).build(); + + // TODO: probably temporary before HIVE-13698; after that we may create one per session. + private static final Cache localClientCache = CacheBuilder + .newBuilder().expireAfterAccess(10, TimeUnit.MINUTES) + .removalListener(new RemovalListener() { + @Override + public void onRemoval(RemovalNotification notification) { + if (notification.getValue() != null) { + notification.getValue().close(); + } + } + }).build(); + + private HiveConf hiveConf; + private String clusterUser; + + LlapCoordinator() { + } + + private void init(HiveConf hiveConf) throws IOException { + // Only do the lightweight stuff in ctor; by default, LLAP coordinator is created during + // HS2 init without the knowledge of LLAP usage (or lack thereof) in the cluster. + this.hiveConf = hiveConf; + this.clusterUser = UserGroupInformation.getCurrentUser().getShortUserName(); + } + + public LlapSigner getLlapSigner(final Configuration jobConf) { + // Note that we create the cluster name from user conf (hence, a user can target a cluster), + // but then we create the signer using hiveConf (hence, we control the ZK config and stuff). + assert UserGroupInformation.isSecurityEnabled(); + final String clusterId = DaemonId.createClusterString( + clusterUser, LlapUtil.generateClusterName(jobConf)); + try { + return signers.get(clusterId, new Callable() { + public LlapSigner call() throws Exception { + return new LlapSignerImpl(hiveConf, clusterId); + } + }); + } catch (ExecutionException e) { + throw new RuntimeException(e); + } + } + + public ApplicationId createExtClientAppId() { + // TODO: moved from UDTF; need JIRA to generate this properly (no dups, etc.)... + return ApplicationId.newInstance(Math.abs(new Random().nextInt()), 0); + // Note that we cannot allow users to provide app ID, since providing somebody else's appId + // would give one LLAP token (and splits) for that app ID. If we could verify it somehow + // (YARN token? nothing we can do in an UDF), we could get it from client already running on + // YARN. As such, the clients running on YARN will have two app IDs to be aware of. + // TODO: Perhaps they can give us their app id as an argument to the UDF, and we'd just append + // a unique string here, for easier tracking? + } + + public LlapTokenLocalClient getLocalTokenClient( + final Configuration conf, String clusterUser) throws IOException { + // Note that we create the cluster name from user conf (hence, a user can target a cluster), + // but then we create the signer using hiveConf (hence, we control the ZK config and stuff). + assert UserGroupInformation.isSecurityEnabled(); + String clusterName = LlapUtil.generateClusterName(conf); + // This assumes that the LLAP cluster and session are both running under HS2 user. + final String clusterId = DaemonId.createClusterString(clusterUser, clusterName); + try { + return localClientCache.get(clusterId, new Callable() { + @Override + public LlapTokenLocalClientImpl call() throws Exception { + return new LlapTokenLocalClientImpl(hiveConf, clusterId); + } + }); + } catch (ExecutionException e) { + throw new IOException(e); + } + } + + public void close() { + try { + localClientCache.invalidateAll(); + signers.invalidateAll(); + localClientCache.cleanUp(); + signers.cleanUp(); + } catch (Exception ex) { + LOG.error("Error closing the coordinator; ignoring", ex); + } + } + + /** TODO: ideally, when the splits UDF is made a proper API, coordinator should not + * be managed as a global. HS2 should create it and then pass it around. */ + private static final LlapCoordinator INSTANCE = new LlapCoordinator(); + public static void initializeInstance(HiveConf hiveConf) throws IOException { + INSTANCE.init(hiveConf); + } + + public static LlapCoordinator getInstance() { + return INSTANCE; + } +} diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java b/llap-client/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java index 0edb1cdd04df578228cad97cf85216186c04007d..3ebae4adbd66d68cf6bd4599757425e0c8f8e098 100644 --- a/llap-client/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java +++ b/llap-client/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java @@ -27,7 +27,9 @@ import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; -import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import com.google.protobuf.InvalidProtocolBufferException; + import org.apache.commons.collections4.ListUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.conf.HiveConf; @@ -35,6 +37,7 @@ import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SignableVertexSpec; import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWorkRequestProto; import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier; +import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexOrBinary; import org.apache.hadoop.hive.llap.protocol.LlapTaskUmbilicalProtocol; import org.apache.hadoop.hive.llap.tez.Converters; import org.apache.hadoop.hive.llap.tez.LlapProtocolClientProxy; @@ -43,12 +46,9 @@ import org.apache.hadoop.ipc.ProtocolSignature; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.service.AbstractService; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.tez.common.security.JobTokenIdentifier; import org.apache.tez.dag.api.TezException; import org.apache.tez.dag.records.TezTaskAttemptID; -import org.apache.tez.runtime.api.Event; import org.apache.tez.runtime.api.impl.EventType; import org.apache.tez.runtime.api.impl.TezEvent; import org.apache.tez.runtime.api.impl.TezHeartbeatRequest; @@ -139,27 +139,33 @@ public InetSocketAddress getAddress() { /** * Submit the work for actual execution. - * @param submitWorkRequestProto + * @throws InvalidProtocolBufferException */ - public void submitWork(final SubmitWorkRequestProto submitWorkRequestProto, String llapHost, int llapPort, List tezEvents) { + public void submitWork(SubmitWorkRequestProto request, String llapHost, int llapPort) { // Register the pending events to be sent for this spec. - SignableVertexSpec vertex = submitWorkRequestProto.getWorkSpec().getVertex(); + VertexOrBinary vob = request.getWorkSpec(); + assert vob.hasVertexBinary() != vob.hasVertex(); + SignableVertexSpec vertex = null; + try { + vertex = vob.hasVertex() ? vob.getVertex() + : SignableVertexSpec.parseFrom(vob.getVertexBinary()); + } catch (InvalidProtocolBufferException e) { + throw new RuntimeException(e); + } VertexIdentifier vId = vertex.getVertexIdentifier(); TezTaskAttemptID attemptId = Converters.createTaskAttemptId( - vId, submitWorkRequestProto.getFragmentNumber(), submitWorkRequestProto.getAttemptNumber()); + vId, request.getFragmentNumber(), request.getAttemptNumber()); final String fragmentId = attemptId.toString(); - PendingEventData pendingEventData = new PendingEventData( - new TaskHeartbeatInfo(fragmentId, llapHost, llapPort), - tezEvents); - pendingEvents.putIfAbsent(fragmentId, pendingEventData); + pendingEvents.putIfAbsent(fragmentId, new PendingEventData( + new TaskHeartbeatInfo(fragmentId, llapHost, llapPort), Lists.newArrayList())); // Setup timer task to check for hearbeat timeouts timer.scheduleAtFixedRate(new HeartbeatCheckTask(), connectionTimeout, connectionTimeout, TimeUnit.MILLISECONDS); // Send out the actual SubmitWorkRequest - communicator.sendSubmitWork(submitWorkRequestProto, llapHost, llapPort, + communicator.sendSubmitWork(request, llapHost, llapPort, new LlapProtocolClientProxy.ExecuteRequestCallback() { @Override diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/io/api/LlapProxy.java b/llap-client/src/java/org/apache/hadoop/hive/llap/io/api/LlapProxy.java index 6c2618bcd3ca9f74302c3260b1535b892f543d23..8f1b59b6181dd0dfec7207881785d93da508a300 100644 --- a/llap-client/src/java/org/apache/hadoop/hive/llap/io/api/LlapProxy.java +++ b/llap-client/src/java/org/apache/hadoop/hive/llap/io/api/LlapProxy.java @@ -17,14 +17,13 @@ */ package org.apache.hadoop.hive.llap.io.api; -import java.io.IOException; import java.lang.reflect.Constructor; - import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.llap.coordinator.LlapCoordinator; @SuppressWarnings("rawtypes") public class LlapProxy { - private final static String IMPL_CLASS = "org.apache.hadoop.hive.llap.io.api.impl.LlapIoImpl"; + private final static String IO_IMPL_CLASS = "org.apache.hadoop.hive.llap.io.api.impl.LlapIoImpl"; // Llap server depends on Hive execution, so the reverse cannot be true. We create the I/O // singleton once (on daemon startup); the said singleton serves as the IO interface. @@ -48,23 +47,18 @@ public static void initializeLlapIo(Configuration conf) { if (io != null) { return; // already initialized } - - try { - io = createIoImpl(conf); - } catch (IOException e) { - throw new RuntimeException("Cannot initialize local server", e); - } + io = createInstance(IO_IMPL_CLASS, conf); } - private static LlapIo createIoImpl(Configuration conf) throws IOException { + private static T createInstance(String className, Configuration conf) { try { @SuppressWarnings("unchecked") - Class clazz = (Class)Class.forName(IMPL_CLASS); - Constructor ctor = clazz.getDeclaredConstructor(Configuration.class); + Class clazz = (Class)Class.forName(className); + Constructor ctor = clazz.getDeclaredConstructor(Configuration.class); ctor.setAccessible(true); return ctor.newInstance(conf); } catch (Exception e) { - throw new RuntimeException("Failed to create impl class", e); + throw new RuntimeException("Failed to create " + className, e); } } diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/security/LlapTokenLocalClient.java b/llap-client/src/java/org/apache/hadoop/hive/llap/security/LlapTokenLocalClient.java deleted file mode 100644 index af889b6a5af6499f01834886c5746243ad5847c1..0000000000000000000000000000000000000000 --- a/llap-client/src/java/org/apache/hadoop/hive/llap/security/LlapTokenLocalClient.java +++ /dev/null @@ -1,59 +0,0 @@ -/** - * 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.llap.security; - -import java.io.IOException; - - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.security.token.Token; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class LlapTokenLocalClient { - private static final Logger LOG = LoggerFactory.getLogger(LlapTokenLocalClient.class); - private final SecretManager secretManager; - - public LlapTokenLocalClient(Configuration conf, String clusterId) { - // TODO: create this centrally in HS2 case - secretManager = SecretManager.createSecretManager(conf, clusterId); - } - - public Token createToken( - String appId, String user, boolean isSignatureRequired) throws IOException { - try { - Token token = secretManager.createLlapToken( - appId, user, isSignatureRequired); - if (LOG.isInfoEnabled()) { - LOG.info("Created a LLAP delegation token locally: " + token); - } - return token; - } catch (Exception ex) { - throw (ex instanceof IOException) ? (IOException)ex : new IOException(ex); - } - } - - public void close() { - try { - secretManager.stopThreads(); - } catch (Exception ex) { - // Ignore. - } - } -} diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/security/LlapTokenLocalClientImpl.java b/llap-client/src/java/org/apache/hadoop/hive/llap/security/LlapTokenLocalClientImpl.java new file mode 100644 index 0000000000000000000000000000000000000000..78510b0cad1cb1fcd118ce8ba14610216b23ca76 --- /dev/null +++ b/llap-client/src/java/org/apache/hadoop/hive/llap/security/LlapTokenLocalClientImpl.java @@ -0,0 +1,61 @@ +/** + * 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.llap.security; + +import java.io.IOException; + + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.token.Token; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class LlapTokenLocalClientImpl implements LlapTokenLocalClient { + private static final Logger LOG = LoggerFactory.getLogger(LlapTokenLocalClientImpl.class); + private final SecretManager secretManager; + + public LlapTokenLocalClientImpl(Configuration conf, String clusterId) { + // TODO: create this centrally in HS2 case + secretManager = SecretManager.createSecretManager(conf, clusterId); + } + + @Override + public Token createToken( + String appId, String user, boolean isSignatureRequired) throws IOException { + try { + Token token = secretManager.createLlapToken( + appId, user, isSignatureRequired); + if (LOG.isInfoEnabled()) { + LOG.info("Created a LLAP delegation token locally: " + token); + } + return token; + } catch (Exception ex) { + throw (ex instanceof IOException) ? (IOException)ex : new IOException(ex); + } + } + + @Override + public void close() { + try { + secretManager.stopThreads(); + } catch (Exception ex) { + // Ignore. + } + } +} diff --git a/llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/daemon/rpc/LlapDaemonProtocolProtos.java b/llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/daemon/rpc/LlapDaemonProtocolProtos.java index 16531e3445a9bfbd55a788674fa9f4132ca6b697..856ea303ae819905da7a721bc676b211b0b5fca7 100644 --- a/llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/daemon/rpc/LlapDaemonProtocolProtos.java +++ b/llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/daemon/rpc/LlapDaemonProtocolProtos.java @@ -8978,6 +8978,905 @@ public Builder clearDagIdentifier() { // @@protoc_insertion_point(class_scope:QueryIdentifierProto) } + public interface NotTezEventOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required bytes input_event_proto_bytes = 1; + /** + * required bytes input_event_proto_bytes = 1; + */ + boolean hasInputEventProtoBytes(); + /** + * required bytes input_event_proto_bytes = 1; + */ + com.google.protobuf.ByteString getInputEventProtoBytes(); + + // required string vertex_name = 2; + /** + * required string vertex_name = 2; + */ + boolean hasVertexName(); + /** + * required string vertex_name = 2; + */ + java.lang.String getVertexName(); + /** + * required string vertex_name = 2; + */ + com.google.protobuf.ByteString + getVertexNameBytes(); + + // required string dest_input_name = 3; + /** + * required string dest_input_name = 3; + */ + boolean hasDestInputName(); + /** + * required string dest_input_name = 3; + */ + java.lang.String getDestInputName(); + /** + * required string dest_input_name = 3; + */ + com.google.protobuf.ByteString + getDestInputNameBytes(); + + // optional int32 key_id = 4; + /** + * optional int32 key_id = 4; + */ + boolean hasKeyId(); + /** + * optional int32 key_id = 4; + */ + int getKeyId(); + } + /** + * Protobuf type {@code NotTezEvent} + * + *
+   **
+   * Tez API implementation derives an enum value from instanceof on the event, then uses that enum
+   * in a giant switch statement to re-implement virtual functions. In addition the event classes
+   * are final and serialization is a mix of writables and protobuf. As a result, it is impossible
+   * three times over to add anything there. So, we'd do our own "inspired" serialization.
+   * Eventually we'll move away from events for API.
+   * 
+ */ + public static final class NotTezEvent extends + com.google.protobuf.GeneratedMessage + implements NotTezEventOrBuilder { + // Use NotTezEvent.newBuilder() to construct. + private NotTezEvent(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private NotTezEvent(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final NotTezEvent defaultInstance; + public static NotTezEvent getDefaultInstance() { + return defaultInstance; + } + + public NotTezEvent getDefaultInstanceForType() { + return defaultInstance; + } + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private NotTezEvent( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + bitField0_ |= 0x00000001; + inputEventProtoBytes_ = input.readBytes(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + vertexName_ = input.readBytes(); + break; + } + case 26: { + bitField0_ |= 0x00000004; + destInputName_ = input.readBytes(); + break; + } + case 32: { + bitField0_ |= 0x00000008; + keyId_ = input.readInt32(); + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_NotTezEvent_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_NotTezEvent_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent.class, org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public NotTezEvent parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new NotTezEvent(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // required bytes input_event_proto_bytes = 1; + public static final int INPUT_EVENT_PROTO_BYTES_FIELD_NUMBER = 1; + private com.google.protobuf.ByteString inputEventProtoBytes_; + /** + * required bytes input_event_proto_bytes = 1; + */ + public boolean hasInputEventProtoBytes() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required bytes input_event_proto_bytes = 1; + */ + public com.google.protobuf.ByteString getInputEventProtoBytes() { + return inputEventProtoBytes_; + } + + // required string vertex_name = 2; + public static final int VERTEX_NAME_FIELD_NUMBER = 2; + private java.lang.Object vertexName_; + /** + * required string vertex_name = 2; + */ + public boolean hasVertexName() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required string vertex_name = 2; + */ + public java.lang.String getVertexName() { + java.lang.Object ref = vertexName_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + vertexName_ = s; + } + return s; + } + } + /** + * required string vertex_name = 2; + */ + public com.google.protobuf.ByteString + getVertexNameBytes() { + java.lang.Object ref = vertexName_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + vertexName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + // required string dest_input_name = 3; + public static final int DEST_INPUT_NAME_FIELD_NUMBER = 3; + private java.lang.Object destInputName_; + /** + * required string dest_input_name = 3; + */ + public boolean hasDestInputName() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * required string dest_input_name = 3; + */ + public java.lang.String getDestInputName() { + java.lang.Object ref = destInputName_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + destInputName_ = s; + } + return s; + } + } + /** + * required string dest_input_name = 3; + */ + public com.google.protobuf.ByteString + getDestInputNameBytes() { + java.lang.Object ref = destInputName_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + destInputName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + // optional int32 key_id = 4; + public static final int KEY_ID_FIELD_NUMBER = 4; + private int keyId_; + /** + * optional int32 key_id = 4; + */ + public boolean hasKeyId() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional int32 key_id = 4; + */ + public int getKeyId() { + return keyId_; + } + + private void initFields() { + inputEventProtoBytes_ = com.google.protobuf.ByteString.EMPTY; + vertexName_ = ""; + destInputName_ = ""; + keyId_ = 0; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasInputEventProtoBytes()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasVertexName()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasDestInputName()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, inputEventProtoBytes_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, getVertexNameBytes()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBytes(3, getDestInputNameBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeInt32(4, keyId_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(1, inputEventProtoBytes_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(2, getVertexNameBytes()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(3, getDestInputNameBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += com.google.protobuf.CodedOutputStream + .computeInt32Size(4, keyId_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent)) { + return super.equals(obj); + } + org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent other = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent) obj; + + boolean result = true; + result = result && (hasInputEventProtoBytes() == other.hasInputEventProtoBytes()); + if (hasInputEventProtoBytes()) { + result = result && getInputEventProtoBytes() + .equals(other.getInputEventProtoBytes()); + } + result = result && (hasVertexName() == other.hasVertexName()); + if (hasVertexName()) { + result = result && getVertexName() + .equals(other.getVertexName()); + } + result = result && (hasDestInputName() == other.hasDestInputName()); + if (hasDestInputName()) { + result = result && getDestInputName() + .equals(other.getDestInputName()); + } + result = result && (hasKeyId() == other.hasKeyId()); + if (hasKeyId()) { + result = result && (getKeyId() + == other.getKeyId()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + private int memoizedHashCode = 0; + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasInputEventProtoBytes()) { + hash = (37 * hash) + INPUT_EVENT_PROTO_BYTES_FIELD_NUMBER; + hash = (53 * hash) + getInputEventProtoBytes().hashCode(); + } + if (hasVertexName()) { + hash = (37 * hash) + VERTEX_NAME_FIELD_NUMBER; + hash = (53 * hash) + getVertexName().hashCode(); + } + if (hasDestInputName()) { + hash = (37 * hash) + DEST_INPUT_NAME_FIELD_NUMBER; + hash = (53 * hash) + getDestInputName().hashCode(); + } + if (hasKeyId()) { + hash = (37 * hash) + KEY_ID_FIELD_NUMBER; + hash = (53 * hash) + getKeyId(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code NotTezEvent} + * + *
+     **
+     * Tez API implementation derives an enum value from instanceof on the event, then uses that enum
+     * in a giant switch statement to re-implement virtual functions. In addition the event classes
+     * are final and serialization is a mix of writables and protobuf. As a result, it is impossible
+     * three times over to add anything there. So, we'd do our own "inspired" serialization.
+     * Eventually we'll move away from events for API.
+     * 
+ */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEventOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_NotTezEvent_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_NotTezEvent_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent.class, org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent.Builder.class); + } + + // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + inputEventProtoBytes_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000001); + vertexName_ = ""; + bitField0_ = (bitField0_ & ~0x00000002); + destInputName_ = ""; + bitField0_ = (bitField0_ & ~0x00000004); + keyId_ = 0; + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_NotTezEvent_descriptor; + } + + public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent getDefaultInstanceForType() { + return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent.getDefaultInstance(); + } + + public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent build() { + org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent buildPartial() { + org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent result = new org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.inputEventProtoBytes_ = inputEventProtoBytes_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.vertexName_ = vertexName_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.destInputName_ = destInputName_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.keyId_ = keyId_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent) { + return mergeFrom((org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent other) { + if (other == org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent.getDefaultInstance()) return this; + if (other.hasInputEventProtoBytes()) { + setInputEventProtoBytes(other.getInputEventProtoBytes()); + } + if (other.hasVertexName()) { + bitField0_ |= 0x00000002; + vertexName_ = other.vertexName_; + onChanged(); + } + if (other.hasDestInputName()) { + bitField0_ |= 0x00000004; + destInputName_ = other.destInputName_; + onChanged(); + } + if (other.hasKeyId()) { + setKeyId(other.getKeyId()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasInputEventProtoBytes()) { + + return false; + } + if (!hasVertexName()) { + + return false; + } + if (!hasDestInputName()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required bytes input_event_proto_bytes = 1; + private com.google.protobuf.ByteString inputEventProtoBytes_ = com.google.protobuf.ByteString.EMPTY; + /** + * required bytes input_event_proto_bytes = 1; + */ + public boolean hasInputEventProtoBytes() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required bytes input_event_proto_bytes = 1; + */ + public com.google.protobuf.ByteString getInputEventProtoBytes() { + return inputEventProtoBytes_; + } + /** + * required bytes input_event_proto_bytes = 1; + */ + public Builder setInputEventProtoBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + inputEventProtoBytes_ = value; + onChanged(); + return this; + } + /** + * required bytes input_event_proto_bytes = 1; + */ + public Builder clearInputEventProtoBytes() { + bitField0_ = (bitField0_ & ~0x00000001); + inputEventProtoBytes_ = getDefaultInstance().getInputEventProtoBytes(); + onChanged(); + return this; + } + + // required string vertex_name = 2; + private java.lang.Object vertexName_ = ""; + /** + * required string vertex_name = 2; + */ + public boolean hasVertexName() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required string vertex_name = 2; + */ + public java.lang.String getVertexName() { + java.lang.Object ref = vertexName_; + if (!(ref instanceof java.lang.String)) { + java.lang.String s = ((com.google.protobuf.ByteString) ref) + .toStringUtf8(); + vertexName_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * required string vertex_name = 2; + */ + public com.google.protobuf.ByteString + getVertexNameBytes() { + java.lang.Object ref = vertexName_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + vertexName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * required string vertex_name = 2; + */ + public Builder setVertexName( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + vertexName_ = value; + onChanged(); + return this; + } + /** + * required string vertex_name = 2; + */ + public Builder clearVertexName() { + bitField0_ = (bitField0_ & ~0x00000002); + vertexName_ = getDefaultInstance().getVertexName(); + onChanged(); + return this; + } + /** + * required string vertex_name = 2; + */ + public Builder setVertexNameBytes( + com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + vertexName_ = value; + onChanged(); + return this; + } + + // required string dest_input_name = 3; + private java.lang.Object destInputName_ = ""; + /** + * required string dest_input_name = 3; + */ + public boolean hasDestInputName() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * required string dest_input_name = 3; + */ + public java.lang.String getDestInputName() { + java.lang.Object ref = destInputName_; + if (!(ref instanceof java.lang.String)) { + java.lang.String s = ((com.google.protobuf.ByteString) ref) + .toStringUtf8(); + destInputName_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * required string dest_input_name = 3; + */ + public com.google.protobuf.ByteString + getDestInputNameBytes() { + java.lang.Object ref = destInputName_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + destInputName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * required string dest_input_name = 3; + */ + public Builder setDestInputName( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + destInputName_ = value; + onChanged(); + return this; + } + /** + * required string dest_input_name = 3; + */ + public Builder clearDestInputName() { + bitField0_ = (bitField0_ & ~0x00000004); + destInputName_ = getDefaultInstance().getDestInputName(); + onChanged(); + return this; + } + /** + * required string dest_input_name = 3; + */ + public Builder setDestInputNameBytes( + com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + destInputName_ = value; + onChanged(); + return this; + } + + // optional int32 key_id = 4; + private int keyId_ ; + /** + * optional int32 key_id = 4; + */ + public boolean hasKeyId() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional int32 key_id = 4; + */ + public int getKeyId() { + return keyId_; + } + /** + * optional int32 key_id = 4; + */ + public Builder setKeyId(int value) { + bitField0_ |= 0x00000008; + keyId_ = value; + onChanged(); + return this; + } + /** + * optional int32 key_id = 4; + */ + public Builder clearKeyId() { + bitField0_ = (bitField0_ & ~0x00000008); + keyId_ = 0; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:NotTezEvent) + } + + static { + defaultInstance = new NotTezEvent(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:NotTezEvent) + } + public interface SubmitWorkRequestProtoOrBuilder extends com.google.protobuf.MessageOrBuilder { @@ -9108,6 +10007,34 @@ public Builder clearDagIdentifier() { * */ org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentRuntimeInfoOrBuilder getFragmentRuntimeInfoOrBuilder(); + + // optional bytes initial_event_bytes = 10; + /** + * optional bytes initial_event_bytes = 10; + * + *
+     * Serialized (and signed) NotTezEvent; used only for external clients for now.
+     * 
+ */ + boolean hasInitialEventBytes(); + /** + * optional bytes initial_event_bytes = 10; + * + *
+     * Serialized (and signed) NotTezEvent; used only for external clients for now.
+     * 
+ */ + com.google.protobuf.ByteString getInitialEventBytes(); + + // optional bytes initial_event_signature = 11; + /** + * optional bytes initial_event_signature = 11; + */ + boolean hasInitialEventSignature(); + /** + * optional bytes initial_event_signature = 11; + */ + com.google.protobuf.ByteString getInitialEventSignature(); } /** * Protobuf type {@code SubmitWorkRequestProto} @@ -9221,6 +10148,16 @@ private SubmitWorkRequestProto( bitField0_ |= 0x00000100; break; } + case 82: { + bitField0_ |= 0x00000200; + initialEventBytes_ = input.readBytes(); + break; + } + case 90: { + bitField0_ |= 0x00000400; + initialEventSignature_ = input.readBytes(); + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -9491,6 +10428,46 @@ public boolean hasFragmentRuntimeInfo() { return fragmentRuntimeInfo_; } + // optional bytes initial_event_bytes = 10; + public static final int INITIAL_EVENT_BYTES_FIELD_NUMBER = 10; + private com.google.protobuf.ByteString initialEventBytes_; + /** + * optional bytes initial_event_bytes = 10; + * + *
+     * Serialized (and signed) NotTezEvent; used only for external clients for now.
+     * 
+ */ + public boolean hasInitialEventBytes() { + return ((bitField0_ & 0x00000200) == 0x00000200); + } + /** + * optional bytes initial_event_bytes = 10; + * + *
+     * Serialized (and signed) NotTezEvent; used only for external clients for now.
+     * 
+ */ + public com.google.protobuf.ByteString getInitialEventBytes() { + return initialEventBytes_; + } + + // optional bytes initial_event_signature = 11; + public static final int INITIAL_EVENT_SIGNATURE_FIELD_NUMBER = 11; + private com.google.protobuf.ByteString initialEventSignature_; + /** + * optional bytes initial_event_signature = 11; + */ + public boolean hasInitialEventSignature() { + return ((bitField0_ & 0x00000400) == 0x00000400); + } + /** + * optional bytes initial_event_signature = 11; + */ + public com.google.protobuf.ByteString getInitialEventSignature() { + return initialEventSignature_; + } + private void initFields() { workSpec_ = org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexOrBinary.getDefaultInstance(); workSpecSignature_ = com.google.protobuf.ByteString.EMPTY; @@ -9501,6 +10478,8 @@ private void initFields() { amPort_ = 0; credentialsBinary_ = com.google.protobuf.ByteString.EMPTY; fragmentRuntimeInfo_ = org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentRuntimeInfo.getDefaultInstance(); + initialEventBytes_ = com.google.protobuf.ByteString.EMPTY; + initialEventSignature_ = com.google.protobuf.ByteString.EMPTY; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -9541,6 +10520,12 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00000100) == 0x00000100)) { output.writeMessage(9, fragmentRuntimeInfo_); } + if (((bitField0_ & 0x00000200) == 0x00000200)) { + output.writeBytes(10, initialEventBytes_); + } + if (((bitField0_ & 0x00000400) == 0x00000400)) { + output.writeBytes(11, initialEventSignature_); + } getUnknownFields().writeTo(output); } @@ -9586,6 +10571,14 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeMessageSize(9, fragmentRuntimeInfo_); } + if (((bitField0_ & 0x00000200) == 0x00000200)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(10, initialEventBytes_); + } + if (((bitField0_ & 0x00000400) == 0x00000400)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(11, initialEventSignature_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -9654,6 +10647,16 @@ public boolean equals(final java.lang.Object obj) { result = result && getFragmentRuntimeInfo() .equals(other.getFragmentRuntimeInfo()); } + result = result && (hasInitialEventBytes() == other.hasInitialEventBytes()); + if (hasInitialEventBytes()) { + result = result && getInitialEventBytes() + .equals(other.getInitialEventBytes()); + } + result = result && (hasInitialEventSignature() == other.hasInitialEventSignature()); + if (hasInitialEventSignature()) { + result = result && getInitialEventSignature() + .equals(other.getInitialEventSignature()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -9703,6 +10706,14 @@ public int hashCode() { hash = (37 * hash) + FRAGMENT_RUNTIME_INFO_FIELD_NUMBER; hash = (53 * hash) + getFragmentRuntimeInfo().hashCode(); } + if (hasInitialEventBytes()) { + hash = (37 * hash) + INITIAL_EVENT_BYTES_FIELD_NUMBER; + hash = (53 * hash) + getInitialEventBytes().hashCode(); + } + if (hasInitialEventSignature()) { + hash = (37 * hash) + INITIAL_EVENT_SIGNATURE_FIELD_NUMBER; + hash = (53 * hash) + getInitialEventSignature().hashCode(); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -9840,6 +10851,10 @@ public Builder clear() { fragmentRuntimeInfoBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000100); + initialEventBytes_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000200); + initialEventSignature_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000400); return this; } @@ -9912,6 +10927,14 @@ public Builder clone() { } else { result.fragmentRuntimeInfo_ = fragmentRuntimeInfoBuilder_.build(); } + if (((from_bitField0_ & 0x00000200) == 0x00000200)) { + to_bitField0_ |= 0x00000200; + } + result.initialEventBytes_ = initialEventBytes_; + if (((from_bitField0_ & 0x00000400) == 0x00000400)) { + to_bitField0_ |= 0x00000400; + } + result.initialEventSignature_ = initialEventSignature_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -9959,6 +10982,12 @@ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtoc if (other.hasFragmentRuntimeInfo()) { mergeFragmentRuntimeInfo(other.getFragmentRuntimeInfo()); } + if (other.hasInitialEventBytes()) { + setInitialEventBytes(other.getInitialEventBytes()); + } + if (other.hasInitialEventSignature()) { + setInitialEventSignature(other.getInitialEventSignature()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -10591,6 +11620,94 @@ public Builder clearFragmentRuntimeInfo() { return fragmentRuntimeInfoBuilder_; } + // optional bytes initial_event_bytes = 10; + private com.google.protobuf.ByteString initialEventBytes_ = com.google.protobuf.ByteString.EMPTY; + /** + * optional bytes initial_event_bytes = 10; + * + *
+       * Serialized (and signed) NotTezEvent; used only for external clients for now.
+       * 
+ */ + public boolean hasInitialEventBytes() { + return ((bitField0_ & 0x00000200) == 0x00000200); + } + /** + * optional bytes initial_event_bytes = 10; + * + *
+       * Serialized (and signed) NotTezEvent; used only for external clients for now.
+       * 
+ */ + public com.google.protobuf.ByteString getInitialEventBytes() { + return initialEventBytes_; + } + /** + * optional bytes initial_event_bytes = 10; + * + *
+       * Serialized (and signed) NotTezEvent; used only for external clients for now.
+       * 
+ */ + public Builder setInitialEventBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000200; + initialEventBytes_ = value; + onChanged(); + return this; + } + /** + * optional bytes initial_event_bytes = 10; + * + *
+       * Serialized (and signed) NotTezEvent; used only for external clients for now.
+       * 
+ */ + public Builder clearInitialEventBytes() { + bitField0_ = (bitField0_ & ~0x00000200); + initialEventBytes_ = getDefaultInstance().getInitialEventBytes(); + onChanged(); + return this; + } + + // optional bytes initial_event_signature = 11; + private com.google.protobuf.ByteString initialEventSignature_ = com.google.protobuf.ByteString.EMPTY; + /** + * optional bytes initial_event_signature = 11; + */ + public boolean hasInitialEventSignature() { + return ((bitField0_ & 0x00000400) == 0x00000400); + } + /** + * optional bytes initial_event_signature = 11; + */ + public com.google.protobuf.ByteString getInitialEventSignature() { + return initialEventSignature_; + } + /** + * optional bytes initial_event_signature = 11; + */ + public Builder setInitialEventSignature(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000400; + initialEventSignature_ = value; + onChanged(); + return this; + } + /** + * optional bytes initial_event_signature = 11; + */ + public Builder clearInitialEventSignature() { + bitField0_ = (bitField0_ & ~0x00000400); + initialEventSignature_ = getDefaultInstance().getInitialEventSignature(); + onChanged(); + return this; + } + // @@protoc_insertion_point(builder_scope:SubmitWorkRequestProto) } @@ -16047,6 +17164,11 @@ private BlockingStub(com.google.protobuf.BlockingRpcChannel channel) { com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_QueryIdentifierProto_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor + internal_static_NotTezEvent_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_NotTezEvent_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor internal_static_SubmitWorkRequestProto_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable @@ -16137,46 +17259,50 @@ private BlockingStub(com.google.protobuf.BlockingRpcChannel channel) { "\030\004 \001(\003\022 \n\030first_attempt_start_time\030\005 \001(\003", "\022\"\n\032current_attempt_start_time\030\006 \001(\003\"F\n\024" + "QueryIdentifierProto\022\026\n\016app_identifier\030\001" + - " \001(\t\022\026\n\016dag_identifier\030\002 \001(\005\"\232\002\n\026SubmitW" + - "orkRequestProto\022\"\n\twork_spec\030\001 \001(\0132\017.Ver" + - "texOrBinary\022\033\n\023work_spec_signature\030\002 \001(\014" + - "\022\027\n\017fragment_number\030\003 \001(\005\022\026\n\016attempt_num" + - "ber\030\004 \001(\005\022\033\n\023container_id_string\030\005 \001(\t\022\017" + - "\n\007am_host\030\006 \001(\t\022\017\n\007am_port\030\007 \001(\005\022\032\n\022cred" + - "entials_binary\030\010 \001(\014\0223\n\025fragment_runtime" + - "_info\030\t \001(\0132\024.FragmentRuntimeInfo\"J\n\027Sub", - "mitWorkResponseProto\022/\n\020submission_state" + - "\030\001 \001(\0162\025.SubmissionStateProto\"\205\001\n\036Source" + - "StateUpdatedRequestProto\022/\n\020query_identi" + - "fier\030\001 \001(\0132\025.QueryIdentifierProto\022\020\n\010src" + - "_name\030\002 \001(\t\022 \n\005state\030\003 \001(\0162\021.SourceState" + - "Proto\"!\n\037SourceStateUpdatedResponseProto" + - "\"w\n\031QueryCompleteRequestProto\022\020\n\010query_i" + - "d\030\001 \001(\t\022/\n\020query_identifier\030\002 \001(\0132\025.Quer" + - "yIdentifierProto\022\027\n\014delete_delay\030\004 \001(\003:\001" + - "0\"\034\n\032QueryCompleteResponseProto\"t\n\035Termi", - "nateFragmentRequestProto\022/\n\020query_identi" + - "fier\030\001 \001(\0132\025.QueryIdentifierProto\022\"\n\032fra" + - "gment_identifier_string\030\002 \001(\t\" \n\036Termina" + - "teFragmentResponseProto\"&\n\024GetTokenReque" + - "stProto\022\016\n\006app_id\030\001 \001(\t\"&\n\025GetTokenRespo" + - "nseProto\022\r\n\005token\030\001 \001(\014*2\n\020SourceStatePr" + - "oto\022\017\n\013S_SUCCEEDED\020\001\022\r\n\tS_RUNNING\020\002*E\n\024S" + - "ubmissionStateProto\022\014\n\010ACCEPTED\020\001\022\014\n\010REJ" + - "ECTED\020\002\022\021\n\rEVICTED_OTHER\020\0032\316\002\n\022LlapDaemo" + - "nProtocol\022?\n\nsubmitWork\022\027.SubmitWorkRequ", - "estProto\032\030.SubmitWorkResponseProto\022W\n\022so" + - "urceStateUpdated\022\037.SourceStateUpdatedReq" + - "uestProto\032 .SourceStateUpdatedResponsePr" + - "oto\022H\n\rqueryComplete\022\032.QueryCompleteRequ" + - "estProto\032\033.QueryCompleteResponseProto\022T\n" + - "\021terminateFragment\022\036.TerminateFragmentRe" + - "questProto\032\037.TerminateFragmentResponsePr" + - "oto2]\n\026LlapManagementProtocol\022C\n\022getDele" + - "gationToken\022\025.GetTokenRequestProto\032\026.Get" + - "TokenResponseProtoBH\n&org.apache.hadoop.", - "hive.llap.daemon.rpcB\030LlapDaemonProtocol" + - "Protos\210\001\001\240\001\001" + " \001(\t\022\026\n\016dag_identifier\030\002 \001(\005\"l\n\013NotTezEv" + + "ent\022\037\n\027input_event_proto_bytes\030\001 \002(\014\022\023\n\013" + + "vertex_name\030\002 \002(\t\022\027\n\017dest_input_name\030\003 \002" + + "(\t\022\016\n\006key_id\030\004 \001(\005\"\330\002\n\026SubmitWorkRequest" + + "Proto\022\"\n\twork_spec\030\001 \001(\0132\017.VertexOrBinar" + + "y\022\033\n\023work_spec_signature\030\002 \001(\014\022\027\n\017fragme" + + "nt_number\030\003 \001(\005\022\026\n\016attempt_number\030\004 \001(\005\022" + + "\033\n\023container_id_string\030\005 \001(\t\022\017\n\007am_host\030", + "\006 \001(\t\022\017\n\007am_port\030\007 \001(\005\022\032\n\022credentials_bi" + + "nary\030\010 \001(\014\0223\n\025fragment_runtime_info\030\t \001(" + + "\0132\024.FragmentRuntimeInfo\022\033\n\023initial_event" + + "_bytes\030\n \001(\014\022\037\n\027initial_event_signature\030" + + "\013 \001(\014\"J\n\027SubmitWorkResponseProto\022/\n\020subm" + + "ission_state\030\001 \001(\0162\025.SubmissionStateProt" + + "o\"\205\001\n\036SourceStateUpdatedRequestProto\022/\n\020" + + "query_identifier\030\001 \001(\0132\025.QueryIdentifier" + + "Proto\022\020\n\010src_name\030\002 \001(\t\022 \n\005state\030\003 \001(\0162\021" + + ".SourceStateProto\"!\n\037SourceStateUpdatedR", + "esponseProto\"w\n\031QueryCompleteRequestProt" + + "o\022\020\n\010query_id\030\001 \001(\t\022/\n\020query_identifier\030" + + "\002 \001(\0132\025.QueryIdentifierProto\022\027\n\014delete_d" + + "elay\030\004 \001(\003:\0010\"\034\n\032QueryCompleteResponsePr" + + "oto\"t\n\035TerminateFragmentRequestProto\022/\n\020" + + "query_identifier\030\001 \001(\0132\025.QueryIdentifier" + + "Proto\022\"\n\032fragment_identifier_string\030\002 \001(" + + "\t\" \n\036TerminateFragmentResponseProto\"&\n\024G" + + "etTokenRequestProto\022\016\n\006app_id\030\001 \001(\t\"&\n\025G" + + "etTokenResponseProto\022\r\n\005token\030\001 \001(\014*2\n\020S", + "ourceStateProto\022\017\n\013S_SUCCEEDED\020\001\022\r\n\tS_RU" + + "NNING\020\002*E\n\024SubmissionStateProto\022\014\n\010ACCEP" + + "TED\020\001\022\014\n\010REJECTED\020\002\022\021\n\rEVICTED_OTHER\020\0032\316" + + "\002\n\022LlapDaemonProtocol\022?\n\nsubmitWork\022\027.Su" + + "bmitWorkRequestProto\032\030.SubmitWorkRespons" + + "eProto\022W\n\022sourceStateUpdated\022\037.SourceSta" + + "teUpdatedRequestProto\032 .SourceStateUpdat" + + "edResponseProto\022H\n\rqueryComplete\022\032.Query" + + "CompleteRequestProto\032\033.QueryCompleteResp" + + "onseProto\022T\n\021terminateFragment\022\036.Termina", + "teFragmentRequestProto\032\037.TerminateFragme" + + "ntResponseProto2]\n\026LlapManagementProtoco" + + "l\022C\n\022getDelegationToken\022\025.GetTokenReques" + + "tProto\032\026.GetTokenResponseProtoBH\n&org.ap" + + "ache.hadoop.hive.llap.daemon.rpcB\030LlapDa" + + "emonProtocolProtos\210\001\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -16237,62 +17363,68 @@ private BlockingStub(com.google.protobuf.BlockingRpcChannel channel) { com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_QueryIdentifierProto_descriptor, new java.lang.String[] { "AppIdentifier", "DagIdentifier", }); - internal_static_SubmitWorkRequestProto_descriptor = + internal_static_NotTezEvent_descriptor = getDescriptor().getMessageTypes().get(9); + internal_static_NotTezEvent_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_NotTezEvent_descriptor, + new java.lang.String[] { "InputEventProtoBytes", "VertexName", "DestInputName", "KeyId", }); + internal_static_SubmitWorkRequestProto_descriptor = + getDescriptor().getMessageTypes().get(10); internal_static_SubmitWorkRequestProto_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_SubmitWorkRequestProto_descriptor, - new java.lang.String[] { "WorkSpec", "WorkSpecSignature", "FragmentNumber", "AttemptNumber", "ContainerIdString", "AmHost", "AmPort", "CredentialsBinary", "FragmentRuntimeInfo", }); + new java.lang.String[] { "WorkSpec", "WorkSpecSignature", "FragmentNumber", "AttemptNumber", "ContainerIdString", "AmHost", "AmPort", "CredentialsBinary", "FragmentRuntimeInfo", "InitialEventBytes", "InitialEventSignature", }); internal_static_SubmitWorkResponseProto_descriptor = - getDescriptor().getMessageTypes().get(10); + getDescriptor().getMessageTypes().get(11); internal_static_SubmitWorkResponseProto_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_SubmitWorkResponseProto_descriptor, new java.lang.String[] { "SubmissionState", }); internal_static_SourceStateUpdatedRequestProto_descriptor = - getDescriptor().getMessageTypes().get(11); + getDescriptor().getMessageTypes().get(12); internal_static_SourceStateUpdatedRequestProto_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_SourceStateUpdatedRequestProto_descriptor, new java.lang.String[] { "QueryIdentifier", "SrcName", "State", }); internal_static_SourceStateUpdatedResponseProto_descriptor = - getDescriptor().getMessageTypes().get(12); + getDescriptor().getMessageTypes().get(13); internal_static_SourceStateUpdatedResponseProto_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_SourceStateUpdatedResponseProto_descriptor, new java.lang.String[] { }); internal_static_QueryCompleteRequestProto_descriptor = - getDescriptor().getMessageTypes().get(13); + getDescriptor().getMessageTypes().get(14); internal_static_QueryCompleteRequestProto_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_QueryCompleteRequestProto_descriptor, new java.lang.String[] { "QueryId", "QueryIdentifier", "DeleteDelay", }); internal_static_QueryCompleteResponseProto_descriptor = - getDescriptor().getMessageTypes().get(14); + getDescriptor().getMessageTypes().get(15); internal_static_QueryCompleteResponseProto_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_QueryCompleteResponseProto_descriptor, new java.lang.String[] { }); internal_static_TerminateFragmentRequestProto_descriptor = - getDescriptor().getMessageTypes().get(15); + getDescriptor().getMessageTypes().get(16); internal_static_TerminateFragmentRequestProto_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_TerminateFragmentRequestProto_descriptor, new java.lang.String[] { "QueryIdentifier", "FragmentIdentifierString", }); internal_static_TerminateFragmentResponseProto_descriptor = - getDescriptor().getMessageTypes().get(16); + getDescriptor().getMessageTypes().get(17); internal_static_TerminateFragmentResponseProto_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_TerminateFragmentResponseProto_descriptor, new java.lang.String[] { }); internal_static_GetTokenRequestProto_descriptor = - getDescriptor().getMessageTypes().get(17); + getDescriptor().getMessageTypes().get(18); internal_static_GetTokenRequestProto_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_GetTokenRequestProto_descriptor, new java.lang.String[] { "AppId", }); internal_static_GetTokenResponseProto_descriptor = - getDescriptor().getMessageTypes().get(18); + getDescriptor().getMessageTypes().get(19); internal_static_GetTokenResponseProto_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_GetTokenResponseProto_descriptor, diff --git a/llap-common/src/java/org/apache/hadoop/hive/llap/NotTezEventHelper.java b/llap-common/src/java/org/apache/hadoop/hive/llap/NotTezEventHelper.java new file mode 100644 index 0000000000000000000000000000000000000000..179e96c6605e839b0695c59afb07f06364beceae --- /dev/null +++ b/llap-common/src/java/org/apache/hadoop/hive/llap/NotTezEventHelper.java @@ -0,0 +1,71 @@ +/** + * 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.llap; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; + +import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent; +import org.apache.hadoop.hive.llap.security.LlapSigner.Signable; +import org.apache.tez.common.ProtoConverters; +import org.apache.tez.runtime.api.events.InputDataInformationEvent; +import org.apache.tez.runtime.api.events.EventProtos.RootInputDataInformationEventProto; +import org.apache.tez.runtime.api.impl.EventMetaData; +import org.apache.tez.runtime.api.impl.TezEvent; + +import com.google.protobuf.InvalidProtocolBufferException; + +/** + * See NotTezEvent class/.proto comment. + */ +public class NotTezEventHelper { + + public static Signable createSignableNotTezEvent( + InputDataInformationEvent event, String vertexName, String destInputName) { + final NotTezEvent.Builder builder = NotTezEvent.newBuilder().setInputEventProtoBytes( + ProtoConverters.convertRootInputDataInformationEventToProto(event).toByteString()) + .setVertexName(vertexName).setDestInputName(destInputName); + return new Signable() { + @Override + public void setSignInfo(int masterKeyId) { + builder.setKeyId(masterKeyId); + } + + @Override + public byte[] serialize() throws IOException { + NotTezEvent nte = builder.build(); + ByteArrayOutputStream baos = new ByteArrayOutputStream(nte.getSerializedSize()); + nte.writeTo(baos); + return baos.toByteArray(); + } + }; + } + + public static TezEvent toTezEvent(NotTezEvent nte) throws InvalidProtocolBufferException { + EventMetaData sourceMetaData = new EventMetaData(EventMetaData.EventProducerConsumerType.INPUT, + nte.getVertexName(), "NULL_VERTEX", null); + EventMetaData destMetaData = new EventMetaData(EventMetaData.EventProducerConsumerType.INPUT, + nte.getVertexName(), nte.getDestInputName(), null); + InputDataInformationEvent event = ProtoConverters.convertRootInputDataInformationEventFromProto( + RootInputDataInformationEventProto.parseFrom(nte.getInputEventProtoBytes())); + TezEvent tezEvent = new TezEvent(event, sourceMetaData, System.currentTimeMillis()); + tezEvent.setDestinationInfo(destMetaData); + return tezEvent; + } +} \ No newline at end of file diff --git a/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapSigner.java b/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapSigner.java index 478a40a4edded3f911df636eda655be2c0516ba3..7b9c8ccb5bb06b68b8b937c6ec8e00fe3d813a2d 100644 --- a/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapSigner.java +++ b/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapSigner.java @@ -24,7 +24,7 @@ /** An object signable by a signer. */ public interface Signable { /** Called by the signer to record key information as part of the message to be signed. */ - void setSignInfo(int masterKeyId, String user); + void setSignInfo(int masterKeyId); /** Called by the signer to get the serialized representation of the message to be signed. */ byte[] serialize() throws IOException; } @@ -38,4 +38,6 @@ SignedMessage serializeAndSign(Signable message) throws IOException; void checkSignature(byte[] message, byte[] signature, int keyId); + + void close(); } \ No newline at end of file diff --git a/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapSignerImpl.java b/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapSignerImpl.java new file mode 100644 index 0000000000000000000000000000000000000000..7f105058ec5ec65ff35d02b5133aa226b0fd8c01 --- /dev/null +++ b/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapSignerImpl.java @@ -0,0 +1,73 @@ +/** + * 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.llap.security; + +import java.io.IOException; +import java.util.Arrays; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.token.delegation.DelegationKey; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.annotations.VisibleForTesting; + +public class LlapSignerImpl implements LlapSigner { + private static final Logger LOG = LoggerFactory.getLogger(LlapSignerImpl.class); + + private final SigningSecretManager secretManager; + + public LlapSignerImpl(Configuration conf, String clusterId) { + // TODO: create this centrally in HS2 case + assert UserGroupInformation.isSecurityEnabled(); + secretManager = SecretManager.createSecretManager(conf, clusterId); + } + + @VisibleForTesting + public LlapSignerImpl(SigningSecretManager sm) { + secretManager = sm; + } + + @Override + public SignedMessage serializeAndSign(Signable message) throws IOException { + SignedMessage result = new SignedMessage(); + DelegationKey key = secretManager.getCurrentKey(); + message.setSignInfo(key.getKeyId()); + result.message = message.serialize(); + result.signature = secretManager.signWithKey(result.message, key); + return result; + } + + @Override + public void checkSignature(byte[] message, byte[] signature, int keyId) + throws SecurityException { + byte[] expectedSignature = secretManager.signWithKey(message, keyId); + if (Arrays.equals(signature, expectedSignature)) return; + throw new SecurityException("Message signature does not match"); + } + + @Override + public void close() { + try { + secretManager.close(); + } catch (Exception ex) { + LOG.error("Error closing the signer", ex); + } + } +} diff --git a/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapTokenLocalClient.java b/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapTokenLocalClient.java new file mode 100644 index 0000000000000000000000000000000000000000..fd0965230722fb04e800dfec2e2a9740185a3bd9 --- /dev/null +++ b/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapTokenLocalClient.java @@ -0,0 +1,30 @@ +/** + * 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.llap.security; + +import java.io.IOException; + +import org.apache.hadoop.security.token.Token; + +public interface LlapTokenLocalClient { + Token createToken( + String appId, String user, boolean isSignatureRequired) throws IOException; + + void close(); +} \ No newline at end of file diff --git a/llap-common/src/java/org/apache/hadoop/hive/llap/security/SecretManager.java b/llap-common/src/java/org/apache/hadoop/hive/llap/security/SecretManager.java index 540f978fd6244050688c6cc767d20716186410d3..58a8e96d35ce3ad1ea616a4aac4ed69ac29b7c4f 100644 --- a/llap-common/src/java/org/apache/hadoop/hive/llap/security/SecretManager.java +++ b/llap-common/src/java/org/apache/hadoop/hive/llap/security/SecretManager.java @@ -179,6 +179,7 @@ public static SecretManager createSecretManager(final Configuration conf, String public static SecretManager createSecretManager( final Configuration conf, String llapPrincipal, String llapKeytab, final String clusterId) { + assert UserGroupInformation.isSecurityEnabled(); final LlapZkConf c = createLlapZkConf(conf, llapPrincipal, llapKeytab, clusterId); return c.zkUgi.doAs(new PrivilegedAction() { @Override @@ -222,6 +223,11 @@ private static void setZkConfIfNotSet(Configuration zkConf, String name, String return token; } + @Override + public void close() { + stopThreads(); + } + 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); diff --git a/llap-common/src/java/org/apache/hadoop/hive/llap/security/SigningSecretManager.java b/llap-common/src/java/org/apache/hadoop/hive/llap/security/SigningSecretManager.java index 067a98eaab81a69ead61c45595da0f3b89936b4e..82b199232c48e7014f3f50821eaa59a3362c1a4f 100644 --- a/llap-common/src/java/org/apache/hadoop/hive/llap/security/SigningSecretManager.java +++ b/llap-common/src/java/org/apache/hadoop/hive/llap/security/SigningSecretManager.java @@ -23,4 +23,5 @@ DelegationKey getCurrentKey(); byte[] signWithKey(byte[] message, DelegationKey key); byte[] signWithKey(byte[] message, int keyId) throws SecurityException; + void close(); } \ No newline at end of file diff --git a/llap-common/src/java/org/apache/hadoop/hive/llap/tez/Converters.java b/llap-common/src/java/org/apache/hadoop/hive/llap/tez/Converters.java index e43b72b5e5f7e78168fc6b2b9c48cabad12c035f..dad5e07e58d14558b470e97d5946ac0141cc4a86 100644 --- a/llap-common/src/java/org/apache/hadoop/hive/llap/tez/Converters.java +++ b/llap-common/src/java/org/apache/hadoop/hive/llap/tez/Converters.java @@ -114,8 +114,8 @@ public static VertexIdentifier createVertexIdentifier( return idBuilder.build(); } - public static SignableVertexSpec convertTaskSpecToProto(TaskSpec taskSpec, - int appAttemptId, String tokenIdentifier, Integer signatureKeyId, String user) { + public static SignableVertexSpec.Builder convertTaskSpecToProto(TaskSpec taskSpec, + int appAttemptId, String tokenIdentifier, String user) { TezTaskAttemptID tId = taskSpec.getTaskAttemptID(); SignableVertexSpec.Builder builder = SignableVertexSpec.newBuilder(); @@ -125,9 +125,6 @@ public static SignableVertexSpec convertTaskSpecToProto(TaskSpec taskSpec, builder.setVertexParallelism(taskSpec.getVertexParallelism()); builder.setTokenIdentifier(tokenIdentifier); builder.setUser(user); - if (signatureKeyId != null) { - builder.setSignatureKeyId(signatureKeyId); - } if (taskSpec.getProcessorDescriptor() != null) { builder.setProcessorDescriptor( @@ -152,7 +149,7 @@ public static SignableVertexSpec convertTaskSpecToProto(TaskSpec taskSpec, } } - return builder.build(); + return builder; } private static ProcessorDescriptor convertProcessorDescriptorFromProto( diff --git a/llap-common/src/protobuf/LlapDaemonProtocol.proto b/llap-common/src/protobuf/LlapDaemonProtocol.proto index 486ba0a19483663b2dec622e6c52c45b12aa8512..45d18086560463125d2397bda95d11a412cbc8f0 100644 --- a/llap-common/src/protobuf/LlapDaemonProtocol.proto +++ b/llap-common/src/protobuf/LlapDaemonProtocol.proto @@ -99,6 +99,20 @@ message QueryIdentifierProto { optional int32 dag_identifier = 2; } +/** + * Tez API implementation derives an enum value from instanceof on the event, then uses that enum + * in a giant switch statement to re-implement virtual functions. In addition the event classes + * are final and serialization is a mix of writables and protobuf. As a result, it is impossible + * three times over to add anything there. So, we'd do our own "inspired" serialization. + * Eventually we'll move away from events for API. + */ +message NotTezEvent { + required bytes input_event_proto_bytes = 1; + required string vertex_name = 2; + required string dest_input_name = 3; + optional int32 key_id = 4; +} + message SubmitWorkRequestProto { optional VertexOrBinary work_spec = 1; optional bytes work_spec_signature = 2; @@ -115,6 +129,10 @@ message SubmitWorkRequestProto { // Not supported/honored for external clients right now. optional FragmentRuntimeInfo fragment_runtime_info = 9; + + // Serialized (and signed) NotTezEvent; used only for external clients for now. + optional bytes initial_event_bytes = 10; + optional bytes initial_event_signature = 11; } diff --git a/llap-common/src/test/org/apache/hadoop/hive/llap/tez/TestConverters.java b/llap-common/src/test/org/apache/hadoop/hive/llap/tez/TestConverters.java index 1df6df03ec7bc8655e085c3247718994334573b5..85c609198a358d0cf2ff1e204beb2fb20c1977fd 100644 --- a/llap-common/src/test/org/apache/hadoop/hive/llap/tez/TestConverters.java +++ b/llap-common/src/test/org/apache/hadoop/hive/llap/tez/TestConverters.java @@ -77,7 +77,7 @@ public void testTaskSpecToFragmentSpec() { new TaskSpec(tezTaskAttemptId, "dagName", "vertexName", 10, processorDescriptor, inputSpecList, outputSpecList, null); - SignableVertexSpec vertexProto = Converters.convertTaskSpecToProto(taskSpec, 0, "", null, ""); + SignableVertexSpec vertexProto = Converters.convertTaskSpecToProto(taskSpec, 0, "", "").build(); assertEquals("dagName", vertexProto.getDagName()); assertEquals("vertexName", vertexProto.getVertexName()); diff --git a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java index 4306c222442546c9c54162f95e3550ea978be1b7..d1748cb77bf493f65dac4400018e2b632eb8c388 100644 --- a/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java +++ b/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java @@ -16,34 +16,25 @@ */ package org.apache.hadoop.hive.llap; -import java.util.ArrayList; -import java.util.List; -import java.util.Set; -import java.util.concurrent.LinkedBlockingQueue; - -import java.sql.SQLException; -import java.sql.Connection; -import java.sql.ResultSet; -import java.sql.Statement; -import java.sql.DriverManager; - -import java.io.IOException; import java.io.DataInput; -import java.io.DataOutput; import java.io.DataInputStream; -import java.io.ByteArrayInputStream; +import java.io.IOException; import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.Socket; import java.nio.ByteBuffer; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; +import java.util.concurrent.LinkedBlockingQueue; import org.apache.commons.collections4.ListUtils; - -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.llap.LlapBaseRecordReader; import org.apache.hadoop.hive.llap.LlapBaseRecordReader.ReaderEvent; -import org.apache.hadoop.hive.llap.LlapInputSplit; import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentRuntimeInfo; import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SignableVertexSpec; import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWorkRequestProto; @@ -56,28 +47,21 @@ import org.apache.hadoop.hive.llap.tez.Converters; import org.apache.hadoop.io.DataInputBuffer; import org.apache.hadoop.io.DataOutputBuffer; +import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.WritableComparable; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.Writable; -import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.InputFormat; import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.InputSplitWithLocationInfo; -import org.apache.hadoop.mapred.SplitLocationInfo; -import org.apache.hadoop.mapred.FileSplit; +import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.RecordReader; import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.security.Credentials; -import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.security.token.TokenIdentifier; -import org.apache.hadoop.util.Progressable; import org.apache.hadoop.yarn.api.ApplicationConstants; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; - import org.apache.tez.common.security.JobTokenIdentifier; import org.apache.tez.common.security.TokenCache; import org.apache.tez.dag.records.TezTaskAttemptID; @@ -85,14 +69,10 @@ import org.apache.tez.runtime.api.impl.EventType; import org.apache.tez.runtime.api.impl.TaskSpec; import org.apache.tez.runtime.api.impl.TezEvent; -import org.apache.tez.runtime.api.impl.TezEvent; import org.apache.tez.runtime.api.impl.TezHeartbeatRequest; -import org.apache.tez.runtime.api.impl.TezHeartbeatResponse; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.protobuf.ByteString; @@ -100,7 +80,8 @@ /** * Base LLAP input format to handle requesting of splits and communication with LLAP daemon. */ -public class LlapBaseInputFormat implements InputFormat { +public class LlapBaseInputFormat> + implements InputFormat { private static final Logger LOG = LoggerFactory.getLogger(LlapBaseInputFormat.class); @@ -148,30 +129,23 @@ public LlapBaseInputFormat() {} LlapRecordReaderTaskUmbilicalExternalResponder umbilicalResponder = new LlapRecordReaderTaskUmbilicalExternalResponder(); + // TODO: close this LlapTaskUmbilicalExternalClient llapClient = new LlapTaskUmbilicalExternalClient(job, submitWorkInfo.getTokenIdentifier(), submitWorkInfo.getToken(), umbilicalResponder); llapClient.init(job); llapClient.start(); - SubmitWorkRequestProto submitWorkRequestProto = - constructSubmitWorkRequestProto(submitWorkInfo, llapSplit.getSplitNum(), - llapClient.getAddress(), submitWorkInfo.getToken()); - - TezEvent tezEvent = new TezEvent(); - DataInputBuffer dib = new DataInputBuffer(); - dib.reset(llapSplit.getFragmentBytes(), 0, llapSplit.getFragmentBytes().length); - tezEvent.readFields(dib); - List tezEventList = Lists.newArrayList(); - tezEventList.add(tezEvent); - - llapClient.submitWork(submitWorkRequestProto, host, llapSubmitPort, tezEventList); + SubmitWorkRequestProto submitWorkRequestProto = constructSubmitWorkRequestProto( + submitWorkInfo, llapSplit.getSplitNum(), llapClient.getAddress(), + submitWorkInfo.getToken(), llapSplit.getFragmentBytes(), + llapSplit.getFragmentBytesSignature()); + llapClient.submitWork(submitWorkRequestProto, host, llapSubmitPort); String id = HiveConf.getVar(job, HiveConf.ConfVars.HIVEQUERYID) + "_" + llapSplit.getSplitNum(); - HiveConf conf = new HiveConf(); - Socket socket = new Socket(host, - serviceInstance.getOutputFormatPort()); + // TODO: security for output channel + Socket socket = new Socket(host, serviceInstance.getOutputFormatPort()); LOG.debug("Socket connected"); @@ -181,7 +155,9 @@ public LlapBaseInputFormat() {} LOG.info("Registered id: " + id); - LlapBaseRecordReader recordReader = new LlapBaseRecordReader(socket.getInputStream(), llapSplit.getSchema(), Text.class, job); + @SuppressWarnings("rawtypes") + LlapBaseRecordReader recordReader = new LlapBaseRecordReader( + socket.getInputStream(), llapSplit.getSchema(), Text.class, job); umbilicalResponder.setRecordReader(recordReader); return recordReader; } @@ -295,25 +271,18 @@ private ServiceInstance selectServiceInstance(Set serviceInstan } private SubmitWorkRequestProto constructSubmitWorkRequestProto(SubmitWorkInfo submitWorkInfo, - int taskNum, - InetSocketAddress address, - Token token) throws - IOException { - TaskSpec taskSpec = submitWorkInfo.getTaskSpec(); + int taskNum, InetSocketAddress address, Token token, + byte[] fragmentBytes, byte[] fragmentBytesSignature) throws IOException { ApplicationId appId = submitWorkInfo.getFakeAppId(); - int attemptId = taskSpec.getTaskAttemptID().getId(); // This works, assuming the executor is running within YARN. String user = System.getenv(ApplicationConstants.Environment.USER.name()); LOG.info("Setting user in submitWorkRequest to: " + user); - SignableVertexSpec svs = Converters.convertTaskSpecToProto( - taskSpec, attemptId, appId.toString(), null, user); // TODO signatureKeyId + // TODO: this is bogus. What does LLAP use this for? ContainerId containerId = - ContainerId.newInstance(ApplicationAttemptId.newInstance(appId, 0), taskNum); + ContainerId.newInstance(ApplicationAttemptId.newInstance(appId, 0), taskNum); - - Credentials taskCredentials = new Credentials(); // Credentials can change across DAGs. Ideally construct only once per DAG. Credentials credentials = new Credentials(); TokenCache.setSessionToken(token, credentials); @@ -324,21 +293,29 @@ private SubmitWorkRequestProto constructSubmitWorkRequestProto(SubmitWorkInfo su runtimeInfo.setWithinDagPriority(0); runtimeInfo.setDagStartTime(submitWorkInfo.getCreationTime()); runtimeInfo.setFirstAttemptStartTime(submitWorkInfo.getCreationTime()); - runtimeInfo.setNumSelfAndUpstreamTasks(taskSpec.getVertexParallelism()); + runtimeInfo.setNumSelfAndUpstreamTasks(submitWorkInfo.getVertexParallelism()); runtimeInfo.setNumSelfAndUpstreamCompletedTasks(0); SubmitWorkRequestProto.Builder builder = SubmitWorkRequestProto.newBuilder(); - builder.setWorkSpec(VertexOrBinary.newBuilder().setVertex(svs).build()); - // TODO work spec signature - builder.setFragmentNumber(taskSpec.getTaskAttemptID().getTaskID().getId()); - builder.setAttemptNumber(0); + VertexOrBinary.Builder vertexBuilder = VertexOrBinary.newBuilder(); + vertexBuilder.setVertexBinary(ByteString.copyFrom(submitWorkInfo.getVertexBinary())); + if (submitWorkInfo.getVertexSignature() != null) { + // Unsecure case? + builder.setWorkSpecSignature(ByteString.copyFrom(submitWorkInfo.getVertexSignature())); + } + builder.setWorkSpec(vertexBuilder.build()); + builder.setFragmentNumber(taskNum); + builder.setAttemptNumber(0); // TODO: hmm builder.setContainerIdString(containerId.toString()); builder.setAmHost(address.getHostName()); builder.setAmPort(address.getPort()); builder.setCredentialsBinary(ByteString.copyFrom(credentialsBinary)); builder.setFragmentRuntimeInfo(runtimeInfo.build()); - + builder.setInitialEventBytes(ByteString.copyFrom(fragmentBytes)); + if (fragmentBytesSignature != null) { + builder.setInitialEventSignature(ByteString.copyFrom(fragmentBytesSignature)); + } return builder.build(); } @@ -351,7 +328,7 @@ private ByteBuffer serializeCredentials(Credentials credentials) throws IOExcept } private static class LlapRecordReaderTaskUmbilicalExternalResponder implements LlapTaskUmbilicalExternalResponder { - protected LlapBaseRecordReader recordReader = null; + protected LlapBaseRecordReader recordReader = null; protected LinkedBlockingQueue queuedEvents = new LinkedBlockingQueue(); public LlapRecordReaderTaskUmbilicalExternalResponder() { @@ -369,7 +346,6 @@ public void submissionFailed(String fragmentId, Throwable throwable) { @Override public void heartbeat(TezHeartbeatRequest request) { - TezTaskAttemptID taskAttemptId = request.getCurrentTaskAttemptID(); List inEvents = request.getEvents(); for (TezEvent tezEvent : ListUtils.emptyIfNull(inEvents)) { EventType eventType = tezEvent.getEventType(); @@ -415,7 +391,7 @@ public void heartbeatTimeout(String taskAttemptId) { } } - public synchronized LlapBaseRecordReader getRecordReader() { + public synchronized LlapBaseRecordReader getRecordReader() { return recordReader; } @@ -441,7 +417,7 @@ public synchronized void setRecordReader(LlapBaseRecordReader recordReader) { * @param readerEvent */ protected synchronized void sendOrQueueEvent(ReaderEvent readerEvent) { - LlapBaseRecordReader recordReader = getRecordReader(); + LlapBaseRecordReader recordReader = getRecordReader(); if (recordReader != null) { recordReader.handleEvent(readerEvent); } else { diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java index 6f21d3cb430cb2a9e8a1b4d94b80f286602b4882..ded84c162bbbd6a7b166448466bdd3423612b952 100644 --- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.llap.DaemonId; +import org.apache.hadoop.hive.llap.NotTezEventHelper; import org.apache.hadoop.hive.llap.daemon.ContainerRunner; import org.apache.hadoop.hive.llap.daemon.FragmentCompletionHandler; import org.apache.hadoop.hive.llap.daemon.HistoryLogger; @@ -38,6 +39,7 @@ import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentRuntimeInfo; import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProto; import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto; +import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.NotTezEvent; import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.QueryCompleteRequestProto; import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.QueryCompleteResponseProto; import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SignableVertexSpec; @@ -70,11 +72,13 @@ import org.apache.tez.hadoop.shim.HadoopShim; import org.apache.tez.hadoop.shim.HadoopShimsLoader; import org.apache.tez.runtime.api.impl.ExecutionContextImpl; +import org.apache.tez.runtime.api.impl.TezEvent; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import com.google.common.base.Preconditions; import com.google.protobuf.ByteString; +import com.google.protobuf.InvalidProtocolBufferException; public class ContainerRunnerImpl extends CompositeService implements ContainerRunner, FragmentCompletionHandler, QueryFailedHandler { @@ -113,7 +117,7 @@ public ContainerRunnerImpl(Configuration conf, int numExecutors, int waitQueueSi this.localShufflePort = localShufflePort; this.amReporter = amReporter; this.signer = UserGroupInformation.isSecurityEnabled() - ? new LlapSignerImpl(conf, daemonId) : null; + ? new LlapSignerImpl(conf, daemonId.getClusterString()) : null; this.fsUgiFactory = fsUgiFactory; this.clusterId = daemonId.getClusterString(); @@ -167,21 +171,9 @@ protected void serviceStop() throws Exception { @Override public SubmitWorkResponseProto submitWork(SubmitWorkRequestProto request) throws IOException { - VertexOrBinary vob = request.getWorkSpec(); - SignableVertexSpec vertex = vob.hasVertex() ? vob.getVertex() : null; - ByteString vertexBinary = vob.hasVertexBinary() ? vob.getVertexBinary() : null; - if (vertex != null && vertexBinary != null) { - throw new IOException( - "Vertex and vertexBinary in VertexOrBinary cannot be set at the same time"); - } - if (vertexBinary != null) { - vertex = SignableVertexSpec.parseFrom(vob.getVertexBinary()); - } - LlapTokenInfo tokenInfo = LlapTokenChecker.getTokenInfo(clusterId); - if (tokenInfo.isSigningRequired) { - checkSignature(vertex, vertexBinary, request, tokenInfo.userName); - } + SignableVertexSpec vertex = extractVertexSpec(request, tokenInfo); + TezEvent initialEvent = extractInitialEvent(request, tokenInfo); if (LOG.isInfoEnabled()) { LOG.info("Queueing container for execution: " + stringifySubmitRequest(request, vertex)); @@ -237,7 +229,7 @@ public SubmitWorkResponseProto submitWork(SubmitWorkRequestProto request) throws TaskRunnerCallable callable = new TaskRunnerCallable(request, fragmentInfo, callableConf, new LlapExecutionContext(localAddress.get().getHostName(), queryTracker), env, credentials, memoryPerExecutor, amReporter, confParams, metrics, killedTaskHandler, - this, tezHadoopShim, attemptId, vertex, taskUgi); + this, tezHadoopShim, attemptId, vertex, initialEvent, taskUgi); submissionState = executorService.schedule(callable); if (LOG.isInfoEnabled()) { @@ -262,6 +254,41 @@ public SubmitWorkResponseProto submitWork(SubmitWorkRequestProto request) throws return responseBuilder.build(); } + private SignableVertexSpec extractVertexSpec(SubmitWorkRequestProto request, + LlapTokenInfo tokenInfo) throws InvalidProtocolBufferException, IOException { + VertexOrBinary vob = request.getWorkSpec(); + SignableVertexSpec vertex = vob.hasVertex() ? vob.getVertex() : null; + ByteString vertexBinary = vob.hasVertexBinary() ? vob.getVertexBinary() : null; + if (vertexBinary != null) { + if (vertex != null) { + throw new IOException( + "Vertex and vertexBinary in VertexOrBinary cannot be set at the same time"); + } + vertex = SignableVertexSpec.parseFrom(vob.getVertexBinary()); + } + + if (tokenInfo.isSigningRequired) { + checkSignature(vertex, vertexBinary, request, tokenInfo.userName); + } + return vertex; + } + + private TezEvent extractInitialEvent(SubmitWorkRequestProto request, LlapTokenInfo tokenInfo) + throws InvalidProtocolBufferException { + if (!request.hasInitialEventBytes()) return null; + ByteString initialEventByteString = request.getInitialEventBytes(); + byte[] initialEventBytes = initialEventByteString.toByteArray(); + NotTezEvent initialEvent = NotTezEvent.parseFrom(initialEventBytes); + if (tokenInfo.isSigningRequired) { + if (!request.hasInitialEventSignature()) { + throw new SecurityException("Unsigned initial event is not allowed"); + } + byte[] signatureBytes = request.getInitialEventSignature().toByteArray(); + signer.checkSignature(initialEventBytes, signatureBytes, initialEvent.getKeyId()); + } + return NotTezEventHelper.toTezEvent(initialEvent); + } + private void checkSignature(SignableVertexSpec vertex, ByteString vertexBinary, SubmitWorkRequestProto request, String tokenUserName) throws SecurityException, IOException { if (!request.hasWorkSpecSignature()) { diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapTaskReporter.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapTaskReporter.java index dc4482eaeb875d6255a389d81e1b79d8a10a4b7a..05558d7e18f7f0d30496ced6efe0e5de74755232 100644 --- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapTaskReporter.java +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapTaskReporter.java @@ -81,6 +81,7 @@ private final AtomicLong requestCounter; private final String containerIdStr; private final String fragmentFullId; + private final TezEvent initialEvent; private final ListeningExecutorService heartbeatExecutor; @@ -89,7 +90,7 @@ public LlapTaskReporter(LlapTaskUmbilicalProtocol umbilical, long amPollInterval, long sendCounterInterval, int maxEventsToGet, AtomicLong requestCounter, - String containerIdStr, final String fragFullId) { + String containerIdStr, final String fragFullId, TezEvent initialEvent) { this.umbilical = umbilical; this.pollInterval = amPollInterval; this.sendCounterInterval = sendCounterInterval; @@ -97,6 +98,7 @@ public LlapTaskReporter(LlapTaskUmbilicalProtocol umbilical, long amPollInterval this.requestCounter = requestCounter; this.containerIdStr = containerIdStr; this.fragmentFullId = fragFullId; + this.initialEvent = initialEvent; ExecutorService executor = Executors.newFixedThreadPool(1, new ThreadFactoryBuilder() .setDaemon(true).setNameFormat("TaskHeartbeatThread").build()); heartbeatExecutor = MoreExecutors.listeningDecorator(executor); @@ -112,7 +114,7 @@ public synchronized void registerTask(RuntimeTask task, FragmentCountersMap.registerCountersForFragment(fragmentFullId, tezCounters); LOG.info("Registered counters for fragment: {} vertexName: {}", fragmentFullId, task.getVertexName()); currentCallable = new HeartbeatCallable(task, umbilical, pollInterval, sendCounterInterval, - maxEventsToGet, requestCounter, containerIdStr); + maxEventsToGet, requestCounter, containerIdStr, initialEvent); ListenableFuture future = heartbeatExecutor.submit(currentCallable); Futures.addCallback(future, new HeartbeatCallback(errorReporter)); } @@ -170,16 +172,18 @@ public void shutdown() { * Tracks the last non-OOB heartbeat number at which counters were sent to the AM. */ private int prevCounterSendHeartbeatNum = 0; + private TezEvent initialEvent; - public HeartbeatCallable(RuntimeTask task, - LlapTaskUmbilicalProtocol umbilical, long amPollInterval, long sendCounterInterval, - int maxEventsToGet, AtomicLong requestCounter, String containerIdStr) { + public HeartbeatCallable(RuntimeTask task, LlapTaskUmbilicalProtocol umbilical, + long amPollInterval, long sendCounterInterval, int maxEventsToGet, + AtomicLong requestCounter, String containerIdStr, TezEvent initialEvent) { this.pollInterval = amPollInterval; this.sendCounterInterval = sendCounterInterval; this.maxEventsToGet = maxEventsToGet; this.requestCounter = requestCounter; this.containerIdStr = containerIdStr; + this.initialEvent = initialEvent; this.task = task; this.umbilical = umbilical; @@ -299,6 +303,7 @@ private synchronized ResponseWrapper heartbeat(Collection eventsArg) t } else { task.setNextFromEventId(response.getNextFromEventId()); task.setNextPreRoutedEventId(response.getNextPreRoutedEventId()); + List taskEvents = null; if (response.getEvents() != null && !response.getEvents().isEmpty()) { if (LOG.isDebugEnabled()) { LOG.debug("Routing events from heartbeat response to task" + ", currentTaskAttemptId=" @@ -308,7 +313,21 @@ private synchronized ResponseWrapper heartbeat(Collection eventsArg) t } // This should ideally happen in a separate thread numEventsReceived = response.getEvents().size(); - task.handleEvents(response.getEvents()); + taskEvents = response.getEvents(); + } + if (initialEvent != null) { + // We currently only give the initial event to the task on the first heartbeat. Given + // that the split is ready, it seems pointless to wait, but that's how Tez works. + List oldEvents = taskEvents; + taskEvents = new ArrayList<>(1 + (taskEvents == null ? 0 : taskEvents.size())); + taskEvents.add(initialEvent); + initialEvent = null; + if (oldEvents != null) { + taskEvents.addAll(oldEvents); + } + } + if (taskEvents != null) { + task.handleEvents(taskEvents); } } return new ResponseWrapper(false, numEventsReceived); diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java index f97585d16609854a50f18b584fe3e1454de88c84..6c853a6618be77f1817bd90a0afc40067720d9a0 100644 --- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java @@ -58,6 +58,7 @@ import org.apache.tez.hadoop.shim.HadoopShim; import org.apache.tez.runtime.api.ExecutionContext; import org.apache.tez.runtime.api.impl.TaskSpec; +import org.apache.tez.runtime.api.impl.TezEvent; import org.apache.tez.runtime.common.objectregistry.ObjectRegistryImpl; import org.apache.tez.runtime.internals.api.TaskReporterInterface; import org.apache.tez.runtime.library.input.UnorderedKVInput; @@ -114,19 +115,17 @@ private final AtomicBoolean isCompleted = new AtomicBoolean(false); private final AtomicBoolean killInvoked = new AtomicBoolean(false); private final SignableVertexSpec vertex; + private final TezEvent initialEvent; private UserGroupInformation taskUgi; @VisibleForTesting public TaskRunnerCallable(SubmitWorkRequestProto request, QueryFragmentInfo fragmentInfo, - Configuration conf, - ExecutionContext executionContext, Map envMap, - Credentials credentials, - long memoryAvailable, AMReporter amReporter, - ConfParams confParams, LlapDaemonExecutorMetrics metrics, - KilledTaskHandler killedTaskHandler, - FragmentCompletionHandler fragmentCompleteHandler, - HadoopShim tezHadoopShim, TezTaskAttemptID attemptId, - SignableVertexSpec vertex, UserGroupInformation taskUgi) { + Configuration conf, ExecutionContext executionContext, Map envMap, + Credentials credentials, long memoryAvailable, AMReporter amReporter, ConfParams confParams, + LlapDaemonExecutorMetrics metrics, KilledTaskHandler killedTaskHandler, + FragmentCompletionHandler fragmentCompleteHandler, HadoopShim tezHadoopShim, + TezTaskAttemptID attemptId, SignableVertexSpec vertex, TezEvent initialEvent, + UserGroupInformation taskUgi) { this.request = request; this.fragmentInfo = fragmentInfo; this.conf = conf; @@ -153,6 +152,7 @@ public TaskRunnerCallable(SubmitWorkRequestProto request, QueryFragmentInfo frag this.killedTaskHandler = killedTaskHandler; this.fragmentCompletionHanler = fragmentCompleteHandler; this.tezHadoopShim = tezHadoopShim; + this.initialEvent = initialEvent; this.taskUgi = taskUgi; } @@ -227,7 +227,8 @@ public LlapTaskUmbilicalProtocol run() throws Exception { confParams.amMaxEventsPerHeartbeat, new AtomicLong(0), request.getContainerIdString(), - fragFullId); + fragFullId, + initialEvent); String attemptId = fragmentInfo.getFragmentIdentifierString(); IOContextMap.setThreadAttemptId(attemptId); diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/security/LlapSignerImpl.java b/llap-server/src/java/org/apache/hadoop/hive/llap/security/LlapSignerImpl.java deleted file mode 100644 index 417459353e3fbe5d64ac95eaed4a37db58c17d1d..0000000000000000000000000000000000000000 --- a/llap-server/src/java/org/apache/hadoop/hive/llap/security/LlapSignerImpl.java +++ /dev/null @@ -1,60 +0,0 @@ -/** - * 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.llap.security; - -import java.io.IOException; -import java.util.Arrays; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.llap.DaemonId; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.security.token.delegation.DelegationKey; - -import com.google.common.annotations.VisibleForTesting; - -public class LlapSignerImpl implements LlapSigner { - private final SigningSecretManager secretManager; - - public LlapSignerImpl(Configuration conf, DaemonId daemonId) { - // TODO: create this centrally in HS2 case - secretManager = SecretManager.createSecretManager(conf, daemonId.getClusterString()); - } - - @VisibleForTesting - public LlapSignerImpl(SigningSecretManager sm) { - secretManager = sm; - } - - @Override - public SignedMessage serializeAndSign(Signable message) throws IOException { - SignedMessage result = new SignedMessage(); - DelegationKey key = secretManager.getCurrentKey(); - message.setSignInfo(key.getKeyId(), UserGroupInformation.getCurrentUser().getUserName()); - result.message = message.serialize(); - result.signature = secretManager.signWithKey(result.message, key); - return result; - } - - @Override - public void checkSignature(byte[] message, byte[] signature, int keyId) - throws SecurityException { - byte[] expectedSignature = secretManager.signWithKey(message, keyId); - if (Arrays.equals(signature, expectedSignature)) return; - throw new SecurityException("Message signature does not match"); - } -} diff --git a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorTestHelpers.java b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorTestHelpers.java index 1df5253f4654f7340f0180d36112a997fa9e51c0..fe2ced5cca6b70a07e7ee15158e63984711487de 100644 --- a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorTestHelpers.java +++ b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorTestHelpers.java @@ -39,6 +39,7 @@ import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.hadoop.shim.DefaultHadoopShim; import org.apache.tez.runtime.api.impl.ExecutionContextImpl; +import org.apache.tez.runtime.api.impl.TezEvent; import org.apache.tez.runtime.task.EndReason; import org.apache.tez.runtime.task.TaskRunner2Result; import org.slf4j.Logger; @@ -51,22 +52,21 @@ public static MockRequest createMockRequest(int fragmentNum, int parallelism, long startTime, boolean canFinish, long workTime) { SubmitWorkRequestProto - requestProto = createSubmitWorkRequestProto(fragmentNum, parallelism, - startTime); + request = createSubmitWorkRequestProto(fragmentNum, parallelism, startTime); + return createMockRequest(canFinish, workTime, request); + } + + private static MockRequest createMockRequest(boolean canFinish, + long workTime, SubmitWorkRequestProto request) { QueryFragmentInfo queryFragmentInfo = createQueryFragmentInfo( - requestProto.getWorkSpec().getVertex(), requestProto.getFragmentNumber()); - MockRequest mockRequest = new MockRequest(requestProto, queryFragmentInfo, canFinish, workTime); - return mockRequest; + request.getWorkSpec().getVertex(), request.getFragmentNumber()); + return new MockRequest(request, queryFragmentInfo, canFinish, workTime, null); } public static TaskExecutorService.TaskWrapper createTaskWrapper( SubmitWorkRequestProto request, boolean canFinish, int workTime) { - QueryFragmentInfo queryFragmentInfo = createQueryFragmentInfo( - request.getWorkSpec().getVertex(), request.getFragmentNumber()); - MockRequest mockRequest = new MockRequest(request, queryFragmentInfo, canFinish, workTime); - TaskExecutorService.TaskWrapper - taskWrapper = new TaskExecutorService.TaskWrapper(mockRequest, null); - return taskWrapper; + return new TaskExecutorService.TaskWrapper( + createMockRequest(canFinish, workTime, request), null); } public static QueryFragmentInfo createQueryFragmentInfo( @@ -145,13 +145,13 @@ public static SubmitWorkRequestProto createSubmitWorkRequestProto( private final Condition finishedCondition = lock.newCondition(); public MockRequest(SubmitWorkRequestProto requestProto, QueryFragmentInfo fragmentInfo, - boolean canFinish, long workTime) { + boolean canFinish, long workTime, TezEvent initialEvent) { super(requestProto, fragmentInfo, new Configuration(), new ExecutionContextImpl("localhost"), null, new Credentials(), 0, mock(AMReporter.class), null, mock( LlapDaemonExecutorMetrics.class), mock(KilledTaskHandler.class), mock( FragmentCompletionHandler.class), new DefaultHadoopShim(), null, - requestProto.getWorkSpec().getVertex(), null); + requestProto.getWorkSpec().getVertex(), initialEvent, null); this.workTime = workTime; this.canFinish = canFinish; } diff --git a/llap-server/src/test/org/apache/hadoop/hive/llap/security/TestLlapSignerImpl.java b/llap-server/src/test/org/apache/hadoop/hive/llap/security/TestLlapSignerImpl.java index 04202254965b6ff78fa19eb122710a9fa01f7a6f..a281fd6fb3544968e0058bc4d61ae17b8fb22b97 100644 --- a/llap-server/src/test/org/apache/hadoop/hive/llap/security/TestLlapSignerImpl.java +++ b/llap-server/src/test/org/apache/hadoop/hive/llap/security/TestLlapSignerImpl.java @@ -127,7 +127,7 @@ public TestSignable(int keyId, byte b) { } @Override - public void setSignInfo(int masterKeyId, String user) { + public void setSignInfo(int masterKeyId) { this.masterKeyId = masterKeyId; } @@ -196,5 +196,10 @@ public DelegationKey getCurrentKey() { public AbstractDelegationTokenIdentifier createIdentifier() { throw new UnsupportedOperationException(); } + + @Override + public void close() { + stopThreads(); + } } } diff --git a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java index 026df3bbfef3fe6f13d1bf9b0dc18b07e3cc817e..80096f56ec6ce08666cec5fa61b305b034dffb32 100644 --- a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java +++ b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java @@ -599,7 +599,7 @@ private SubmitWorkRequestProto constructSubmitWorkRequest(ContainerId containerI } builder.setCredentialsBinary(ByteString.copyFrom(credentialsBinary)); builder.setWorkSpec(VertexOrBinary.newBuilder().setVertex(Converters.convertTaskSpecToProto( - taskSpec, appAttemptId, getTokenIdentifier(), null, user)).build()); + taskSpec, appAttemptId, getTokenIdentifier(), user)).build()); // Don't call builder.setWorkSpecSignature() - Tez doesn't sign fragments builder.setFragmentRuntimeInfo(fragmentRuntimeInfo); return builder.build(); diff --git a/orc/src/java/org/apache/orc/OrcConf.java b/orc/src/java/org/apache/orc/OrcConf.java index 6fcbb72aeea40dea9d9143a334b42fb81c7f4404..357318d73948a405e550271cfc8860b74a08a8c2 100644 --- a/orc/src/java/org/apache/orc/OrcConf.java +++ b/orc/src/java/org/apache/orc/OrcConf.java @@ -40,6 +40,8 @@ " number of rows n index entry represents.)"), BUFFER_SIZE("orc.compress.size", "hive.exec.orc.default.buffer.size", 256 * 1024, "Define the default ORC buffer size, in bytes."), + BASE_DELTA_RATIO("orc.base.delta.ratio", "hive.exec.orc.base.delta.ratio", 8, + "The ratio of base writer and delta writer in terms of STRIPE_SIZE and BUFFER_SIZE."), BLOCK_PADDING("orc.block.padding", "hive.exec.orc.default.block.padding", true, "Define whether stripes should be padded to the HDFS block boundaries."), diff --git a/ql/pom.xml b/ql/pom.xml index db03978389900ff00e725cdabdf19226403f8804..02ddb805a228ed23694c8a81953dd2400d7308c6 100644 --- a/ql/pom.xml +++ b/ql/pom.xml @@ -812,6 +812,7 @@ org.apache.hive:hive-common org.apache.hive:hive-exec org.apache.hive:hive-serde + org.apache.hive:hive-llap-common org.apache.hive:hive-llap-client org.apache.hive:hive-metastore org.apache.hive:hive-service-rpc diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Registry.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Registry.java index 86df74d26ddd60736214c0f9acb835f4253a2c17..a4584e3eaebc052a6870c543bfbfc9e5d78e771b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Registry.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Registry.java @@ -556,7 +556,16 @@ private FunctionInfo getQualifiedFunctionInfoUnderLock(String qualifiedName) thr // and the current thread may not be able to resolve the UDF. Test for this condition // and if necessary load the JARs in this thread. if (isNative && info != null && info.isPersistent()) { - return registerToSessionRegistry(qualifiedName, info); + Class functionClass; + try { + functionClass = info.getFunctionClass(); + } catch (Exception e) { + return registerToSessionRegistry(qualifiedName, info); + } + if (functionClass == null) { + return registerToSessionRegistry(qualifiedName, info); + } + return info; } if (info != null || !isNative) { return info; // We have the UDF, or we are in the session registry (or both). diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java index 54fc3af94c6b45f2e7ed8edb0b8f45d4fa0dbf2b..5dd5e80214e0737c61eb347f304756b97d565c94 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java @@ -33,9 +33,6 @@ import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hive.common.JavaUtils; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.llap.registry.ServiceInstance; -import org.apache.hadoop.hive.llap.registry.ServiceInstanceSet; -import org.apache.hadoop.hive.llap.registry.impl.LlapRegistryService; import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.hive.serde2.SerDeException; @@ -47,7 +44,6 @@ import org.apache.hadoop.mapred.split.SplitLocationProvider; import org.apache.hadoop.mapreduce.split.TezMapReduceSplitsGrouper; import org.apache.hadoop.util.ReflectionUtils; -import org.apache.hadoop.yarn.api.records.Resource; import org.apache.tez.common.TezUtils; import org.apache.tez.dag.api.TaskLocationHint; import org.apache.tez.dag.api.VertexLocationHint; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java index d04cfa3b102e1674bb2be2d8bebceb717dabc46a..919b35a59d2e6a3c030438ceb3dea3f6a9379bc8 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java @@ -51,12 +51,11 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hadoop.hive.llap.DaemonId; -import org.apache.hadoop.hive.llap.LlapUtil; +import org.apache.hadoop.hive.llap.coordinator.LlapCoordinator; import org.apache.hadoop.hive.llap.impl.LlapProtocolClientImpl; +import org.apache.hadoop.hive.llap.io.api.LlapProxy; import org.apache.hadoop.hive.llap.security.LlapTokenClient; import org.apache.hadoop.hive.llap.security.LlapTokenIdentifier; -import org.apache.hadoop.hive.llap.security.LlapTokenLocalClient; import org.apache.hadoop.hive.llap.tez.LlapProtocolClientProxy; import org.apache.hadoop.hive.llap.tezplugins.LlapContainerLauncher; import org.apache.hadoop.hive.llap.tezplugins.LlapTaskCommunicator; @@ -85,11 +84,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.cache.Cache; -import com.google.common.cache.CacheBuilder; -import com.google.common.cache.RemovalListener; -import com.google.common.cache.RemovalNotification; - /** * Holds session state related to Tez */ @@ -338,19 +332,6 @@ public TezClient call() throws Exception { } } - // Only cache ZK connections (ie local clients); these are presumed to be used in HS2. - // TODO: temporary before HIVE-13698. - private static final Cache localClientCache = CacheBuilder - .newBuilder().expireAfterAccess(10, TimeUnit.MINUTES) - .removalListener(new RemovalListener() { - @Override - public void onRemoval(RemovalNotification notification) { - if (notification.getValue() != null) { - notification.getValue().close(); - } - } - }).build(); - private static Token getLlapToken( String user, final Configuration conf) throws IOException { // TODO: parts of this should be moved out of TezSession to reuse the clients, but there's @@ -359,21 +340,17 @@ public void onRemoval(RemovalNotification notifica boolean isInHs2 = session != null && session.isHiveServerQuery(); Token token = null; // For Tez, we don't use appId to distinguish the tokens. + + LlapCoordinator coordinator = null; if (isInHs2) { // We are in HS2, get the token locally. - String clusterName = LlapUtil.generateClusterName(conf); - // This assumes that the LLAP cluster and session are both running under HS2 user. - final String clusterId = DaemonId.createClusterString(user, clusterName); - try { - token = localClientCache.get(clusterId, new Callable() { - @Override - public LlapTokenLocalClient call() throws Exception { - return new LlapTokenLocalClient(conf, clusterId); - } - }).createToken(null, null, false); // Signature is not required for Tez. - } catch (ExecutionException e) { - throw new IOException(e); + // TODO: coordinator should be passed in; HIVE-13698. Must be initialized for now. + coordinator = LlapCoordinator.getInstance(); + if (coordinator == null) { + throw new IOException("LLAP coordinator not initialized; cannot get LLAP tokens"); } + // Signing is not required for Tez. + token = coordinator.getLocalTokenClient(conf, user).createToken(null, null, false); } else { // We are not in HS2; always create a new client for now. token = new LlapTokenClient(conf).getDelegationToken(null); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRecordUpdater.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRecordUpdater.java index 4bf2403704f4f7dd1c83ebc28f34eb01e18a0066..e577961c2b88e8c016ac51e2597f7625ddf693dd 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRecordUpdater.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRecordUpdater.java @@ -27,6 +27,7 @@ import org.apache.orc.impl.AcidStats; import org.apache.orc.impl.OrcAcidUtils; +import org.apache.orc.OrcConf; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -204,19 +205,38 @@ static StructObjectInspector createEventSchema(ObjectInspector rowInspector) { flushLengths = null; } OrcFile.WriterOptions writerOptions = null; - if (options instanceof OrcOptions) { - writerOptions = ((OrcOptions) options).getOrcOptions(); - } - if (writerOptions == null) { - writerOptions = OrcFile.writerOptions(options.getTableProperties(), - options.getConfiguration()); - } - writerOptions.fileSystem(fs).callback(indexBuilder); - if (!options.isWritingBase()) { + // If writing delta dirs, we need to make a clone of original options, to avoid polluting it for + // the base writer + if (options.isWritingBase()) { + if (options instanceof OrcOptions) { + writerOptions = ((OrcOptions) options).getOrcOptions(); + } + if (writerOptions == null) { + writerOptions = OrcFile.writerOptions(options.getTableProperties(), + options.getConfiguration()); + } + } else { // delta writer + AcidOutputFormat.Options optionsCloneForDelta = options.clone(); + + if (optionsCloneForDelta instanceof OrcOptions) { + writerOptions = ((OrcOptions) optionsCloneForDelta).getOrcOptions(); + } + if (writerOptions == null) { + writerOptions = OrcFile.writerOptions(optionsCloneForDelta.getTableProperties(), + optionsCloneForDelta.getConfiguration()); + } + + // get buffer size and stripe size for base writer + int baseBufferSizeValue = writerOptions.getBufferSize(); + long baseStripeSizeValue = writerOptions.getStripeSize(); + + // overwrite buffer size and stripe size for delta writer, based on BASE_DELTA_RATIO + int ratio = (int) OrcConf.BASE_DELTA_RATIO.getLong(options.getConfiguration()); + writerOptions.bufferSize(baseBufferSizeValue / ratio); + writerOptions.stripeSize(baseStripeSizeValue / ratio); writerOptions.blockPadding(false); - writerOptions.bufferSize(DELTA_BUFFER_SIZE); - writerOptions.stripeSize(DELTA_STRIPE_SIZE); } + writerOptions.fileSystem(fs).callback(indexBuilder); rowInspector = (StructObjectInspector)options.getInspector(); writerOptions.inspector(createEventSchema(findRecId(options.getInspector(), options.getRecordIdColumn()))); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/PTFTranslator.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/PTFTranslator.java index 018d8d0d4bf35178e23453e5c6f0917e7edefc55..f32d02b9529ebea63863754b8e960b10dd3bf94d 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/PTFTranslator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/PTFTranslator.java @@ -599,7 +599,7 @@ private static void validateValueBoundaryExprType(ObjectInspector OI) if (!OI.getCategory().equals(Category.PRIMITIVE)) { throw new SemanticException( String.format( - "Value Boundary expression must be of primitve type. Found: %s", + "Value Boundary expression must be of primitive type. Found: %s", OI.getTypeName())); } @@ -608,6 +608,7 @@ private static void validateValueBoundaryExprType(ObjectInspector OI) switch (pC) { + case BOOLEAN: case BYTE: case DOUBLE: case FLOAT: @@ -618,10 +619,12 @@ private static void validateValueBoundaryExprType(ObjectInspector OI) case TIMESTAMP: case DATE: case STRING: + case VARCHAR: + case CHAR: break; default: throw new SemanticException( - String.format("Primitve type %s not supported in Value Boundary expression", + String.format("Primitive type %s not supported in Value Boundary expression", pC)); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java index 140dbdab5162ac865a9d6427aaf4a972cebbd827..cff5ee103241f3c7a0be567738af59fd537a45c0 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java @@ -28,7 +28,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import java.util.Random; import java.util.Set; import java.util.UUID; @@ -44,11 +43,18 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.llap.LlapInputSplit; import org.apache.hadoop.hive.llap.LlapOutputFormat; +import org.apache.hadoop.hive.llap.NotTezEventHelper; import org.apache.hadoop.hive.llap.SubmitWorkInfo; import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.llap.Schema; import org.apache.hadoop.hive.llap.FieldDesc; import org.apache.hadoop.hive.llap.TypeDesc; +import org.apache.hadoop.hive.llap.coordinator.LlapCoordinator; +import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SignableVertexSpec; +import org.apache.hadoop.hive.llap.security.LlapSigner; +import org.apache.hadoop.hive.llap.security.LlapSigner.Signable; +import org.apache.hadoop.hive.llap.security.LlapSigner.SignedMessage; +import org.apache.hadoop.hive.llap.tez.Converters; import org.apache.hadoop.hive.ql.CommandNeedRetryException; import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.Driver; @@ -61,7 +67,6 @@ import org.apache.hadoop.hive.ql.exec.tez.DagUtils; import org.apache.hadoop.hive.ql.exec.tez.HiveSplitGenerator; import org.apache.hadoop.hive.ql.exec.tez.TezTask; -import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.hive.ql.plan.TezWork; @@ -89,16 +94,17 @@ import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.api.records.LocalResourceType; import org.apache.tez.dag.api.DAG; +import org.apache.tez.dag.api.InputDescriptor; +import org.apache.tez.dag.api.InputInitializerDescriptor; +import org.apache.tez.dag.api.RootInputLeafOutput; import org.apache.tez.dag.api.TaskLocationHint; import org.apache.tez.dag.api.TaskSpecBuilder; -import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.api.Vertex; import org.apache.tez.mapreduce.grouper.TezSplitGrouper; import org.apache.tez.runtime.api.Event; import org.apache.tez.runtime.api.events.InputConfigureVertexTasksEvent; -import org.apache.tez.runtime.api.impl.EventMetaData; +import org.apache.tez.runtime.api.events.InputDataInformationEvent; import org.apache.tez.runtime.api.impl.TaskSpec; -import org.apache.tez.runtime.api.impl.TezEvent; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -112,11 +118,8 @@ + "Returns an array of length int serialized splits for the referenced tables string.") @UDFType(deterministic = false) public class GenericUDTFGetSplits extends GenericUDTF { - private static final Logger LOG = LoggerFactory.getLogger(GenericUDTFGetSplits.class); - private static final String LLAP_INTERNAL_INPUT_FORMAT_NAME = "org.apache.hadoop.hive.llap.LlapInputFormat"; - protected transient StringObjectInspector stringOI; protected transient IntObjectInspector intOI; protected transient JobConf jc; @@ -289,6 +292,7 @@ public PlanFragment createPlanFragment(String query, int num) throws HiveExcepti // bunch of things get setup in the context based on conf but we need only the MR tmp directory // for the following method. JobConf wxConf = utils.initializeVertexConf(job, ctx, mapWork); + // TODO: should we also whitelist input formats here? from mapred.input.format.class Path scratchDir = utils.createTezDir(ctx.getMRScratchDir(), job); FileSystem fs = scratchDir.getFileSystem(job); try { @@ -310,7 +314,6 @@ public PlanFragment createPlanFragment(String query, int num) throws HiveExcepti List eventList = splitGenerator.initialize(); InputSplit[] result = new InputSplit[eventList.size() - 1]; - DataOutputBuffer dob = new DataOutputBuffer(); InputConfigureVertexTasksEvent configureEvent = (InputConfigureVertexTasksEvent) eventList.get(0); @@ -320,58 +323,122 @@ public PlanFragment createPlanFragment(String query, int num) throws HiveExcepti Preconditions.checkState(hints.size() == eventList.size() - 1); if (LOG.isDebugEnabled()) { - LOG.debug("NumEvents=" + eventList.size()); - LOG.debug("NumSplits=" + result.length); + LOG.debug("NumEvents=" + eventList.size() + ", NumSplits=" + result.length); + } + + LlapCoordinator coordinator = LlapCoordinator.getInstance(); + if (coordinator == null) { + throw new IOException("LLAP coordinator is not initialized; must be running in HS2 with " + + ConfVars.LLAP_HS2_ENABLE_COORDINATOR.varname + " enabled"); } - ApplicationId fakeApplicationId - = ApplicationId.newInstance(Math.abs(new Random().nextInt()), 0); + // See the discussion in the implementation as to why we generate app ID. + ApplicationId fakeApplicationId = coordinator.createExtClientAppId(); + // This assumes LLAP cluster owner is always the HS2 user. String llapUser = UserGroupInformation.getLoginUser().getShortUserName(); + LlapSigner signer = UserGroupInformation.isSecurityEnabled() + ? coordinator.getLlapSigner(job) : null; + LOG.info("Number of splits: " + (eventList.size() - 1)); + SignedMessage signedSvs = null; + DataOutputBuffer dob = new DataOutputBuffer(); for (int i = 0; i < eventList.size() - 1; i++) { - - TaskSpec taskSpec = - new TaskSpecBuilder().constructTaskSpec(dag, vertexName, + TaskSpec taskSpec = new TaskSpecBuilder().constructTaskSpec(dag, vertexName, eventList.size() - 1, fakeApplicationId, i); - SubmitWorkInfo submitWorkInfo = - new SubmitWorkInfo(taskSpec, fakeApplicationId, System.currentTimeMillis()); - EventMetaData sourceMetaData = - new EventMetaData(EventMetaData.EventProducerConsumerType.INPUT, vertexName, - "NULL_VERTEX", null); - EventMetaData destinationMetaInfo = new TaskSpecBuilder().getDestingationMetaData(wx); - - // Creating the TezEvent here itself, since it's easy to serialize. - Event event = eventList.get(i + 1); - TaskLocationHint hint = hints.get(i); - Set hosts = hint.getHosts(); - if (hosts.size() != 1) { - LOG.warn("Bad # of locations: " + hosts.size()); + // 1. Generate the vertex/submit information. + if (i == 0) { + // Despite the differences in TaskSpec, the vertex spec should be the same. + signedSvs = createSignedVertexSpec(signer, taskSpec, fakeApplicationId); } - SplitLocationInfo[] locations = new SplitLocationInfo[hosts.size()]; - int j = 0; - for (String host : hosts) { - locations[j++] = new SplitLocationInfo(host, false); - } - TezEvent tezEvent = new TezEvent(event, sourceMetaData, System.currentTimeMillis()); - tezEvent.setDestinationInfo(destinationMetaInfo); + SubmitWorkInfo submitWorkInfo = new SubmitWorkInfo(fakeApplicationId, + System.currentTimeMillis(), taskSpec.getVertexParallelism(), signedSvs.message, + signedSvs.signature); + byte[] submitWorkBytes = SubmitWorkInfo.toBytes(submitWorkInfo); - bos.reset(); - dob.reset(); - tezEvent.write(dob); + // 2. Generate input event. + SignedMessage eventBytes = makeEventBytes( + wx, vertexName, eventList.get(i + 1), dob, signer); - byte[] submitWorkBytes = SubmitWorkInfo.toBytes(submitWorkInfo); + // 3. Make location hints. + SplitLocationInfo[] locations = makeLocationHints(hints.get(i)); - result[i] = new LlapInputSplit(i, submitWorkBytes, dob.getData(), locations, schema, llapUser); - } + result[i] = new LlapInputSplit(i, submitWorkBytes, eventBytes.message, + eventBytes.signature, locations, schema, llapUser); + } return result; } catch (Exception e) { throw new IOException(e); } } + private SplitLocationInfo[] makeLocationHints(TaskLocationHint hint) { + Set hosts = hint.getHosts(); + if (hosts.size() != 1) { + LOG.warn("Bad # of locations: " + hosts.size()); + } + SplitLocationInfo[] locations = new SplitLocationInfo[hosts.size()]; + int j = 0; + for (String host : hosts) { + locations[j++] = new SplitLocationInfo(host, false); + } + return locations; + } + + private SignedMessage makeEventBytes(Vertex wx, String vertexName, + Event event, DataOutputBuffer dob, LlapSigner signer) throws IOException { + assert event instanceof InputDataInformationEvent; + List> inputs = + TaskSpecBuilder.getVertexInputs(wx); + Preconditions.checkState(inputs.size() == 1); + + Signable signableNte = NotTezEventHelper.createSignableNotTezEvent( + (InputDataInformationEvent)event, vertexName, inputs.get(0).getName()); + if (signer != null) { + return signer.serializeAndSign(signableNte); + } else { + SignedMessage sm = new SignedMessage(); + sm.message = signableNte.serialize(); + return sm; + } + } + + private SignedMessage createSignedVertexSpec(LlapSigner signer, TaskSpec taskSpec, + ApplicationId fakeApplicationId) throws IOException { + // We put the query user, not LLAP user, in the message (and the token later?) + String user = SessionState.getUserFromAuthenticator(); + if (user == null) { + user = UserGroupInformation.getCurrentUser().getUserName(); + LOG.warn("Cannot determine the session user; using " + user + " instead"); + } + final SignableVertexSpec.Builder svsb = Converters.convertTaskSpecToProto( + taskSpec, 0, fakeApplicationId.toString(), user); + if (signer == null) { + SignedMessage result = new SignedMessage(); + result.message = serializeVertexSpec(svsb); + return result; + } + return signer.serializeAndSign(new Signable() { + @Override + public void setSignInfo(int masterKeyId) { + svsb.setSignatureKeyId(masterKeyId); + } + + @Override + public byte[] serialize() throws IOException { + return serializeVertexSpec(svsb); + } + }); + } + + private static byte[] serializeVertexSpec(SignableVertexSpec.Builder svsb) throws IOException { + ByteArrayOutputStream os = new ByteArrayOutputStream(); + svsb.build().writeTo(os); + return os.toByteArray(); + } + /** * Returns a local resource representing a jar. This resource will be used to execute the plan on * the cluster. diff --git a/ql/src/java/org/apache/tez/dag/api/TaskSpecBuilder.java b/ql/src/java/org/apache/tez/dag/api/TaskSpecBuilder.java index 6d3180200ce57eec8ce69c6fa8e98d1a9b6d7a52..2c719eecb02939afbed63267f9ca25d4f24c0114 100644 --- a/ql/src/java/org/apache/tez/dag/api/TaskSpecBuilder.java +++ b/ql/src/java/org/apache/tez/dag/api/TaskSpecBuilder.java @@ -63,15 +63,8 @@ public TaskSpec constructTaskSpec(DAG dag, String vertexName, int numSplits, App return new TaskSpec(taskAttemptId, dag.getName(), vertexName, numSplits, processorDescriptor, inputSpecs, outputSpecs, null); } - public EventMetaData getDestingationMetaData(Vertex vertex) { - List> inputs = - vertex.getInputs(); - Preconditions.checkState(inputs.size() == 1); - String inputName = inputs.get(0).getName(); - EventMetaData destMeta = - new EventMetaData(EventMetaData.EventProducerConsumerType.INPUT, vertex.getName(), - inputName, null); - return destMeta; + public static List> getVertexInputs(Vertex wx) { + return wx.getInputs(); } - } diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcRecordUpdater.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcRecordUpdater.java index 0a61fb884a8e5825e95b442d4c97efa247da4170..67c473e1888038eab2ade8bc440f192b8dd26776 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcRecordUpdater.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcRecordUpdater.java @@ -30,6 +30,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.io.AcidOutputFormat; import org.apache.hadoop.hive.ql.io.AcidUtils; import org.apache.hadoop.hive.ql.io.RecordIdentifier; @@ -197,6 +198,8 @@ public void testWriterTblProperties() throws Exception { } Properties tblProps = new Properties(); tblProps.setProperty("orc.compress", "SNAPPY"); + tblProps.setProperty("orc.compress.size", "8192"); + HiveConf.setIntVar(conf, HiveConf.ConfVars.HIVE_ORC_BASE_DELTA_RATIO, 4); AcidOutputFormat.Options options = new AcidOutputFormat.Options(conf) .filesystem(fs) .bucket(10) @@ -223,6 +226,7 @@ public void testWriterTblProperties() throws Exception { System.out.flush(); String outDump = new String(myOut.toByteArray()); assertEquals(true, outDump.contains("Compression: SNAPPY")); + assertEquals(true, outDump.contains("Compression size: 2048")); System.setOut(origOut); updater.close(false); } diff --git a/ql/src/test/queries/clientpositive/windowing_range_multiorder.q b/ql/src/test/queries/clientpositive/windowing_range_multiorder.q index d8ca4d64620d01834c24ae729332dd8d053ea1b6..24ea1e6b359acf4c1b94b81f45d0755793707c44 100644 --- a/ql/src/test/queries/clientpositive/windowing_range_multiorder.q +++ b/ql/src/test/queries/clientpositive/windowing_range_multiorder.q @@ -32,3 +32,9 @@ select s, si, i, min(i) over (partition by s order by si, i range between unboun select s, si, i, avg(i) over (partition by s order by si, i desc range between unbounded preceding and current row) from over10k limit 100; select si, bo, i, f, max(i) over (partition by si, bo order by i, f desc range between unbounded preceding and current row) from over10k limit 100; + +select bo, rank() over (partition by i order by bo nulls first, b nulls last range between unbounded preceding and unbounded following) from over10k limit 100; + +select CAST(s as CHAR(12)), rank() over (partition by i order by CAST(s as CHAR(12)) nulls last range between unbounded preceding and unbounded following) from over10k limit 100; + +select CAST(s as VARCHAR(12)), rank() over (partition by i order by CAST(s as VARCHAR(12)) nulls last range between unbounded preceding and unbounded following) from over10k limit 100; diff --git a/ql/src/test/results/clientpositive/tez/acid_globallimit.q.out b/ql/src/test/results/clientpositive/tez/acid_globallimit.q.out index 804e5e25a65915fb867e0ca23181aafbc791ffdc..a460182333f2d343ad4d3e3bcae7021a19f4da9b 100644 --- a/ql/src/test/results/clientpositive/tez/acid_globallimit.q.out +++ b/ql/src/test/results/clientpositive/tez/acid_globallimit.q.out @@ -45,7 +45,7 @@ Table Parameters: numFiles 3 numRows 0 rawDataSize 0 - totalSize 100540 + totalSize 101147 transactional true #### A masked pattern was here #### @@ -75,9 +75,9 @@ Stage-0 File Output Operator [FS_3] Limit [LIM_2] (rows=10 width=4) Number of rows:10 - Select Operator [SEL_1] (rows=25135 width=4) + Select Operator [SEL_1] (rows=25286 width=4) Output:["_col0"] - TableScan [TS_0] (rows=25135 width=4) + TableScan [TS_0] (rows=25286 width=4) default@acidtest1,acidtest1, ACID table,Tbl:COMPLETE,Col:NONE,Output:["c1"] PREHOOK: query: select cast (c1 as string) from acidtest1 limit 10 diff --git a/ql/src/test/results/clientpositive/windowing_range_multiorder.q.out b/ql/src/test/results/clientpositive/windowing_range_multiorder.q.out index 9910883680f45108bf3db59041d9b5ec6e749cdb..5f9c3d65d9c56484b76e6468f3a1ca3648329f49 100644 --- a/ql/src/test/results/clientpositive/windowing_range_multiorder.q.out +++ b/ql/src/test/results/clientpositive/windowing_range_multiorder.q.out @@ -908,3 +908,327 @@ POSTHOOK: Input: default@over10k 258 true 65568 13.57 65568 258 true 65579 47.52 65579 258 true 65603 2.61 65603 +PREHOOK: query: select bo, rank() over (partition by i order by bo nulls first, b nulls last range between unbounded preceding and unbounded following) from over10k limit 100 +PREHOOK: type: QUERY +PREHOOK: Input: default@over10k +#### A masked pattern was here #### +POSTHOOK: query: select bo, rank() over (partition by i order by bo nulls first, b nulls last range between unbounded preceding and unbounded following) from over10k limit 100 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@over10k +#### A masked pattern was here #### +false 1 +false 2 +false 3 +false 4 +false 5 +false 6 +false 7 +false 8 +false 9 +false 10 +false 11 +false 11 +false 13 +false 14 +false 15 +false 16 +false 17 +false 18 +false 19 +false 20 +false 20 +false 22 +true 23 +true 24 +true 25 +true 26 +true 27 +true 28 +true 29 +true 30 +true 31 +true 32 +true 33 +true 34 +true 35 +true 36 +true 37 +true 37 +true 39 +true 40 +true 41 +true 42 +true 43 +true 44 +true 45 +false 1 +false 2 +false 3 +false 4 +false 5 +false 5 +false 5 +false 8 +false 9 +false 10 +false 11 +false 12 +false 13 +false 14 +false 15 +false 16 +false 17 +true 18 +true 19 +true 20 +true 21 +true 22 +true 23 +true 24 +true 25 +true 26 +true 27 +true 27 +true 29 +true 30 +true 31 +true 32 +true 33 +true 34 +true 35 +false 1 +false 2 +false 3 +false 4 +false 4 +false 6 +false 7 +false 8 +false 9 +false 10 +false 11 +false 12 +false 13 +false 14 +false 15 +false 16 +false 17 +false 18 +true 19 +true 20 +PREHOOK: query: select CAST(s as CHAR(12)), rank() over (partition by i order by CAST(s as CHAR(12)) nulls last range between unbounded preceding and unbounded following) from over10k limit 100 +PREHOOK: type: QUERY +PREHOOK: Input: default@over10k +#### A masked pattern was here #### +POSTHOOK: query: select CAST(s as CHAR(12)), rank() over (partition by i order by CAST(s as CHAR(12)) nulls last range between unbounded preceding and unbounded following) from over10k limit 100 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@over10k +#### A masked pattern was here #### +alice ichabo 1 +alice robins 2 +bob robinson 3 +calvin thomp 4 +david johnso 5 +david laerte 6 +david nixon 7 +david nixon 7 +ethan johnso 9 +ethan ovid 10 +ethan underh 11 +fred miller 12 +fred miller 12 +gabriella ga 14 +gabriella un 15 +holly white 16 +irene johnso 17 +katie elliso 18 +luke allen 19 +mike quirini 20 +mike white 21 +nick davidso 22 +oscar allen 23 +oscar garcia 24 +oscar ichabo 25 +oscar ovid 26 +oscar steinb 27 +priscilla ga 28 +priscilla wh 29 +priscilla xy 30 +priscilla yo 31 +rachel brown 32 +rachel ichab 33 +rachel xylop 34 +sarah thomps 35 +sarah thomps 35 +tom johnson 37 +tom steinbec 38 +ulysses polk 39 +victor johns 40 +wendy polk 41 +xavier david 42 +yuri ellison 43 +zach allen 44 +zach hernand 45 +alice elliso 1 +bob carson 2 +calvin brown 3 +david xyloph 4 +ethan white 5 +fred johnson 6 +fred van bur 7 +gabriella ic 8 +holly laerte 9 +holly quirin 10 +jessica hern 11 +katie robins 12 +katie thomps 13 +luke nixon 14 +mike garcia 15 +mike hernand 16 +nick carson 17 +nick davidso 18 +oscar carson 19 +oscar robins 20 +priscilla wh 21 +sarah falkne 22 +sarah ichabo 23 +ulysses falk 24 +victor xylop 25 +wendy garcia 26 +wendy van bu 27 +xavier under 28 +yuri garcia 29 +yuri quirini 30 +yuri white 31 +zach falkner 32 +zach ichabod 33 +zach nixon 34 +zach ovid 35 +alice ichabo 1 +alice king 2 +alice robins 3 +calvin allen 4 +gabriella jo 5 +gabriella ni 6 +holly falkne 7 +holly hernan 8 +holly thomps 9 +katie nixon 10 +luke brown 11 +luke davidso 12 +luke white 13 +mike brown 14 +nick quirini 15 +oscar white 16 +priscilla xy 17 +quinn garcia 18 +quinn laerte 19 +rachel young 20 +PREHOOK: query: select CAST(s as VARCHAR(12)), rank() over (partition by i order by CAST(s as VARCHAR(12)) nulls last range between unbounded preceding and unbounded following) from over10k limit 100 +PREHOOK: type: QUERY +PREHOOK: Input: default@over10k +#### A masked pattern was here #### +POSTHOOK: query: select CAST(s as VARCHAR(12)), rank() over (partition by i order by CAST(s as VARCHAR(12)) nulls last range between unbounded preceding and unbounded following) from over10k limit 100 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@over10k +#### A masked pattern was here #### +alice ichabo 1 +alice robins 2 +bob robinson 3 +calvin thomp 4 +david johnso 5 +david laerte 6 +david nixon 7 +david nixon 7 +ethan johnso 9 +ethan ovid 10 +ethan underh 11 +fred miller 12 +fred miller 12 +gabriella ga 14 +gabriella un 15 +holly white 16 +irene johnso 17 +katie elliso 18 +luke allen 19 +mike quirini 20 +mike white 21 +nick davidso 22 +oscar allen 23 +oscar garcia 24 +oscar ichabo 25 +oscar ovid 26 +oscar steinb 27 +priscilla ga 28 +priscilla wh 29 +priscilla xy 30 +priscilla yo 31 +rachel brown 32 +rachel ichab 33 +rachel xylop 34 +sarah thomps 35 +sarah thomps 35 +tom johnson 37 +tom steinbec 38 +ulysses polk 39 +victor johns 40 +wendy polk 41 +xavier david 42 +yuri ellison 43 +zach allen 44 +zach hernand 45 +alice elliso 1 +bob carson 2 +calvin brown 3 +david xyloph 4 +ethan white 5 +fred johnson 6 +fred van bur 7 +gabriella ic 8 +holly laerte 9 +holly quirin 10 +jessica hern 11 +katie robins 12 +katie thomps 13 +luke nixon 14 +mike garcia 15 +mike hernand 16 +nick carson 17 +nick davidso 18 +oscar carson 19 +oscar robins 20 +priscilla wh 21 +sarah falkne 22 +sarah ichabo 23 +ulysses falk 24 +victor xylop 25 +wendy garcia 26 +wendy van bu 27 +xavier under 28 +yuri garcia 29 +yuri quirini 30 +yuri white 31 +zach falkner 32 +zach ichabod 33 +zach nixon 34 +zach ovid 35 +alice ichabo 1 +alice king 2 +alice robins 3 +calvin allen 4 +gabriella jo 5 +gabriella ni 6 +holly falkne 7 +holly hernan 8 +holly thomps 9 +katie nixon 10 +luke brown 11 +luke davidso 12 +luke white 13 +mike brown 14 +nick quirini 15 +oscar white 16 +priscilla xy 17 +quinn garcia 18 +quinn laerte 19 +rachel young 20 diff --git a/service/src/java/org/apache/hive/service/server/HiveServer2.java b/service/src/java/org/apache/hive/service/server/HiveServer2.java index d61edf50df576af582aa9e9fb7112f8e72224043..14e1e0d305c59595cbd54da7e15b2c1bb572d7ca 100644 --- a/service/src/java/org/apache/hive/service/server/HiveServer2.java +++ b/service/src/java/org/apache/hive/service/server/HiveServer2.java @@ -50,6 +50,8 @@ import org.apache.hadoop.hive.common.metrics.common.MetricsFactory; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.llap.coordinator.LlapCoordinator; +import org.apache.hadoop.hive.llap.io.api.LlapProxy; import org.apache.hadoop.hive.ql.exec.spark.session.SparkSessionManagerImpl; import org.apache.hadoop.hive.ql.exec.tez.TezSessionPoolManager; import org.apache.hadoop.hive.ql.util.ZooKeeperHiveHelper; @@ -133,6 +135,14 @@ public void run() { } catch (Throwable t) { throw new Error("Unable to intitialize HiveServer2", t); } + if (HiveConf.getBoolVar(hiveConf, ConfVars.LLAP_HS2_ENABLE_COORDINATOR)) { + // See method comment. + try { + LlapCoordinator.initializeInstance(hiveConf); + } catch (IOException e) { + throw new RuntimeException(e); + } + } // Setup web UI try { if (hiveConf.getBoolVar(ConfVars.HIVE_IN_TEST)) { diff --git a/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java b/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java index 68fac17311474108389fdf2dcea2baa52504ffd9..571bb251b4a56f471d074c0b81ae7b9f85d4e8a4 100644 --- a/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java +++ b/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java @@ -1266,9 +1266,9 @@ public long getFileId(FileSystem fs, String path) throws IOException { java.lang.reflect.Constructor ctor; String error = null; try { - method = clazz.getMethod("getSubject"); + method = clazz.getDeclaredMethod("getSubject"); method.setAccessible(true); - ctor = clazz.getConstructor(Subject.class); + ctor = clazz.getDeclaredConstructor(Subject.class); ctor.setAccessible(true); } catch (Throwable t) { error = t.getMessage();