You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jd...@apache.org on 2016/05/04 09:22:04 UTC
[01/10] hive git commit: HIVE-13445 : LLAP: token should encode
application and cluster ids (Sergey Shelukhin, reviewed by Siddharth Seth)
Repository: hive
Updated Branches:
refs/heads/llap bc75d72b8 -> e05790973
HIVE-13445 : LLAP: token should encode application and cluster ids (Sergey Shelukhin, reviewed by Siddharth Seth)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/868e5e14
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/868e5e14
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/868e5e14
Branch: refs/heads/llap
Commit: 868e5e141856ce75af48d854d9e3eb13372d11f4
Parents: b621827
Author: Sergey Shelukhin <se...@apache.org>
Authored: Tue May 3 12:01:32 2016 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Tue May 3 13:38:03 2016 -0700
----------------------------------------------------------------------
.../daemon/rpc/LlapDaemonProtocolProtos.java | 209 +++++++++++++++++--
.../org/apache/hadoop/hive/llap/DaemonId.java | 41 ++++
.../hive/llap/security/LlapTokenIdentifier.java | 39 +++-
.../hive/llap/security/LlapTokenProvider.java | 2 +-
.../src/protobuf/LlapDaemonProtocol.proto | 1 +
.../hive/llap/daemon/ContainerRunner.java | 9 +-
.../llap/daemon/impl/ContainerRunnerImpl.java | 47 +++--
.../hive/llap/daemon/impl/LlapDaemon.java | 52 ++++-
.../daemon/impl/LlapProtocolServerImpl.java | 41 ++--
.../hive/llap/daemon/impl/LlapTokenChecker.java | 137 ++++++++++++
.../hadoop/hive/llap/daemon/impl/QueryInfo.java | 17 +-
.../hive/llap/daemon/impl/QueryTracker.java | 85 +++++---
.../hadoop/hive/llap/daemon/impl/Scheduler.java | 2 +
.../llap/daemon/impl/TaskExecutorService.java | 9 +
.../hive/llap/security/LlapSecurityHelper.java | 15 +-
.../hive/llap/security/SecretManager.java | 19 +-
.../hive/llap/daemon/MiniLlapCluster.java | 2 +-
.../daemon/impl/TaskExecutorTestHelpers.java | 2 +-
.../impl/TestLlapDaemonProtocolServerImpl.java | 2 +-
.../llap/daemon/impl/TestLlapTokenChecker.java | 96 +++++++++
.../hive/ql/exec/tez/TezSessionState.java | 3 +-
21 files changed, 702 insertions(+), 128 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/daemon/rpc/LlapDaemonProtocolProtos.java
----------------------------------------------------------------------
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 4ab7b32..820f6be 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
@@ -12821,6 +12821,21 @@ public final class LlapDaemonProtocolProtos {
public interface GetTokenRequestProtoOrBuilder
extends com.google.protobuf.MessageOrBuilder {
+
+ // optional string app_id = 1;
+ /**
+ * <code>optional string app_id = 1;</code>
+ */
+ boolean hasAppId();
+ /**
+ * <code>optional string app_id = 1;</code>
+ */
+ java.lang.String getAppId();
+ /**
+ * <code>optional string app_id = 1;</code>
+ */
+ com.google.protobuf.ByteString
+ getAppIdBytes();
}
/**
* Protobuf type {@code GetTokenRequestProto}
@@ -12855,6 +12870,7 @@ public final class LlapDaemonProtocolProtos {
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 {
@@ -12872,6 +12888,11 @@ public final class LlapDaemonProtocolProtos {
}
break;
}
+ case 10: {
+ bitField0_ |= 0x00000001;
+ appId_ = input.readBytes();
+ break;
+ }
}
}
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -12911,7 +12932,52 @@ public final class LlapDaemonProtocolProtos {
return PARSER;
}
+ private int bitField0_;
+ // optional string app_id = 1;
+ public static final int APP_ID_FIELD_NUMBER = 1;
+ private java.lang.Object appId_;
+ /**
+ * <code>optional string app_id = 1;</code>
+ */
+ public boolean hasAppId() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>optional string app_id = 1;</code>
+ */
+ public java.lang.String getAppId() {
+ java.lang.Object ref = appId_;
+ 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()) {
+ appId_ = s;
+ }
+ return s;
+ }
+ }
+ /**
+ * <code>optional string app_id = 1;</code>
+ */
+ public com.google.protobuf.ByteString
+ getAppIdBytes() {
+ java.lang.Object ref = appId_;
+ if (ref instanceof java.lang.String) {
+ com.google.protobuf.ByteString b =
+ com.google.protobuf.ByteString.copyFromUtf8(
+ (java.lang.String) ref);
+ appId_ = b;
+ return b;
+ } else {
+ return (com.google.protobuf.ByteString) ref;
+ }
+ }
+
private void initFields() {
+ appId_ = "";
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
@@ -12925,6 +12991,9 @@ public final class LlapDaemonProtocolProtos {
public void writeTo(com.google.protobuf.CodedOutputStream output)
throws java.io.IOException {
getSerializedSize();
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ output.writeBytes(1, getAppIdBytes());
+ }
getUnknownFields().writeTo(output);
}
@@ -12934,6 +13003,10 @@ public final class LlapDaemonProtocolProtos {
if (size != -1) return size;
size = 0;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBytesSize(1, getAppIdBytes());
+ }
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
return size;
@@ -12957,6 +13030,11 @@ public final class LlapDaemonProtocolProtos {
org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetTokenRequestProto other = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetTokenRequestProto) obj;
boolean result = true;
+ result = result && (hasAppId() == other.hasAppId());
+ if (hasAppId()) {
+ result = result && getAppId()
+ .equals(other.getAppId());
+ }
result = result &&
getUnknownFields().equals(other.getUnknownFields());
return result;
@@ -12970,6 +13048,10 @@ public final class LlapDaemonProtocolProtos {
}
int hash = 41;
hash = (19 * hash) + getDescriptorForType().hashCode();
+ if (hasAppId()) {
+ hash = (37 * hash) + APP_ID_FIELD_NUMBER;
+ hash = (53 * hash) + getAppId().hashCode();
+ }
hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash;
return hash;
@@ -13079,6 +13161,8 @@ public final class LlapDaemonProtocolProtos {
public Builder clear() {
super.clear();
+ appId_ = "";
+ bitField0_ = (bitField0_ & ~0x00000001);
return this;
}
@@ -13105,6 +13189,13 @@ public final class LlapDaemonProtocolProtos {
public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetTokenRequestProto buildPartial() {
org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetTokenRequestProto result = new org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetTokenRequestProto(this);
+ int from_bitField0_ = bitField0_;
+ int to_bitField0_ = 0;
+ if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+ to_bitField0_ |= 0x00000001;
+ }
+ result.appId_ = appId_;
+ result.bitField0_ = to_bitField0_;
onBuilt();
return result;
}
@@ -13120,6 +13211,11 @@ public final class LlapDaemonProtocolProtos {
public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetTokenRequestProto other) {
if (other == org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetTokenRequestProto.getDefaultInstance()) return this;
+ if (other.hasAppId()) {
+ bitField0_ |= 0x00000001;
+ appId_ = other.appId_;
+ onChanged();
+ }
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
@@ -13145,6 +13241,81 @@ public final class LlapDaemonProtocolProtos {
}
return this;
}
+ private int bitField0_;
+
+ // optional string app_id = 1;
+ private java.lang.Object appId_ = "";
+ /**
+ * <code>optional string app_id = 1;</code>
+ */
+ public boolean hasAppId() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>optional string app_id = 1;</code>
+ */
+ public java.lang.String getAppId() {
+ java.lang.Object ref = appId_;
+ if (!(ref instanceof java.lang.String)) {
+ java.lang.String s = ((com.google.protobuf.ByteString) ref)
+ .toStringUtf8();
+ appId_ = s;
+ return s;
+ } else {
+ return (java.lang.String) ref;
+ }
+ }
+ /**
+ * <code>optional string app_id = 1;</code>
+ */
+ public com.google.protobuf.ByteString
+ getAppIdBytes() {
+ java.lang.Object ref = appId_;
+ if (ref instanceof String) {
+ com.google.protobuf.ByteString b =
+ com.google.protobuf.ByteString.copyFromUtf8(
+ (java.lang.String) ref);
+ appId_ = b;
+ return b;
+ } else {
+ return (com.google.protobuf.ByteString) ref;
+ }
+ }
+ /**
+ * <code>optional string app_id = 1;</code>
+ */
+ public Builder setAppId(
+ java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000001;
+ appId_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional string app_id = 1;</code>
+ */
+ public Builder clearAppId() {
+ bitField0_ = (bitField0_ & ~0x00000001);
+ appId_ = getDefaultInstance().getAppId();
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional string app_id = 1;</code>
+ */
+ public Builder setAppIdBytes(
+ com.google.protobuf.ByteString value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000001;
+ appId_ = value;
+ onChanged();
+ return this;
+ }
// @@protoc_insertion_point(builder_scope:GetTokenRequestProto)
}
@@ -14414,24 +14585,24 @@ public final class LlapDaemonProtocolProtos {
"RequestProto\022/\n\020query_identifier\030\001 \001(\0132\025" +
".QueryIdentifierProto\022\"\n\032fragment_identi" +
"fier_string\030\002 \001(\t\" \n\036TerminateFragmentRe" +
- "sponseProto\"\026\n\024GetTokenRequestProto\"&\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"
+ "sponseProto\"&\n\024GetTokenRequestProto\022\016\n\006a",
+ "pp_id\030\001 \001(\t\"&\n\025GetTokenResponseProto\022\r\n\005" +
+ "token\030\001 \001(\014*2\n\020SourceStateProto\022\017\n\013S_SUC" +
+ "CEEDED\020\001\022\r\n\tS_RUNNING\020\002*E\n\024SubmissionSta" +
+ "teProto\022\014\n\010ACCEPTED\020\001\022\014\n\010REJECTED\020\002\022\021\n\rE" +
+ "VICTED_OTHER\020\0032\316\002\n\022LlapDaemonProtocol\022?\n" +
+ "\nsubmitWork\022\027.SubmitWorkRequestProto\032\030.S" +
+ "ubmitWorkResponseProto\022W\n\022sourceStateUpd" +
+ "ated\022\037.SourceStateUpdatedRequestProto\032 ." +
+ "SourceStateUpdatedResponseProto\022H\n\rquery" +
+ "Complete\022\032.QueryCompleteRequestProto\032\033.Q",
+ "ueryCompleteResponseProto\022T\n\021terminateFr" +
+ "agment\022\036.TerminateFragmentRequestProto\032\037" +
+ ".TerminateFragmentResponseProto2]\n\026LlapM" +
+ "anagementProtocol\022C\n\022getDelegationToken\022" +
+ "\025.GetTokenRequestProto\032\026.GetTokenRespons" +
+ "eProtoBH\n&org.apache.hadoop.hive.llap.da" +
+ "emon.rpcB\030LlapDaemonProtocolProtos\210\001\001\240\001\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -14533,7 +14704,7 @@ public final class LlapDaemonProtocolProtos {
internal_static_GetTokenRequestProto_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_GetTokenRequestProto_descriptor,
- new java.lang.String[] { });
+ new java.lang.String[] { "AppId", });
internal_static_GetTokenResponseProto_descriptor =
getDescriptor().getMessageTypes().get(16);
internal_static_GetTokenResponseProto_fieldAccessorTable = new
http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-common/src/java/org/apache/hadoop/hive/llap/DaemonId.java
----------------------------------------------------------------------
diff --git a/llap-common/src/java/org/apache/hadoop/hive/llap/DaemonId.java b/llap-common/src/java/org/apache/hadoop/hive/llap/DaemonId.java
new file mode 100644
index 0000000..18355e6
--- /dev/null
+++ b/llap-common/src/java/org/apache/hadoop/hive/llap/DaemonId.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed 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;
+
+public class DaemonId {
+ private final String userName;
+ private final String clusterName;
+ private final String appId;
+ private final String hostName;
+ private final long startTime;
+
+ public DaemonId(String userName, String clusterName, String hostName, String appId,
+ long startTime) {
+ this.userName = userName;
+ this.clusterName = clusterName;
+ this.appId = appId;
+ this.hostName = hostName;
+ this.startTime = startTime;
+ // TODO: we could also get an unique number per daemon.
+ }
+
+ public String getClusterString() {
+ return userName + "_" + clusterName + "_" + appId;
+ }
+
+ public String getApplicationId() {
+ return appId;
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapTokenIdentifier.java
----------------------------------------------------------------------
diff --git a/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapTokenIdentifier.java b/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapTokenIdentifier.java
index 23980d0..e28eddd 100644
--- a/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapTokenIdentifier.java
+++ b/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapTokenIdentifier.java
@@ -22,6 +22,7 @@ import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
+import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.token.Token;
@@ -31,25 +32,32 @@ import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdenti
public class LlapTokenIdentifier extends AbstractDelegationTokenIdentifier {
private static final String KIND = "LLAP_TOKEN";
public static final Text KIND_NAME = new Text(KIND);
+ private String clusterId;
+ private String appId;
public LlapTokenIdentifier() {
super();
}
- public LlapTokenIdentifier(Text owner, Text renewer, Text realUser) {
+ public LlapTokenIdentifier(Text owner, Text renewer, Text realUser,
+ String clusterId, String appId) {
super(owner, renewer, realUser);
+ this.clusterId = clusterId;
+ this.appId = appId == null ? "" : appId;
}
@Override
public void write(DataOutput out) throws IOException {
super.write(out);
- // Nothing right now.
+ out.writeUTF(clusterId);
+ out.writeUTF(appId);
}
@Override
public void readFields(DataInput in) throws IOException {
super.readFields(in);
- // Nothing right now.
+ clusterId = in.readUTF();
+ appId = in.readUTF();
}
@Override
@@ -57,21 +65,34 @@ public class LlapTokenIdentifier extends AbstractDelegationTokenIdentifier {
return KIND_NAME;
}
+ public String getAppId() {
+ return appId;
+ }
+
+ public String getClusterId() {
+ return clusterId;
+ }
+
@Override
public int hashCode() {
- // Nothing else right now.
- return super.hashCode();
+ final int prime = 31;
+ int result = prime * super.hashCode() + ((appId == null) ? 0 : appId.hashCode());
+ return prime * result + ((clusterId == null) ? 0 : clusterId.hashCode());
}
@Override
- public boolean equals(Object other) {
- // Nothing else right now.
- return super.equals(other);
+ public boolean equals(Object obj) {
+ if (this == obj) return true;
+ if (!(obj instanceof LlapTokenIdentifier) || !super.equals(obj)) return false;
+ LlapTokenIdentifier other = (LlapTokenIdentifier) obj;
+ return (appId == null ? other.appId == null : appId.equals(other.appId))
+ && (clusterId == null ? other.clusterId == null : clusterId.equals(other.clusterId));
}
@Override
public String toString() {
- return KIND + "; " + super.toString();
+ return KIND + "; " + super.toString() + ", cluster " + clusterId + ", app secret hash "
+ + (StringUtils.isBlank(appId) ? 0 : appId.hashCode());
}
@InterfaceAudience.Private
http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapTokenProvider.java
----------------------------------------------------------------------
diff --git a/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapTokenProvider.java b/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapTokenProvider.java
index 2e99a28..edf9b18 100644
--- a/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapTokenProvider.java
+++ b/llap-common/src/java/org/apache/hadoop/hive/llap/security/LlapTokenProvider.java
@@ -23,5 +23,5 @@ import java.io.IOException;
import org.apache.hadoop.security.token.Token;
public interface LlapTokenProvider {
- Token<LlapTokenIdentifier> getDelegationToken() throws IOException;
+ Token<LlapTokenIdentifier> getDelegationToken(String appId) throws IOException;
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-common/src/protobuf/LlapDaemonProtocol.proto
----------------------------------------------------------------------
diff --git a/llap-common/src/protobuf/LlapDaemonProtocol.proto b/llap-common/src/protobuf/LlapDaemonProtocol.proto
index 944c96c..5cdc02e 100644
--- a/llap-common/src/protobuf/LlapDaemonProtocol.proto
+++ b/llap-common/src/protobuf/LlapDaemonProtocol.proto
@@ -130,6 +130,7 @@ message TerminateFragmentResponseProto {
}
message GetTokenRequestProto {
+ optional string app_id = 1;
}
message GetTokenResponseProto {
http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/ContainerRunner.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/ContainerRunner.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/ContainerRunner.java
index fc29371..c346aed 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/ContainerRunner.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/ContainerRunner.java
@@ -29,9 +29,12 @@ public interface ContainerRunner {
SubmitWorkResponseProto submitWork(SubmitWorkRequestProto request) throws IOException;
- SourceStateUpdatedResponseProto sourceStateUpdated(SourceStateUpdatedRequestProto request);
+ SourceStateUpdatedResponseProto sourceStateUpdated(
+ SourceStateUpdatedRequestProto request) throws IOException;
- QueryCompleteResponseProto queryComplete(QueryCompleteRequestProto request);
+ QueryCompleteResponseProto queryComplete(
+ QueryCompleteRequestProto request) throws IOException;
- TerminateFragmentResponseProto terminateFragment(TerminateFragmentRequestProto request);
+ TerminateFragmentResponseProto terminateFragment(
+ TerminateFragmentRequestProto request) throws IOException;
}
http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
----------------------------------------------------------------------
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 3d45c7a..78b37f7 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
@@ -92,7 +92,7 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
boolean enablePreemption, String[] localDirsBase, AtomicReference<Integer> localShufflePort,
AtomicReference<InetSocketAddress> localAddress,
long totalMemoryAvailableBytes, LlapDaemonExecutorMetrics metrics,
- AMReporter amReporter, ClassLoader classLoader) {
+ AMReporter amReporter, ClassLoader classLoader, String clusterId) {
super("ContainerRunnerImpl");
this.conf = conf;
Preconditions.checkState(numExecutors > 0,
@@ -101,7 +101,7 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
this.localShufflePort = localShufflePort;
this.amReporter = amReporter;
- this.queryTracker = new QueryTracker(conf, localDirsBase);
+ this.queryTracker = new QueryTracker(conf, localDirsBase, clusterId);
addIfService(queryTracker);
String waitQueueSchedulerClassName = HiveConf.getVar(
conf, ConfVars.LLAP_DAEMON_WAIT_QUEUE_COMPARATOR_CLASS_NAME);
@@ -175,7 +175,8 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
fragmentSpec.getFragmentIdentifierString());
int dagIdentifier = taskAttemptId.getTaskID().getVertexID().getDAGId().getId();
- QueryIdentifier queryIdentifier = new QueryIdentifier(request.getApplicationIdString(), dagIdentifier);
+ QueryIdentifier queryIdentifier = new QueryIdentifier(
+ request.getApplicationIdString(), dagIdentifier);
Credentials credentials = new Credentials();
DataInputBuffer dib = new DataInputBuffer();
@@ -193,6 +194,7 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
fragmentSpec.getAttemptNumber(), request.getUser(), request.getFragmentSpec(),
jobToken);
+
String[] localDirs = fragmentInfo.getLocalDirs();
Preconditions.checkNotNull(localDirs);
if (LOG.isDebugEnabled()) {
@@ -200,7 +202,6 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
}
// May need to setup localDir for re-localization, which is usually setup as Environment.PWD.
// Used for re-localization, to add the user specified configuration (conf_pb_binary_stream)
-
TaskRunnerCallable callable = new TaskRunnerCallable(request, fragmentInfo, new Configuration(getConfig()),
new LlapExecutionContext(localAddress.get().getHostName(), queryTracker), env,
credentials, memoryPerExecutor, amReporter, confParams, metrics, killedTaskHandler,
@@ -248,24 +249,23 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
@Override
public SourceStateUpdatedResponseProto sourceStateUpdated(
- SourceStateUpdatedRequestProto request) {
+ SourceStateUpdatedRequestProto request) throws IOException {
LOG.info("Processing state update: " + stringifySourceStateUpdateRequest(request));
- queryTracker.registerSourceStateChange(
- new QueryIdentifier(request.getQueryIdentifier().getAppIdentifier(),
- request.getQueryIdentifier().getDagIdentifier()), request.getSrcName(),
- request.getState());
+ QueryIdentifier queryId = new QueryIdentifier(request.getQueryIdentifier().getAppIdentifier(),
+ request.getQueryIdentifier().getDagIdentifier());
+ queryTracker.registerSourceStateChange(queryId, request.getSrcName(), request.getState());
return SourceStateUpdatedResponseProto.getDefaultInstance();
}
@Override
- public QueryCompleteResponseProto queryComplete(QueryCompleteRequestProto request) {
+ public QueryCompleteResponseProto queryComplete(
+ QueryCompleteRequestProto request) throws IOException {
QueryIdentifier queryIdentifier =
new QueryIdentifier(request.getQueryIdentifier().getAppIdentifier(),
request.getQueryIdentifier().getDagIdentifier());
LOG.info("Processing queryComplete notification for {}", queryIdentifier);
- List<QueryFragmentInfo> knownFragments =
- queryTracker
- .queryComplete(queryIdentifier, request.getDeleteDelay());
+ List<QueryFragmentInfo> knownFragments = queryTracker.queryComplete(
+ queryIdentifier, request.getDeleteDelay(), false);
LOG.info("DBG: Pending fragment count for completed query {} = {}", queryIdentifier,
knownFragments.size());
for (QueryFragmentInfo fragmentInfo : knownFragments) {
@@ -277,9 +277,16 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
}
@Override
- public TerminateFragmentResponseProto terminateFragment(TerminateFragmentRequestProto request) {
- LOG.info("DBG: Received terminateFragment request for {}", request.getFragmentIdentifierString());
- executorService.killFragment(request.getFragmentIdentifierString());
+ public TerminateFragmentResponseProto terminateFragment(
+ TerminateFragmentRequestProto request) throws IOException {
+ String fragmentId = request.getFragmentIdentifierString();
+ LOG.info("DBG: Received terminateFragment request for {}", fragmentId);
+ // TODO: ideally, QueryTracker should have fragment-to-query mapping.
+ QueryIdentifier queryId = executorService.findQueryByFragment(fragmentId);
+ // checkPermissions returns false if query is not found, throws on failure.
+ if (queryId != null && queryTracker.checkPermissionsForQuery(queryId)) {
+ executorService.killFragment(fragmentId);
+ }
return TerminateFragmentResponseProto.getDefaultInstance();
}
@@ -355,8 +362,12 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
@Override
public void queryFailed(QueryIdentifier queryIdentifier) {
LOG.info("Processing query failed notification for {}", queryIdentifier);
- List<QueryFragmentInfo> knownFragments =
- queryTracker.queryComplete(queryIdentifier, -1);
+ List<QueryFragmentInfo> knownFragments;
+ try {
+ knownFragments = queryTracker.queryComplete(queryIdentifier, -1, true);
+ } catch (IOException e) {
+ throw new RuntimeException(e); // Should never happen here, no permission check.
+ }
LOG.info("DBG: Pending fragment count for failed query {} = {}", queryIdentifier,
knownFragments.size());
for (QueryFragmentInfo fragmentInfo : knownFragments) {
http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
index 63cb16b..d23a44a 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
@@ -26,12 +26,14 @@ import java.util.List;
import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
+import java.util.regex.Pattern;
import javax.management.ObjectName;
import org.apache.hadoop.conf.Configuration;
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.configuration.LlapDaemonConfiguration;
import org.apache.hadoop.hive.llap.daemon.ContainerRunner;
@@ -57,11 +59,13 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge;
import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge.UdfWhitelistChecker;
import org.apache.hadoop.metrics2.util.MBeans;
+import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.service.CompositeService;
import org.apache.hadoop.util.ExitUtil;
import org.apache.hadoop.util.JvmPauseMonitor;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.util.ConverterUtils;
import org.apache.hive.common.util.ShutdownHookManager;
import org.apache.logging.log4j.core.config.Configurator;
import org.slf4j.Logger;
@@ -97,6 +101,13 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
private final int numExecutors;
private final long maxJvmMemory;
private final String[] localDirs;
+ private final DaemonId daemonId;
+
+ private final static Pattern hostsRe = Pattern.compile("[^A-Za-z0-9_-]");
+ private static String generateClusterName(Configuration conf) {
+ String hosts = HiveConf.getTrimmedVar(conf, ConfVars.LLAP_DAEMON_SERVICE_HOSTS);
+ return hostsRe.matcher(hosts.startsWith("@") ? hosts.substring(1) : hosts).replaceAll("_");
+ }
// TODO Not the best way to share the address
private final AtomicReference<InetSocketAddress> srvAddress = new AtomicReference<>(),
@@ -105,11 +116,10 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
public LlapDaemon(Configuration daemonConf, int numExecutors, long executorMemoryBytes,
boolean ioEnabled, boolean isDirectCache, long ioMemoryBytes, String[] localDirs, int srvPort,
- int mngPort, int shufflePort, int webPort) {
+ int mngPort, int shufflePort, int webPort, String appName) {
super("LlapDaemon");
initializeLogging();
-
printAsciiArt();
Preconditions.checkArgument(numExecutors > 0);
@@ -129,6 +139,14 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
"LLAP service hosts startswith '@' but hive.zookeeper.quorum is not set." +
" hive.zookeeper.quorum must be set.");
}
+ String hostName = MetricsUtils.getHostName();
+ try {
+ daemonId = new DaemonId(UserGroupInformation.getCurrentUser().getUserName(),
+ generateClusterName(daemonConf), hostName, appName, System.currentTimeMillis());
+ } catch (IOException ex) {
+ throw new RuntimeException(ex);
+ }
+
this.maxJvmMemory = getTotalHeapSize();
this.llapIoEnabled = ioEnabled;
@@ -193,7 +211,7 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
LlapMetricsSystem.initialize("LlapDaemon");
this.pauseMonitor = new JvmPauseMonitor(daemonConf);
pauseMonitor.start();
- String displayName = "LlapDaemonExecutorMetrics-" + MetricsUtils.getHostName();
+ String displayName = "LlapDaemonExecutorMetrics-" + hostName;
String sessionId = MetricsUtils.getUUID();
daemonConf.set("llap.daemon.metrics.sessionid", sessionId);
String[] strIntervals = HiveConf.getTrimmedStringsVar(daemonConf,
@@ -223,11 +241,11 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
this.amReporter = new AMReporter(srvAddress, new QueryFailedHandlerProxy(), daemonConf);
this.server = new LlapProtocolServerImpl(
- numHandlers, this, srvAddress, mngAddress, srvPort, mngPort);
+ numHandlers, this, srvAddress, mngAddress, srvPort, mngPort, daemonId);
this.containerRunner = new ContainerRunnerImpl(daemonConf, numExecutors, waitQueueSize,
enablePreemption, localDirs, this.shufflePort, srvAddress, executorMemoryBytes, metrics,
- amReporter, executorClassLoader);
+ amReporter, executorClassLoader, daemonId.getClusterString());
addIfService(containerRunner);
// Not adding the registry as a service, since we need to control when it is initialized - conf used to pickup properties.
@@ -377,10 +395,18 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
LlapDaemonConfiguration daemonConf = new LlapDaemonConfiguration();
String containerIdStr = System.getenv(ApplicationConstants.Environment.CONTAINER_ID.name());
+ String appName = null;
if (containerIdStr != null && !containerIdStr.isEmpty()) {
daemonConf.set(ConfVars.LLAP_DAEMON_CONTAINER_ID.varname, containerIdStr);
+ appName = ConverterUtils.toContainerId(containerIdStr)
+ .getApplicationAttemptId().getApplicationId().toString();
} else {
daemonConf.unset(ConfVars.LLAP_DAEMON_CONTAINER_ID.varname);
+ // Note, we assume production LLAP always runs under YARN.
+ LOG.error("Cannot find " + ApplicationConstants.Environment.CONTAINER_ID.toString()
+ + "; LLAP tokens may grant access to subsequent instances of the cluster with"
+ + " the same name");
+ appName = null;
}
int numExecutors = HiveConf.getIntVar(daemonConf, ConfVars.LLAP_DAEMON_NUM_EXECUTORS);
@@ -400,7 +426,8 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
boolean isDirectCache = HiveConf.getBoolVar(daemonConf, ConfVars.LLAP_ALLOCATOR_DIRECT);
boolean isLlapIo = HiveConf.getBoolVar(daemonConf, HiveConf.ConfVars.LLAP_IO_ENABLED, true);
llapDaemon = new LlapDaemon(daemonConf, numExecutors, executorMemoryBytes, isLlapIo,
- isDirectCache, ioMemoryBytes, localDirs, rpcPort, mngPort, shufflePort, webPort);
+ isDirectCache, ioMemoryBytes, localDirs, rpcPort, mngPort, shufflePort, webPort,
+ appName);
LOG.info("Adding shutdown hook for LlapDaemon");
ShutdownHookManager.addShutdownHook(new CompositeServiceShutdownHook(llapDaemon), 1);
@@ -420,24 +447,27 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
}
@Override
- public SubmitWorkResponseProto submitWork(SubmitWorkRequestProto request) throws
- IOException {
+ public SubmitWorkResponseProto submitWork(
+ SubmitWorkRequestProto request) throws IOException {
numSubmissions.incrementAndGet();
return containerRunner.submitWork(request);
}
@Override
- public SourceStateUpdatedResponseProto sourceStateUpdated(SourceStateUpdatedRequestProto request) {
+ public SourceStateUpdatedResponseProto sourceStateUpdated(
+ SourceStateUpdatedRequestProto request) throws IOException {
return containerRunner.sourceStateUpdated(request);
}
@Override
- public QueryCompleteResponseProto queryComplete(QueryCompleteRequestProto request) {
+ public QueryCompleteResponseProto queryComplete(
+ QueryCompleteRequestProto request) throws IOException {
return containerRunner.queryComplete(request);
}
@Override
- public TerminateFragmentResponseProto terminateFragment(TerminateFragmentRequestProto request) {
+ public TerminateFragmentResponseProto terminateFragment(
+ TerminateFragmentRequestProto request) throws IOException {
return containerRunner.terminateFragment(request);
}
http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapProtocolServerImpl.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapProtocolServerImpl.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapProtocolServerImpl.java
index dae1a3a..db8bfa6 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapProtocolServerImpl.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapProtocolServerImpl.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
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.daemon.rpc.LlapDaemonProtocolProtos;
import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetTokenRequestProto;
import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetTokenResponseProto;
@@ -71,13 +72,11 @@ public class LlapProtocolServerImpl extends AbstractService
private final AtomicReference<InetSocketAddress> srvAddress, mngAddress;
private SecretManager zkSecretManager;
private String restrictedToUser = null;
+ private final DaemonId daemonId;
- public LlapProtocolServerImpl(int numHandlers,
- ContainerRunner containerRunner,
- AtomicReference<InetSocketAddress> srvAddress,
- AtomicReference<InetSocketAddress> mngAddress,
- int srvPort,
- int mngPort) {
+ public LlapProtocolServerImpl(int numHandlers, ContainerRunner containerRunner,
+ AtomicReference<InetSocketAddress> srvAddress, AtomicReference<InetSocketAddress> mngAddress,
+ int srvPort, int mngPort, DaemonId daemonId) {
super("LlapDaemonProtocolServerImpl");
this.numHandlers = numHandlers;
this.containerRunner = containerRunner;
@@ -85,14 +84,14 @@ public class LlapProtocolServerImpl extends AbstractService
this.srvPort = srvPort;
this.mngAddress = mngAddress;
this.mngPort = mngPort;
+ this.daemonId = daemonId;
LOG.info("Creating: " + LlapProtocolServerImpl.class.getSimpleName() +
" with port configured to: " + srvPort);
}
@Override
public SubmitWorkResponseProto submitWork(RpcController controller,
- SubmitWorkRequestProto request) throws
- ServiceException {
+ SubmitWorkRequestProto request) throws ServiceException {
try {
return containerRunner.submitWork(request);
} catch (IOException e) {
@@ -103,20 +102,31 @@ public class LlapProtocolServerImpl extends AbstractService
@Override
public SourceStateUpdatedResponseProto sourceStateUpdated(RpcController controller,
SourceStateUpdatedRequestProto request) throws ServiceException {
- return containerRunner.sourceStateUpdated(request);
+ try {
+ return containerRunner.sourceStateUpdated(request);
+ } catch (IOException e) {
+ throw new ServiceException(e);
+ }
}
@Override
public QueryCompleteResponseProto queryComplete(RpcController controller,
QueryCompleteRequestProto request) throws ServiceException {
- return containerRunner.queryComplete(request);
+ try {
+ return containerRunner.queryComplete(request);
+ } catch (IOException e) {
+ throw new ServiceException(e);
+ }
}
@Override
public TerminateFragmentResponseProto terminateFragment(
- RpcController controller,
- TerminateFragmentRequestProto request) throws ServiceException {
- return containerRunner.terminateFragment(request);
+ RpcController controller, TerminateFragmentRequestProto request) throws ServiceException {
+ try {
+ return containerRunner.terminateFragment(request);
+ } catch (IOException e) {
+ throw new ServiceException(e);
+ }
}
@Override
@@ -140,7 +150,7 @@ public class LlapProtocolServerImpl extends AbstractService
}
String llapPrincipal = HiveConf.getVar(conf, ConfVars.LLAP_KERBEROS_PRINCIPAL),
llapKeytab = HiveConf.getVar(conf, ConfVars.LLAP_KERBEROS_KEYTAB_FILE);
- zkSecretManager = SecretManager.createSecretManager(conf, llapPrincipal, llapKeytab);
+ zkSecretManager = SecretManager.createSecretManager(conf, llapPrincipal, llapKeytab, daemonId);
// Start the protocol server after properly authenticating with daemon keytab.
UserGroupInformation daemonUgi = null;
@@ -275,7 +285,8 @@ public class LlapProtocolServerImpl extends AbstractService
realUser = new Text(ugi.getRealUser().getUserName());
}
Text renewer = new Text(ugi.getShortUserName());
- LlapTokenIdentifier llapId = new LlapTokenIdentifier(owner, renewer, realUser);
+ LlapTokenIdentifier llapId = new LlapTokenIdentifier(owner, renewer, realUser,
+ daemonId.getClusterString(), request.hasAppId() ? request.getAppId() : null);
// TODO: note that the token is not renewable right now and will last for 2 weeks by default.
Token<LlapTokenIdentifier> token = new Token<LlapTokenIdentifier>(llapId, zkSecretManager);
if (LOG.isInfoEnabled()) {
http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapTokenChecker.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapTokenChecker.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapTokenChecker.java
new file mode 100644
index 0000000..03ee055
--- /dev/null
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapTokenChecker.java
@@ -0,0 +1,137 @@
+/*
+ * Licensed 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.daemon.impl;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import java.util.ArrayList;
+
+import java.util.List;
+
+import java.io.IOException;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hive.llap.security.LlapTokenIdentifier;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public final class LlapTokenChecker {
+ private static final Logger LOG = LoggerFactory.getLogger(LlapTokenChecker.class);
+
+ private static final ImmutablePair<String, String> NO_SECURITY = new ImmutablePair<>(null, null);
+ public static Pair<String, String> getTokenInfo(String clusterId) throws IOException {
+ if (!UserGroupInformation.isSecurityEnabled()) return NO_SECURITY;
+ UserGroupInformation current = UserGroupInformation.getCurrentUser();
+ String kerberosName = current.hasKerberosCredentials() ? current.getShortUserName() : null;
+ List<LlapTokenIdentifier> tokens = getLlapTokens(current, clusterId);
+ if ((tokens == null || tokens.isEmpty()) && kerberosName == null) {
+ throw new SecurityException("No tokens or kerberos for " + current);
+ }
+ return getTokenInfoInternal(kerberosName, tokens);
+ }
+
+ private static List<LlapTokenIdentifier> getLlapTokens(
+ UserGroupInformation ugi, String clusterId) {
+ List<LlapTokenIdentifier> tokens = null;
+ for (TokenIdentifier id : ugi.getTokenIdentifiers()) {
+ if (!LlapTokenIdentifier.KIND_NAME.equals(id.getKind())) continue;
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Token {}", id);
+ }
+ LlapTokenIdentifier llapId = (LlapTokenIdentifier)id;
+ if (!clusterId.equals(llapId.getClusterId())) continue;
+ if (tokens == null) {
+ tokens = new ArrayList<>();
+ }
+ tokens.add((LlapTokenIdentifier)id);
+ }
+ return tokens;
+ }
+
+ @VisibleForTesting
+ static Pair<String, String> getTokenInfoInternal(
+ String kerberosName, List<LlapTokenIdentifier> tokens) {
+ assert (tokens != null && !tokens.isEmpty()) || kerberosName != null;
+ if (tokens == null) {
+ return new ImmutablePair<String, String>(kerberosName, null);
+ }
+ String userName = kerberosName, appId = null;
+ for (LlapTokenIdentifier llapId : tokens) {
+ String newUserName = llapId.getRealUser().toString();
+ if (userName != null && !userName.equals(newUserName)) {
+ throw new SecurityException("Ambiguous user name from credentials - " + userName
+ + " and " + newUserName + " from " + llapId
+ + ((kerberosName == null) ? ("; has kerberos credentials for " + kerberosName) : ""));
+ }
+ userName = newUserName;
+ String newAppId = llapId.getAppId();
+ if (!StringUtils.isEmpty(newAppId)) {
+ if (!StringUtils.isEmpty(appId) && !appId.equals(newAppId)) {
+ throw new SecurityException("Ambiguous app ID from credentials - " + appId
+ + " and " + newAppId + " from " + llapId);
+ }
+ appId = newAppId;
+ }
+ }
+ assert userName != null;
+ return new ImmutablePair<String, String>(userName, appId);
+ }
+
+ public static void checkPermissions(
+ String clusterId, String userName, String appId, Object hint) throws IOException {
+ if (!UserGroupInformation.isSecurityEnabled()) return;
+ UserGroupInformation current = UserGroupInformation.getCurrentUser();
+ String kerberosName = current.hasKerberosCredentials() ? current.getShortUserName() : null;
+ List<LlapTokenIdentifier> tokens = getLlapTokens(current, clusterId);
+ checkPermissionsInternal(kerberosName, tokens, userName, appId, hint);
+ }
+
+ @VisibleForTesting
+ static void checkPermissionsInternal(String kerberosName, List<LlapTokenIdentifier> tokens,
+ String userName, String appId, Object hint) {
+ if (kerberosName != null && StringUtils.isEmpty(appId) && kerberosName.equals(userName)) {
+ return;
+ }
+ if (tokens != null) {
+ for (LlapTokenIdentifier llapId : tokens) {
+ String tokenUser = llapId.getRealUser().toString(), tokenAppId = llapId.getAppId();
+ if (checkTokenPermissions(userName, appId, tokenUser, tokenAppId)) return;
+ }
+ }
+ throw new SecurityException("Unauthorized to access "
+ + userName + ", " + appId.hashCode() + " (" + hint + ")");
+ }
+
+ public static void checkPermissions(
+ Pair<String, String> prm, String userName, String appId, Object hint) {
+ if (userName == null) {
+ assert StringUtils.isEmpty(appId);
+ return;
+ }
+ if (!checkTokenPermissions(userName, appId, prm.getLeft(), prm.getRight())) {
+ throw new SecurityException("Unauthorized to access "
+ + userName + ", " + appId.hashCode() + " (" + hint + ")");
+ }
+ }
+
+ private static boolean checkTokenPermissions(
+ String userName, String appId, String tokenUser, String tokenAppId) {
+ return userName.equals(tokenUser)
+ && (StringUtils.isEmpty(appId) || appId.equals(tokenAppId));
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryInfo.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryInfo.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryInfo.java
index 64c2b58..8daef9e 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryInfo.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryInfo.java
@@ -55,10 +55,11 @@ public class QueryInfo {
private final ConcurrentMap<String, SourceStateProto> sourceStateMap;
private final FinishableStateTracker finishableStateTracker = new FinishableStateTracker();
+ private final String tokenUserName, appId;
- public QueryInfo(QueryIdentifier queryIdentifier, String appIdString, String dagName, int dagIdentifier,
- String user, ConcurrentMap<String, SourceStateProto> sourceStateMap,
- String[] localDirsBase, FileSystem localFs) {
+ public QueryInfo(QueryIdentifier queryIdentifier, String appIdString, String dagName,
+ int dagIdentifier, String user, ConcurrentMap<String, SourceStateProto> sourceStateMap,
+ String[] localDirsBase, FileSystem localFs, String tokenUserName, String tokenAppId) {
this.queryIdentifier = queryIdentifier;
this.appIdString = appIdString;
this.dagName = dagName;
@@ -67,6 +68,8 @@ public class QueryInfo {
this.user = user;
this.localDirsBase = localDirsBase;
this.localFs = localFs;
+ this.tokenUserName = tokenUserName;
+ this.appId = tokenAppId;
}
public QueryIdentifier getQueryIdentifier() {
@@ -270,4 +273,12 @@ public class QueryInfo {
this.lastFinishableState = lastFinishableState;
}
}
+
+ public String getTokenUserName() {
+ return tokenUserName;
+ }
+
+ public String getTokenAppId() {
+ return appId;
+ }
}
http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryTracker.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryTracker.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryTracker.java
index 14657e6..cb3be2b 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryTracker.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryTracker.java
@@ -18,6 +18,8 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.security.token.Token;
import org.apache.tez.common.CallableWithNdc;
@@ -60,6 +62,7 @@ public class QueryTracker extends AbstractService {
private final String[] localDirsBase;
private final FileSystem localFs;
+ private final String clusterId;
private final long defaultDeleteDelaySeconds;
// TODO At the moment there's no way of knowing whether a query is running or not.
@@ -89,9 +92,10 @@ public class QueryTracker extends AbstractService {
private final ConcurrentHashMap<QueryIdentifier, String> queryIdentifierToHiveQueryId =
new ConcurrentHashMap<>();
- public QueryTracker(Configuration conf, String[] localDirsBase) {
+ public QueryTracker(Configuration conf, String[] localDirsBase, String clusterId) {
super("QueryTracker");
this.localDirsBase = localDirsBase;
+ this.clusterId = clusterId;
try {
localFs = FileSystem.getLocal(conf);
} catch (IOException e) {
@@ -119,35 +123,50 @@ public class QueryTracker extends AbstractService {
* @param user
* @throws IOException
*/
- QueryFragmentInfo registerFragment(QueryIdentifier queryIdentifier, String appIdString, String dagName,
- int dagIdentifier, String vertexName, int fragmentNumber, int attemptNumber, String user,
- FragmentSpecProto fragmentSpec, Token<JobTokenIdentifier> appToken) throws IOException {
+ QueryFragmentInfo registerFragment(QueryIdentifier queryIdentifier, String appIdString,
+ String dagName, int dagIdentifier, String vertexName, int fragmentNumber, int attemptNumber,
+ String user, FragmentSpecProto fragmentSpec, Token<JobTokenIdentifier> appToken)
+ throws IOException {
ReadWriteLock dagLock = getDagLock(queryIdentifier);
dagLock.readLock().lock();
try {
- if (!completedDagMap.contains(queryIdentifier)) {
- QueryInfo queryInfo = queryInfoMap.get(queryIdentifier);
- if (queryInfo == null) {
- queryInfo = new QueryInfo(queryIdentifier, appIdString, dagName, dagIdentifier, user,
- getSourceCompletionMap(queryIdentifier), localDirsBase, localFs);
- queryInfoMap.putIfAbsent(queryIdentifier, queryInfo);
- }
-
- if (LOG.isDebugEnabled()) {
- LOG.debug("Registering request for {} with the ShuffleHandler", queryIdentifier);
- }
- ShuffleHandler.get()
- .registerDag(appIdString, dagIdentifier, appToken,
- user, queryInfo.getLocalDirs());
-
- return queryInfo.registerFragment(vertexName, fragmentNumber, attemptNumber, fragmentSpec);
- } else {
+ if (completedDagMap.contains(queryIdentifier)) {
// Cleanup the dag lock here, since it may have been created after the query completed
dagSpecificLocks.remove(queryIdentifier);
throw new RuntimeException(
"Dag " + dagName + " already complete. Rejecting fragment ["
+ vertexName + ", " + fragmentNumber + ", " + attemptNumber + "]");
}
+ // TODO: for now, we get the secure username out of UGI... after signing, we can take it
+ // out of the request provided that it's signed.
+ Pair<String, String> tokenInfo = LlapTokenChecker.getTokenInfo(clusterId);
+ boolean isExistingQueryInfo = true;
+ QueryInfo queryInfo = queryInfoMap.get(queryIdentifier);
+ if (queryInfo == null) {
+ queryInfo = new QueryInfo(queryIdentifier, appIdString, dagName, dagIdentifier, user,
+ getSourceCompletionMap(queryIdentifier), localDirsBase, localFs,
+ tokenInfo.getLeft(), tokenInfo.getRight());
+ QueryInfo old = queryInfoMap.putIfAbsent(queryIdentifier, queryInfo);
+ if (old != null) {
+ queryInfo = old;
+ } else {
+ isExistingQueryInfo = false;
+ }
+ }
+ if (isExistingQueryInfo) {
+ // We already retrieved the incoming info, check without UGI.
+ LlapTokenChecker.checkPermissions(tokenInfo, queryInfo.getTokenUserName(),
+ queryInfo.getTokenAppId(), queryInfo.getQueryIdentifier());
+ }
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Registering request for {} with the ShuffleHandler", queryIdentifier);
+ }
+ ShuffleHandler.get()
+ .registerDag(appIdString, dagIdentifier, appToken,
+ user, queryInfo.getLocalDirs());
+
+ return queryInfo.registerFragment(vertexName, fragmentNumber, attemptNumber, fragmentSpec);
} finally {
dagLock.readLock().unlock();
}
@@ -174,17 +193,20 @@ public class QueryTracker extends AbstractService {
* @param queryIdentifier
* @param deleteDelay
*/
- List<QueryFragmentInfo> queryComplete(QueryIdentifier queryIdentifier, long deleteDelay) {
+ List<QueryFragmentInfo> queryComplete(QueryIdentifier queryIdentifier, long deleteDelay,
+ boolean isInternal) throws IOException {
if (deleteDelay == -1) {
deleteDelay = defaultDeleteDelaySeconds;
}
ReadWriteLock dagLock = getDagLock(queryIdentifier);
dagLock.writeLock().lock();
try {
+ QueryInfo queryInfo = isInternal
+ ? queryInfoMap.get(queryIdentifier) : checkPermissionsAndGetQuery(queryIdentifier);
rememberCompletedDag(queryIdentifier);
LOG.info("Processing queryComplete for queryIdentifier={} with deleteDelay={} seconds", queryIdentifier,
deleteDelay);
- QueryInfo queryInfo = queryInfoMap.remove(queryIdentifier);
+ queryInfoMap.remove(queryIdentifier);
if (queryInfo == null) {
LOG.warn("Ignoring query complete for unknown dag: {}", queryIdentifier);
return Collections.emptyList();
@@ -229,9 +251,10 @@ public class QueryTracker extends AbstractService {
* @param sourceName
* @param sourceState
*/
- void registerSourceStateChange(QueryIdentifier queryIdentifier, String sourceName, SourceStateProto sourceState) {
+ void registerSourceStateChange(QueryIdentifier queryIdentifier, String sourceName,
+ SourceStateProto sourceState) throws IOException {
getSourceCompletionMap(queryIdentifier).put(sourceName, sourceState);
- QueryInfo queryInfo = queryInfoMap.get(queryIdentifier);
+ QueryInfo queryInfo = checkPermissionsAndGetQuery(queryIdentifier);
if (queryInfo != null) {
queryInfo.sourceStateUpdated(sourceName);
} else {
@@ -322,4 +345,16 @@ public class QueryTracker extends AbstractService {
return null;
}
}
+
+ private QueryInfo checkPermissionsAndGetQuery(QueryIdentifier queryId) throws IOException {
+ QueryInfo queryInfo = queryInfoMap.get(queryId);
+ if (queryInfo == null) return null;
+ LlapTokenChecker.checkPermissions(clusterId, queryInfo.getTokenAppId(),
+ queryInfo.getTokenUserName(), queryInfo.getQueryIdentifier());
+ return queryInfo;
+ }
+
+ public boolean checkPermissionsForQuery(QueryIdentifier queryId) throws IOException {
+ return checkPermissionsAndGetQuery(queryId) != null;
+ }
}
http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/Scheduler.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/Scheduler.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/Scheduler.java
index 26c8e55..fd6234a 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/Scheduler.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/Scheduler.java
@@ -44,4 +44,6 @@ public interface Scheduler<T> {
void killFragment(String fragmentId);
Set<String> getExecutorsStatus();
+
+ QueryIdentifier findQueryByFragment(String fragmentId);
}
http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorService.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorService.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorService.java
index f621af2..1933eb1 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorService.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorService.java
@@ -420,6 +420,15 @@ public class TaskExecutorService extends AbstractService implements Scheduler<Ta
}
@Override
+ public QueryIdentifier findQueryByFragment(String fragmentId) {
+ synchronized (lock) {
+ TaskWrapper taskWrapper = knownTasks.get(fragmentId);
+ return taskWrapper == null ? null : taskWrapper.getTaskRunnerCallable()
+ .getFragmentInfo().getQueryInfo().getQueryIdentifier();
+ }
+ }
+
+ @Override
public void killFragment(String fragmentId) {
synchronized (lock) {
TaskWrapper taskWrapper = knownTasks.remove(fragmentId);
http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-server/src/java/org/apache/hadoop/hive/llap/security/LlapSecurityHelper.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/security/LlapSecurityHelper.java b/llap-server/src/java/org/apache/hadoop/hive/llap/security/LlapSecurityHelper.java
index 76ba225..f958bc4 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/security/LlapSecurityHelper.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/security/LlapSecurityHelper.java
@@ -26,6 +26,7 @@ import java.util.concurrent.TimeUnit;
import javax.net.SocketFactory;
+import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hive.llap.impl.LlapManagementProtocolClientImpl;
import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GetTokenRequestProto;
@@ -81,7 +82,7 @@ public class LlapSecurityHelper implements LlapTokenProvider {
}
@Override
- public Token<LlapTokenIdentifier> getDelegationToken() throws IOException {
+ public Token<LlapTokenIdentifier> getDelegationToken(String appId) throws IOException {
if (!UserGroupInformation.isSecurityEnabled()) return null;
if (llapUgi == null) {
llapUgi = UserGroupInformation.getCurrentUser();
@@ -98,7 +99,7 @@ public class LlapSecurityHelper implements LlapTokenProvider {
boolean hasRefreshed = false;
while (true) {
try {
- tokenBytes = getTokenBytes();
+ tokenBytes = getTokenBytes(appId);
break;
} catch (InterruptedException ie) {
throw new RuntimeException(ie);
@@ -128,7 +129,8 @@ public class LlapSecurityHelper implements LlapTokenProvider {
return token;
}
- private ByteString getTokenBytes() throws InterruptedException, IOException {
+ private ByteString getTokenBytes(
+ final String appId) throws InterruptedException, IOException {
return llapUgi.doAs(new PrivilegedExceptionAction<ByteString>() {
@Override
public ByteString run() throws Exception {
@@ -138,8 +140,11 @@ public class LlapSecurityHelper implements LlapTokenProvider {
clientInstance.getManagementPort(), retryPolicy, socketFactory);
}
// Client only connects on the first call, so this has to be done in doAs.
- GetTokenRequestProto req = GetTokenRequestProto.newBuilder().build();
- return client.getDelegationToken(null, req).getToken();
+ GetTokenRequestProto.Builder req = GetTokenRequestProto.newBuilder();
+ if (!StringUtils.isBlank(appId)) {
+ req.setAppId(appId);
+ }
+ return client.getDelegationToken(null, req.build()).getToken();
}
});
}
http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-server/src/java/org/apache/hadoop/hive/llap/security/SecretManager.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/security/SecretManager.java b/llap-server/src/java/org/apache/hadoop/hive/llap/security/SecretManager.java
index 8c7a539..c54e726 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/security/SecretManager.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/security/SecretManager.java
@@ -23,6 +23,7 @@ import java.util.regex.Pattern;
import org.apache.hadoop.conf.Configuration;
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.security.LlapTokenIdentifier;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
@@ -33,6 +34,7 @@ import org.slf4j.LoggerFactory;
public class SecretManager extends ZKDelegationTokenSecretManager<LlapTokenIdentifier> {
private static final Logger LOG = LoggerFactory.getLogger(SecretManager.class);
+
public SecretManager(Configuration conf) {
super(conf);
checkForZKDTSMBug(conf);
@@ -82,16 +84,8 @@ public class SecretManager extends ZKDelegationTokenSecretManager<LlapTokenIdent
return id;
}
- private final static Pattern hostsRe = Pattern.compile("[^A-Za-z0-9_-]");
- private static String deriveZkPath(Configuration conf) throws IOException {
- String hosts = HiveConf.getTrimmedVar(conf, ConfVars.LLAP_DAEMON_SERVICE_HOSTS);
- String clusterName = hosts.startsWith("@") ? hosts.substring(1) : hosts;
- String userName = UserGroupInformation.getCurrentUser().getShortUserName();
- return hostsRe.matcher(userName + "_" + clusterName).replaceAll("_") ;
- }
-
public static SecretManager createSecretManager(
- final Configuration conf, String llapPrincipal, String llapKeytab) {
+ final Configuration conf, String llapPrincipal, String llapKeytab, DaemonId daemonId) {
// Create ZK connection under a separate ugi (if specified) - ZK works in mysterious ways.
UserGroupInformation zkUgi = null;
String principal = HiveConf.getVar(conf, ConfVars.LLAP_ZKSM_KERBEROS_PRINCIPAL, llapPrincipal);
@@ -110,12 +104,7 @@ public class SecretManager extends ZKDelegationTokenSecretManager<LlapTokenIdent
zkConf.setLong(DelegationTokenManager.RENEW_INTERVAL, tokenLifetime);
zkConf.set(SecretManager.ZK_DTSM_ZK_KERBEROS_PRINCIPAL, principal);
zkConf.set(SecretManager.ZK_DTSM_ZK_KERBEROS_KEYTAB, keyTab);
- String zkPath;
- try {
- zkPath = deriveZkPath(conf);
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
+ String zkPath = daemonId.getClusterString();
LOG.info("Using {} as ZK secret manager path", zkPath);
zkConf.set(SecretManager.ZK_DTSM_ZNODE_WORKING_PATH, "zkdtsm_" + zkPath);
setZkConfIfNotSet(zkConf, SecretManager.ZK_DTSM_ZK_AUTH_TYPE, "sasl");
http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java
----------------------------------------------------------------------
diff --git a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java
index 610f266..dde5be0 100644
--- a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java
+++ b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java
@@ -193,7 +193,7 @@ public class MiniLlapCluster extends AbstractService {
LOG.info("Initializing {} llap instances for MiniLlapCluster with name={}", numInstances, clusterNameTrimmed);
for (int i = 0 ;i < numInstances ; i++) {
llapDaemons[i] = new LlapDaemon(conf, numExecutorsPerService, execBytesPerService, llapIoEnabled,
- ioIsDirect, ioBytesPerService, localDirs, rpcPort, mngPort, shufflePort, webPort);
+ ioIsDirect, ioBytesPerService, localDirs, rpcPort, mngPort, shufflePort, webPort, clusterNameTrimmed);
llapDaemons[i].init(new Configuration(conf));
}
LOG.info("Initialized {} llap instances for MiniLlapCluster with name={}", numInstances, clusterNameTrimmed);
http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorTestHelpers.java
----------------------------------------------------------------------
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 24f4442..c6ba14e 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
@@ -78,7 +78,7 @@ public class TaskExecutorTestHelpers {
QueryInfo queryInfo =
new QueryInfo(queryIdentifier, "fake_app_id_string", "fake_dag_name", 1, "fakeUser",
new ConcurrentHashMap<String, LlapDaemonProtocolProtos.SourceStateProto>(),
- new String[0], null);
+ new String[0], null, null, null);
return queryInfo;
}
http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestLlapDaemonProtocolServerImpl.java
----------------------------------------------------------------------
diff --git a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestLlapDaemonProtocolServerImpl.java b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestLlapDaemonProtocolServerImpl.java
index a65bf5c..fd37a06 100644
--- a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestLlapDaemonProtocolServerImpl.java
+++ b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestLlapDaemonProtocolServerImpl.java
@@ -46,7 +46,7 @@ public class TestLlapDaemonProtocolServerImpl {
LlapProtocolServerImpl server =
new LlapProtocolServerImpl(numHandlers, containerRunnerMock,
new AtomicReference<InetSocketAddress>(), new AtomicReference<InetSocketAddress>(),
- rpcPort, rpcPort + 1);
+ rpcPort, rpcPort + 1, null);
when(containerRunnerMock.submitWork(any(SubmitWorkRequestProto.class))).thenReturn(
SubmitWorkResponseProto
.newBuilder()
http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestLlapTokenChecker.java
----------------------------------------------------------------------
diff --git a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestLlapTokenChecker.java b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestLlapTokenChecker.java
new file mode 100644
index 0000000..aaaa762
--- /dev/null
+++ b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestLlapTokenChecker.java
@@ -0,0 +1,96 @@
+/**
+ * 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.daemon.impl;
+
+import static org.junit.Assert.*;
+
+import org.apache.hadoop.io.Text;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hive.llap.security.LlapTokenIdentifier;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.Test;
+
+public class TestLlapTokenChecker {
+
+ @Test
+ public void testGetToken() {
+ check(LlapTokenChecker.getTokenInfoInternal("u", null), "u", null);
+ check(LlapTokenChecker.getTokenInfoInternal(null, createTokens("u", null)), "u", null);
+ check(LlapTokenChecker.getTokenInfoInternal(null, createTokens("u", "a")), "u", "a");
+ check(LlapTokenChecker.getTokenInfoInternal("u", createTokens("u", "a")), "u", "a");
+ check(LlapTokenChecker.getTokenInfoInternal("u", createTokens("u", "a", "u", null)),
+ "u", "a");
+ // Note - some of these scenarios could be handled, but they are not supported right now.
+ // The reason is that we bind a query to app/user using the signed token information, and
+ // we don't want to bother figuring out which one to use in case of ambiguity w/o a use case.
+ checkGetThrows("u", createTokens("u2", "a")); // Ambiguous user.
+ checkGetThrows("u2", createTokens("u2", "a", "u3", "a")); // Ambiguous user.
+ checkGetThrows(null, createTokens("u2", "a", "u3", "a")); // Ambiguous user.
+ checkGetThrows(null, createTokens("u2", "a", "u2", "a1")); // Ambiguous app.
+ }
+
+ @Test
+ public void testCheckPermissions() {
+ LlapTokenChecker.checkPermissionsInternal("u", null, "u", null, null);
+ LlapTokenChecker.checkPermissionsInternal(null, createTokens("u", null) , "u", null, null);
+ LlapTokenChecker.checkPermissionsInternal("u", createTokens("u", "a") , "u", "a", null);
+ // No access.
+ checkPrmThrows("u2", null, "u", "a");
+ checkPrmThrows("u", null, "u", "a"); // Note - Kerberos user w/o appId doesn't have access.
+ checkPrmThrows(null, createTokens("u2", "a"), "u", "a");
+ checkPrmThrows(null, createTokens("u", "a2"), "u", "a");
+ checkPrmThrows(null, createTokens("u", null), "u", "a");
+ }
+
+ private List<LlapTokenIdentifier> createTokens(String... args) {
+ List<LlapTokenIdentifier> tokens = new ArrayList<>();
+ for (int i = 0; i < args.length; i += 2) {
+ tokens.add(new LlapTokenIdentifier(null, null, new Text(args[i]), "c", args[i + 1]));
+ }
+ return tokens;
+ }
+
+ private void checkGetThrows(String kerberosName, List<LlapTokenIdentifier> tokens) {
+ try {
+ LlapTokenChecker.getTokenInfoInternal(kerberosName, tokens);
+ fail("Didn't throw");
+ } catch (SecurityException ex) {
+ // Expected.
+ }
+ }
+
+ private void checkPrmThrows(
+ String kerberosName, List<LlapTokenIdentifier> tokens, String userName, String appId) {
+ try {
+ LlapTokenChecker.checkPermissionsInternal(kerberosName, tokens, userName, appId, null);
+ fail("Didn't throw");
+ } catch (SecurityException ex) {
+ // Expected.
+ }
+ }
+
+ private void check(Pair<String, String> p, String user, String appId) {
+ assertEquals(user, p.getLeft());
+ assertEquals(appId, p.getRight());
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hive/blob/868e5e14/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java
----------------------------------------------------------------------
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 3ea5ef9..fd6465a 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
@@ -275,7 +275,8 @@ public class TezSessionState {
if (llapMode) {
if (UserGroupInformation.isSecurityEnabled()) {
LlapTokenProvider tp = LlapProxy.getOrInitTokenProvider(conf);
- Token<LlapTokenIdentifier> token = tp.getDelegationToken();
+ // For Tez, we don't use appId to distinguish the tokens; security scope is the user.
+ Token<LlapTokenIdentifier> token = tp.getDelegationToken(null);
if (LOG.isInfoEnabled()) {
LOG.info("Obtained a LLAP token: " + token);
}
[10/10] hive git commit: Merge branch 'master' into llap
Posted by jd...@apache.org.
Merge branch 'master' into llap
Conflicts:
llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/daemon/rpc/LlapDaemonProtocolProtos.java
llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/e0579097
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/e0579097
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/e0579097
Branch: refs/heads/llap
Commit: e057909732b40b581fcad3f61fb798600f01ecdf
Parents: 4847f65 8729966
Author: Jason Dere <jd...@hortonworks.com>
Authored: Wed May 4 00:17:12 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Wed May 4 00:17:12 2016 -0700
----------------------------------------------------------------------
HIVE-13509.2.patch | 478 --
.../ext/LlapTaskUmbilicalExternalClient.java | 18 +-
.../daemon/rpc/LlapDaemonProtocolProtos.java | 7000 +++++++++++-------
.../org/apache/hadoop/hive/llap/DaemonId.java | 41 +
.../hive/llap/security/LlapTokenIdentifier.java | 39 +-
.../hive/llap/security/LlapTokenProvider.java | 2 +-
.../apache/hadoop/hive/llap/tez/Converters.java | 84 +-
.../src/protobuf/LlapDaemonProtocol.proto | 70 +-
.../hadoop/hive/llap/tez/TestConverters.java | 51 +-
.../hadoop/hive/llap/LlapBaseInputFormat.java | 32 +-
.../hive/llap/daemon/ContainerRunner.java | 9 +-
.../llap/daemon/impl/ContainerRunnerImpl.java | 135 +-
.../hive/llap/daemon/impl/LlapDaemon.java | 52 +-
.../daemon/impl/LlapProtocolServerImpl.java | 41 +-
.../hive/llap/daemon/impl/LlapTokenChecker.java | 137 +
.../llap/daemon/impl/QueryFragmentInfo.java | 23 +-
.../hadoop/hive/llap/daemon/impl/QueryInfo.java | 26 +-
.../hive/llap/daemon/impl/QueryTracker.java | 97 +-
.../hadoop/hive/llap/daemon/impl/Scheduler.java | 2 +
.../llap/daemon/impl/TaskExecutorService.java | 17 +-
.../llap/daemon/impl/TaskRunnerCallable.java | 77 +-
.../hive/llap/security/LlapSecurityHelper.java | 15 +-
.../hive/llap/security/SecretManager.java | 19 +-
.../hive/llap/daemon/MiniLlapCluster.java | 2 +-
.../daemon/impl/TaskExecutorTestHelpers.java | 44 +-
.../impl/TestLlapDaemonProtocolServerImpl.java | 2 +-
.../llap/daemon/impl/TestLlapTokenChecker.java | 96 +
.../TestFirstInFirstOutComparator.java | 27 +-
.../llap/tezplugins/LlapTaskCommunicator.java | 31 +-
.../hadoop/hive/metastore/txn/TxnHandler.java | 5 +
.../hive/ql/exec/tez/TezSessionState.java | 3 +-
.../hadoop/hive/ql/lockmgr/DbTxnManager.java | 12 +
.../hadoop/hive/ql/parse/SemanticAnalyzer.java | 1 -
.../ql/parse/UpdateDeleteSemanticAnalyzer.java | 16 +-
.../apache/hadoop/hive/ql/TestTxnCommands2.java | 22 +
.../hive/ql/lockmgr/TestDbTxnManager2.java | 114 +
.../dynpart_sort_optimization_acid.q.out | 120 +-
37 files changed, 5479 insertions(+), 3481 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/e0579097/llap-client/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
----------------------------------------------------------------------
diff --cc llap-client/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
index fe2fd7c,0000000..6e2c85d
mode 100644,000000..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
@@@ -1,413 -1,0 +1,421 @@@
+package org.apache.hadoop.hive.llap.ext;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.collections4.ListUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos;
++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.protocol.LlapTaskUmbilicalProtocol;
++import org.apache.hadoop.hive.llap.tez.Converters;
+import org.apache.hadoop.hive.llap.tez.LlapProtocolClientProxy;
+import org.apache.hadoop.hive.llap.tezplugins.helpers.LlapTaskUmbilicalServer;
+import org.apache.hadoop.io.Text;
+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;
+import org.apache.tez.runtime.api.impl.TezHeartbeatResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class LlapTaskUmbilicalExternalClient extends AbstractService {
+
+ private static final Logger LOG = LoggerFactory.getLogger(LlapTaskUmbilicalExternalClient.class);
+
+ private final LlapProtocolClientProxy communicator;
+ private volatile LlapTaskUmbilicalServer llapTaskUmbilicalServer;
+ private final Configuration conf;
+ private final LlapTaskUmbilicalProtocol umbilical;
+
+ protected final String tokenIdentifier;
+ protected final Token<JobTokenIdentifier> sessionToken;
+
+ private final ConcurrentMap<String, PendingEventData> pendingEvents = new ConcurrentHashMap<>();
+ private final ConcurrentMap<String, TaskHeartbeatInfo> registeredTasks= new ConcurrentHashMap<String, TaskHeartbeatInfo>();
+ private LlapTaskUmbilicalExternalResponder responder = null;
+ private final ScheduledThreadPoolExecutor timer;
+ private final long connectionTimeout;
+
+ private static class TaskHeartbeatInfo {
+ final String taskAttemptId;
+ final String hostname;
+ final int port;
+ final AtomicLong lastHeartbeat = new AtomicLong();
+
+ public TaskHeartbeatInfo(String taskAttemptId, String hostname, int port) {
+ this.taskAttemptId = taskAttemptId;
+ this.hostname = hostname;
+ this.port = port;
+ this.lastHeartbeat.set(System.currentTimeMillis());
+ }
+ }
+
+ private static class PendingEventData {
+ final TaskHeartbeatInfo heartbeatInfo;
+ final List<TezEvent> tezEvents;
+
+ public PendingEventData(TaskHeartbeatInfo heartbeatInfo, List<TezEvent> tezEvents) {
+ this.heartbeatInfo = heartbeatInfo;
+ this.tezEvents = tezEvents;
+ }
+ }
+
+ // TODO KKK Work out the details of the tokenIdentifier, and the session token.
+ // It may just be possible to create one here - since Shuffle is not involved, and this is only used
+ // for communication from LLAP-Daemons to the server. It will need to be sent in as part
+ // of the job submission request.
+ public LlapTaskUmbilicalExternalClient(Configuration conf, String tokenIdentifier,
+ Token<JobTokenIdentifier> sessionToken, LlapTaskUmbilicalExternalResponder responder) {
+ super(LlapTaskUmbilicalExternalClient.class.getName());
+ this.conf = conf;
+ this.umbilical = new LlapTaskUmbilicalExternalImpl();
+ this.tokenIdentifier = tokenIdentifier;
+ this.sessionToken = sessionToken;
+ this.responder = responder;
+ this.timer = new ScheduledThreadPoolExecutor(1);
+ this.connectionTimeout = HiveConf.getTimeVar(conf,
+ HiveConf.ConfVars.LLAP_DAEMON_AM_LIVENESS_CONNECTION_TIMEOUT_MS, TimeUnit.MILLISECONDS);
+ // TODO. No support for the LLAP token yet. Add support for configurable threads, however 1 should always be enough.
+ this.communicator = new LlapProtocolClientProxy(1, conf, null);
+ this.communicator.init(conf);
+ }
+
+ @Override
+ public void serviceStart() throws IOException {
+ // If we use a single server for multiple external clients, then consider using more than one handler.
+ int numHandlers = 1;
+ llapTaskUmbilicalServer = new LlapTaskUmbilicalServer(conf, umbilical, numHandlers, tokenIdentifier, sessionToken);
+ communicator.start();
+ }
+
+ @Override
+ public void serviceStop() {
+ llapTaskUmbilicalServer.shutdownServer();
+ timer.shutdown();
+ if (this.communicator != null) {
+ this.communicator.stop();
+ }
+ }
+
+ public InetSocketAddress getAddress() {
+ return llapTaskUmbilicalServer.getAddress();
+ }
+
+
+ /**
+ * Submit the work for actual execution.
+ * @param submitWorkRequestProto
+ */
+ public void submitWork(final SubmitWorkRequestProto submitWorkRequestProto, String llapHost, int llapPort, List<TezEvent> tezEvents) {
+ // Register the pending events to be sent for this spec.
- String fragmentId = submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString();
++ SignableVertexSpec vertex = submitWorkRequestProto.getWorkSpec().getVertex();
++ VertexIdentifier vId = vertex.getVertexIdentifier();
++ TezTaskAttemptID attemptId = Converters.createTaskAttemptId(
++ vId, submitWorkRequestProto.getFragmentNumber(), submitWorkRequestProto.getAttemptNumber());
++ final String fragmentId = attemptId.toString();
++
+ PendingEventData pendingEventData = new PendingEventData(
+ new TaskHeartbeatInfo(fragmentId, llapHost, llapPort),
+ tezEvents);
+ pendingEvents.putIfAbsent(fragmentId, pendingEventData);
+
+ // 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,
+ new LlapProtocolClientProxy.ExecuteRequestCallback<LlapDaemonProtocolProtos.SubmitWorkResponseProto>() {
+
+ @Override
+ public void setResponse(LlapDaemonProtocolProtos.SubmitWorkResponseProto response) {
+ if (response.hasSubmissionState()) {
+ if (response.getSubmissionState().equals(LlapDaemonProtocolProtos.SubmissionStateProto.REJECTED)) {
- String msg = "Fragment: " + submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString() + " rejected. Server Busy.";
++ String msg = "Fragment: " + fragmentId + " rejected. Server Busy.";
+ LOG.info(msg);
+ if (responder != null) {
+ Throwable err = new RuntimeException(msg);
- responder.submissionFailed(submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString(), err);
++ responder.submissionFailed(fragmentId, err);
+ }
+ return;
+ }
+ }
+ }
+
+ @Override
+ public void indicateError(Throwable t) {
- String msg = "Failed to submit: " + submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString();
++ String msg = "Failed to submit: " + fragmentId;
+ LOG.error(msg, t);
+ Throwable err = new RuntimeException(msg, t);
- responder.submissionFailed(submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString(), err);
++ responder.submissionFailed(fragmentId, err);
+ }
+ });
+
+
+
+
+
+// // TODO Also send out information saying that the fragment is finishable - if that is not already included in the main fragment.
+// // This entire call is only required if we're doing more than scans. MRInput has no dependencies and is always finishable
+// QueryIdentifierProto queryIdentifier = QueryIdentifierProto
+// .newBuilder()
+// .setAppIdentifier(submitWorkRequestProto.getApplicationIdString()).setDagIdentifier(submitWorkRequestProto.getFragmentSpec().getDagId())
+// .build();
+// LlapDaemonProtocolProtos.SourceStateUpdatedRequestProto sourceStateUpdatedRequest =
+// LlapDaemonProtocolProtos.SourceStateUpdatedRequestProto.newBuilder().setQueryIdentifier(queryIdentifier).setState(
+// LlapDaemonProtocolProtos.SourceStateProto.S_SUCCEEDED).
+// setSrcName(TODO)
+// communicator.sendSourceStateUpdate(LlapDaemonProtocolProtos.SourceStateUpdatedRequestProto.newBuilder().setQueryIdentifier(submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString()).set);
+
+
+ }
+
+ private void updateHeartbeatInfo(String taskAttemptId) {
+ int updateCount = 0;
+
+ PendingEventData pendingEventData = pendingEvents.get(taskAttemptId);
+ if (pendingEventData != null) {
+ pendingEventData.heartbeatInfo.lastHeartbeat.set(System.currentTimeMillis());
+ updateCount++;
+ }
+
+ TaskHeartbeatInfo heartbeatInfo = registeredTasks.get(taskAttemptId);
+ if (heartbeatInfo != null) {
+ heartbeatInfo.lastHeartbeat.set(System.currentTimeMillis());
+ updateCount++;
+ }
+
+ if (updateCount == 0) {
+ LOG.warn("No tasks found for heartbeat from taskAttemptId " + taskAttemptId);
+ }
+ }
+
+ private void updateHeartbeatInfo(String hostname, int port) {
+ int updateCount = 0;
+
+ for (String key : pendingEvents.keySet()) {
+ PendingEventData pendingEventData = pendingEvents.get(key);
+ if (pendingEventData != null) {
+ if (pendingEventData.heartbeatInfo.hostname.equals(hostname)
+ && pendingEventData.heartbeatInfo.port == port) {
+ pendingEventData.heartbeatInfo.lastHeartbeat.set(System.currentTimeMillis());
+ updateCount++;
+ }
+ }
+ }
+
+ for (String key : registeredTasks.keySet()) {
+ TaskHeartbeatInfo heartbeatInfo = registeredTasks.get(key);
+ if (heartbeatInfo != null) {
+ if (heartbeatInfo.hostname.equals(hostname)
+ && heartbeatInfo.port == port) {
+ heartbeatInfo.lastHeartbeat.set(System.currentTimeMillis());
+ updateCount++;
+ }
+ }
+ }
+
+ if (updateCount == 0) {
+ LOG.info("No tasks found for heartbeat from hostname " + hostname + ", port " + port);
+ }
+ }
+
+ private class HeartbeatCheckTask implements Runnable {
+ public void run() {
+ long currentTime = System.currentTimeMillis();
+ List<String> timedOutTasks = new ArrayList<String>();
+
+ // Check both pending and registered tasks for timeouts
+ for (String key : pendingEvents.keySet()) {
+ PendingEventData pendingEventData = pendingEvents.get(key);
+ if (pendingEventData != null) {
+ if (currentTime - pendingEventData.heartbeatInfo.lastHeartbeat.get() >= connectionTimeout) {
+ timedOutTasks.add(key);
+ }
+ }
+ }
+ for (String timedOutTask : timedOutTasks) {
+ LOG.info("Pending taskAttemptId " + timedOutTask + " timed out");
+ responder.heartbeatTimeout(timedOutTask);
+ pendingEvents.remove(timedOutTask);
+ // TODO: Do we need to tell the LLAP daemon we are no longer interested in this task?
+ }
+
+ timedOutTasks.clear();
+ for (String key : registeredTasks.keySet()) {
+ TaskHeartbeatInfo heartbeatInfo = registeredTasks.get(key);
+ if (heartbeatInfo != null) {
+ if (currentTime - heartbeatInfo.lastHeartbeat.get() >= connectionTimeout) {
+ timedOutTasks.add(key);
+ }
+ }
+ }
+ for (String timedOutTask : timedOutTasks) {
+ LOG.info("Running taskAttemptId " + timedOutTask + " timed out");
+ responder.heartbeatTimeout(timedOutTask);
+ registeredTasks.remove(timedOutTask);
+ // TODO: Do we need to tell the LLAP daemon we are no longer interested in this task?
+ }
+ }
+ }
+
+ public interface LlapTaskUmbilicalExternalResponder {
+ void submissionFailed(String fragmentId, Throwable throwable);
+ void heartbeat(TezHeartbeatRequest request);
+ void taskKilled(TezTaskAttemptID taskAttemptId);
+ void heartbeatTimeout(String fragmentId);
+ }
+
+
+
+ // TODO Ideally, the server should be shared across all client sessions running on the same node.
+ private class LlapTaskUmbilicalExternalImpl implements LlapTaskUmbilicalProtocol {
+
+ @Override
+ public boolean canCommit(TezTaskAttemptID taskid) throws IOException {
+ // Expecting only a single instance of a task to be running.
+ return true;
+ }
+
+ @Override
+ public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request) throws IOException,
+ TezException {
+ // Keep-alive information. The client should be informed and will have to take care of re-submitting the work.
+ // Some parts of fault tolerance go here.
+
+ // This also provides completion information, and a possible notification when task actually starts running (first heartbeat)
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Received heartbeat from container, request=" + request);
+ }
+
+ // Incoming events can be ignored until the point when shuffle needs to be handled, instead of just scans.
+ TezHeartbeatResponse response = new TezHeartbeatResponse();
+
+ response.setLastRequestId(request.getRequestId());
+ // Assuming TaskAttemptId and FragmentIdentifierString are the same. Verify this.
+ TezTaskAttemptID taskAttemptId = request.getCurrentTaskAttemptID();
+ String taskAttemptIdString = taskAttemptId.toString();
+
+ updateHeartbeatInfo(taskAttemptIdString);
+
+ List<TezEvent> tezEvents = null;
+ PendingEventData pendingEventData = pendingEvents.remove(taskAttemptIdString);
+ if (pendingEventData == null) {
+ tezEvents = Collections.emptyList();
+
+ // If this heartbeat was not from a pending event and it's not in our list of registered tasks,
+ if (!registeredTasks.containsKey(taskAttemptIdString)) {
+ LOG.info("Unexpected heartbeat from " + taskAttemptIdString);
+ response.setShouldDie(); // Do any of the other fields need to be set?
+ return response;
+ }
+ } else {
+ tezEvents = pendingEventData.tezEvents;
+ // Tasks removed from the pending list should then be added to the registered list.
+ registeredTasks.put(taskAttemptIdString, pendingEventData.heartbeatInfo);
+ }
+
+ response.setLastRequestId(request.getRequestId());
+ // Irrelevant from eventIds. This can be tracked in the AM itself, instead of polluting the task.
+ // Also since we have all the MRInput events here - they'll all be sent in together.
+ response.setNextFromEventId(0); // Irrelevant. See comment above.
+ response.setNextPreRoutedEventId(0); //Irrelevant. See comment above.
+ response.setEvents(tezEvents);
+
+ List<TezEvent> inEvents = request.getEvents();
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Heartbeat from " + taskAttemptIdString +
+ " events: " + (inEvents != null ? inEvents.size() : -1));
+ }
+ for (TezEvent tezEvent : ListUtils.emptyIfNull(inEvents)) {
+ EventType eventType = tezEvent.getEventType();
+ switch (eventType) {
+ case TASK_ATTEMPT_COMPLETED_EVENT:
+ LOG.debug("Task completed event for " + taskAttemptIdString);
+ registeredTasks.remove(taskAttemptIdString);
+ break;
+ case TASK_ATTEMPT_FAILED_EVENT:
+ LOG.debug("Task failed event for " + taskAttemptIdString);
+ registeredTasks.remove(taskAttemptIdString);
+ break;
+ case TASK_STATUS_UPDATE_EVENT:
+ // If we want to handle counters
+ LOG.debug("Task update event for " + taskAttemptIdString);
+ break;
+ default:
+ LOG.warn("Unhandled event type " + eventType);
+ break;
+ }
+ }
+
+ // Pass the request on to the responder
+ try {
+ if (responder != null) {
+ responder.heartbeat(request);
+ }
+ } catch (Exception err) {
+ LOG.error("Error during responder execution", err);
+ }
+
+ return response;
+ }
+
+ @Override
+ public void nodeHeartbeat(Text hostname, int port) throws IOException {
+ updateHeartbeatInfo(hostname.toString(), port);
+ // No need to propagate to this to the responder
+ }
+
+ @Override
+ public void taskKilled(TezTaskAttemptID taskAttemptId) throws IOException {
+ String taskAttemptIdString = taskAttemptId.toString();
+ LOG.error("Task killed - " + taskAttemptIdString);
+ registeredTasks.remove(taskAttemptIdString);
+
+ try {
+ if (responder != null) {
+ responder.taskKilled(taskAttemptId);
+ }
+ } catch (Exception err) {
+ LOG.error("Error during responder execution", err);
+ }
+ }
+
+ @Override
+ public long getProtocolVersion(String protocol, long clientVersion) throws IOException {
+ return 0;
+ }
+
+ @Override
+ public ProtocolSignature getProtocolSignature(String protocol, long clientVersion,
+ int clientMethodsHash) throws IOException {
+ return ProtocolSignature.getProtocolSignature(this, protocol,
+ clientVersion, clientMethodsHash);
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/e0579097/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java
----------------------------------------------------------------------
diff --cc llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java
index 8db2f88,0000000..988002f
mode 100644,000000..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
@@@ -1,476 -1,0 +1,480 @@@
+/*
+ * 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.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.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.nio.ByteBuffer;
+
+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;
++import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexOrBinary;
+import org.apache.hadoop.hive.llap.ext.LlapTaskUmbilicalExternalClient;
+import org.apache.hadoop.hive.llap.ext.LlapTaskUmbilicalExternalClient.LlapTaskUmbilicalExternalResponder;
+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.llap.tez.Converters;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+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.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;
+import org.apache.tez.runtime.api.events.TaskAttemptFailedEvent;
+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;
+
+
+public class LlapBaseInputFormat<V extends WritableComparable> implements InputFormat<NullWritable, V> {
+
+ private static final Logger LOG = LoggerFactory.getLogger(LlapBaseInputFormat.class);
+
+ private static String driverName = "org.apache.hive.jdbc.HiveDriver";
+ private String url; // "jdbc:hive2://localhost:10000/default"
+ private String user; // "hive",
+ private String pwd; // ""
+ private String query;
+
+ public static final String URL_KEY = "llap.if.hs2.connection";
+ public static final String QUERY_KEY = "llap.if.query";
+ public static final String USER_KEY = "llap.if.user";
+ public static final String PWD_KEY = "llap.if.pwd";
+
+ public final String SPLIT_QUERY = "select get_splits(\"%s\",%d)";
+
+ private Connection con;
+ private Statement stmt;
+
+ public LlapBaseInputFormat(String url, String user, String pwd, String query) {
+ this.url = url;
+ this.user = user;
+ this.pwd = pwd;
+ this.query = query;
+ }
+
+ public LlapBaseInputFormat() {}
+
+
+ @Override
+ public RecordReader<NullWritable, V> getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
+
+ LlapInputSplit llapSplit = (LlapInputSplit) split;
+
+ // Set conf to use LLAP user rather than current user for LLAP Zk registry.
+ HiveConf.setVar(job, HiveConf.ConfVars.LLAP_ZK_REGISTRY_USER, llapSplit.getLlapUser());
+ SubmitWorkInfo submitWorkInfo = SubmitWorkInfo.fromBytes(llapSplit.getPlanBytes());
+
+ ServiceInstance serviceInstance = getServiceInstance(job, llapSplit);
+ String host = serviceInstance.getHost();
+ int llapSubmitPort = serviceInstance.getRpcPort();
+
+ LOG.info("Found service instance for host " + host + " with rpc port " + llapSubmitPort
+ + " and outputformat port " + serviceInstance.getOutputFormatPort());
+
+ LlapRecordReaderTaskUmbilicalExternalResponder umbilicalResponder =
+ new LlapRecordReaderTaskUmbilicalExternalResponder();
+ 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<TezEvent> tezEventList = Lists.newArrayList();
+ tezEventList.add(tezEvent);
+
+ llapClient.submitWork(submitWorkRequestProto, host, llapSubmitPort, tezEventList);
+
+ String id = HiveConf.getVar(job, HiveConf.ConfVars.HIVEQUERYID) + "_" + llapSplit.getSplitNum();
+
+ HiveConf conf = new HiveConf();
+ Socket socket = new Socket(host,
+ serviceInstance.getOutputFormatPort());
+
+ LOG.debug("Socket connected");
+
+ socket.getOutputStream().write(id.getBytes());
+ socket.getOutputStream().write(0);
+ socket.getOutputStream().flush();
+
+ LOG.info("Registered id: " + id);
+
+ LlapBaseRecordReader recordReader = new LlapBaseRecordReader(socket.getInputStream(), llapSplit.getSchema(), Text.class);
+ umbilicalResponder.setRecordReader(recordReader);
+ return recordReader;
+ }
+
+ @Override
+ public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
+ List<InputSplit> ins = new ArrayList<InputSplit>();
+
+ if (url == null) url = job.get(URL_KEY);
+ if (query == null) query = job.get(QUERY_KEY);
+ if (user == null) user = job.get(USER_KEY);
+ if (pwd == null) pwd = job.get(PWD_KEY);
+
+ if (url == null || query == null) {
+ throw new IllegalStateException();
+ }
+
+ try {
+ Class.forName(driverName);
+ } catch (ClassNotFoundException e) {
+ throw new IOException(e);
+ }
+
+ try {
+ con = DriverManager.getConnection(url,user,pwd);
+ stmt = con.createStatement();
+ String sql = String.format(SPLIT_QUERY, query, numSplits);
+ ResultSet res = stmt.executeQuery(sql);
+ while (res.next()) {
+ // deserialize split
+ DataInput in = new DataInputStream(res.getBinaryStream(1));
+ InputSplitWithLocationInfo is = new LlapInputSplit();
+ is.readFields(in);
+ ins.add(is);
+ }
+
+ res.close();
+ stmt.close();
+ } catch (Exception e) {
+ throw new IOException(e);
+ }
+ return ins.toArray(new InputSplit[ins.size()]);
+ }
+
+ public void close() {
+ try {
+ con.close();
+ } catch (Exception e) {
+ // ignore
+ }
+ }
+
+ private ServiceInstance getServiceInstance(JobConf job, LlapInputSplit llapSplit) throws IOException {
+ LlapRegistryService registryService = LlapRegistryService.getClient(job);
+ String host = llapSplit.getLocations()[0];
+
+ ServiceInstance serviceInstance = getServiceInstanceForHost(registryService, host);
+ if (serviceInstance == null) {
+ throw new IOException("No service instances found for " + host + " in registry");
+ }
+
+ return serviceInstance;
+ }
+
+ private ServiceInstance getServiceInstanceForHost(LlapRegistryService registryService, String host) throws IOException {
+ InetAddress address = InetAddress.getByName(host);
+ ServiceInstanceSet instanceSet = registryService.getInstances();
+ ServiceInstance serviceInstance = null;
+
+ // The name used in the service registry may not match the host name we're using.
+ // Try hostname/canonical hostname/host address
+
+ String name = address.getHostName();
+ LOG.info("Searching service instance by hostname " + name);
+ serviceInstance = selectServiceInstance(instanceSet.getByHost(name));
+ if (serviceInstance != null) {
+ return serviceInstance;
+ }
+
+ name = address.getCanonicalHostName();
+ LOG.info("Searching service instance by canonical hostname " + name);
+ serviceInstance = selectServiceInstance(instanceSet.getByHost(name));
+ if (serviceInstance != null) {
+ return serviceInstance;
+ }
+
+ name = address.getHostAddress();
+ LOG.info("Searching service instance by address " + name);
+ serviceInstance = selectServiceInstance(instanceSet.getByHost(name));
+ if (serviceInstance != null) {
+ return serviceInstance;
+ }
+
+ return serviceInstance;
+ }
+
+ private ServiceInstance selectServiceInstance(Set<ServiceInstance> serviceInstances) {
+ if (serviceInstances == null || serviceInstances.isEmpty()) {
+ return null;
+ }
+
+ // Get the first live service instance
+ for (ServiceInstance serviceInstance : serviceInstances) {
+ if (serviceInstance.isAlive()) {
+ return serviceInstance;
+ }
+ }
+
+ LOG.info("No live service instances were found");
+ return null;
+ }
+
+ private SubmitWorkRequestProto constructSubmitWorkRequestProto(SubmitWorkInfo submitWorkInfo,
+ int taskNum,
+ InetSocketAddress address,
+ Token<JobTokenIdentifier> token) throws
+ IOException {
+ TaskSpec taskSpec = submitWorkInfo.getTaskSpec();
+ ApplicationId appId = submitWorkInfo.getFakeAppId();
+
- SubmitWorkRequestProto.Builder builder = SubmitWorkRequestProto.newBuilder();
++ int attemptId = taskSpec.getTaskAttemptID().getId();
+ // This works, assuming the executor is running within YARN.
- LOG.info("Setting user in submitWorkRequest to: " +
- System.getenv(ApplicationConstants.Environment.USER.name()));
- builder.setUser(System.getenv(ApplicationConstants.Environment.USER.name()));
- builder.setApplicationIdString(appId.toString());
- builder.setAppAttemptNumber(0);
- builder.setTokenIdentifier(appId.toString());
++ 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
+
+ ContainerId containerId =
+ ContainerId.newInstance(ApplicationAttemptId.newInstance(appId, 0), taskNum);
- builder.setContainerIdString(containerId.toString());
+
- builder.setAmHost(address.getHostName());
- builder.setAmPort(address.getPort());
++
+ Credentials taskCredentials = new Credentials();
+ // Credentials can change across DAGs. Ideally construct only once per DAG.
+ // TODO Figure out where credentials will come from. Normally Hive sets up
+ // URLs on the tez dag, for which Tez acquires credentials.
+
+ // taskCredentials.addAll(getContext().getCredentials());
+
+ // Preconditions.checkState(currentQueryIdentifierProto.getDagIdentifier() ==
+ // taskSpec.getTaskAttemptID().getTaskID().getVertexID().getDAGId().getId());
+ // ByteBuffer credentialsBinary = credentialMap.get(currentQueryIdentifierProto);
+ // if (credentialsBinary == null) {
+ // credentialsBinary = serializeCredentials(getContext().getCredentials());
+ // credentialMap.putIfAbsent(currentQueryIdentifierProto, credentialsBinary.duplicate());
+ // } else {
+ // credentialsBinary = credentialsBinary.duplicate();
+ // }
+ // builder.setCredentialsBinary(ByteString.copyFrom(credentialsBinary));
+ Credentials credentials = new Credentials();
+ TokenCache.setSessionToken(token, credentials);
+ ByteBuffer credentialsBinary = serializeCredentials(credentials);
- builder.setCredentialsBinary(ByteString.copyFrom(credentialsBinary));
-
-
- builder.setFragmentSpec(Converters.convertTaskSpecToProto(taskSpec));
+
+ FragmentRuntimeInfo.Builder runtimeInfo = FragmentRuntimeInfo.newBuilder();
+ runtimeInfo.setCurrentAttemptStartTime(System.currentTimeMillis());
+ runtimeInfo.setWithinDagPriority(0);
+ runtimeInfo.setDagStartTime(submitWorkInfo.getCreationTime());
+ runtimeInfo.setFirstAttemptStartTime(submitWorkInfo.getCreationTime());
+ runtimeInfo.setNumSelfAndUpstreamTasks(taskSpec.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);
++ builder.setContainerIdString(containerId.toString());
++ builder.setAmHost(address.getHostName());
++ builder.setAmPort(address.getPort());
++ builder.setCredentialsBinary(ByteString.copyFrom(credentialsBinary));
+ builder.setFragmentRuntimeInfo(runtimeInfo.build());
++
+ return builder.build();
+ }
+
+ private ByteBuffer serializeCredentials(Credentials credentials) throws IOException {
+ Credentials containerCredentials = new Credentials();
+ containerCredentials.addAll(credentials);
+ DataOutputBuffer containerTokens_dob = new DataOutputBuffer();
+ containerCredentials.writeTokenStorageToStream(containerTokens_dob);
+ return ByteBuffer.wrap(containerTokens_dob.getData(), 0, containerTokens_dob.getLength());
+ }
+
+ private static class LlapRecordReaderTaskUmbilicalExternalResponder implements LlapTaskUmbilicalExternalResponder {
+ protected LlapBaseRecordReader recordReader = null;
+ protected LinkedBlockingQueue<ReaderEvent> queuedEvents = new LinkedBlockingQueue<ReaderEvent>();
+
+ public LlapRecordReaderTaskUmbilicalExternalResponder() {
+ }
+
+ @Override
+ public void submissionFailed(String fragmentId, Throwable throwable) {
+ try {
+ sendOrQueueEvent(ReaderEvent.errorEvent(
+ "Received submission failed event for fragment ID " + fragmentId));
+ } catch (Exception err) {
+ LOG.error("Error during heartbeat responder:", err);
+ }
+ }
+
+ @Override
+ public void heartbeat(TezHeartbeatRequest request) {
+ TezTaskAttemptID taskAttemptId = request.getCurrentTaskAttemptID();
+ List<TezEvent> inEvents = request.getEvents();
+ for (TezEvent tezEvent : ListUtils.emptyIfNull(inEvents)) {
+ EventType eventType = tezEvent.getEventType();
+ try {
+ switch (eventType) {
+ case TASK_ATTEMPT_COMPLETED_EVENT:
+ sendOrQueueEvent(ReaderEvent.doneEvent());
+ break;
+ case TASK_ATTEMPT_FAILED_EVENT:
+ TaskAttemptFailedEvent taskFailedEvent = (TaskAttemptFailedEvent) tezEvent.getEvent();
+ sendOrQueueEvent(ReaderEvent.errorEvent(taskFailedEvent.getDiagnostics()));
+ break;
+ case TASK_STATUS_UPDATE_EVENT:
+ // If we want to handle counters
+ break;
+ default:
+ LOG.warn("Unhandled event type " + eventType);
+ break;
+ }
+ } catch (Exception err) {
+ LOG.error("Error during heartbeat responder:", err);
+ }
+ }
+ }
+
+ @Override
+ public void taskKilled(TezTaskAttemptID taskAttemptId) {
+ try {
+ sendOrQueueEvent(ReaderEvent.errorEvent(
+ "Received task killed event for task ID " + taskAttemptId));
+ } catch (Exception err) {
+ LOG.error("Error during heartbeat responder:", err);
+ }
+ }
+
+ @Override
+ public void heartbeatTimeout(String taskAttemptId) {
+ try {
+ sendOrQueueEvent(ReaderEvent.errorEvent(
+ "Timed out waiting for heartbeat for task ID " + taskAttemptId));
+ } catch (Exception err) {
+ LOG.error("Error during heartbeat responder:", err);
+ }
+ }
+
+ public synchronized LlapBaseRecordReader getRecordReader() {
+ return recordReader;
+ }
+
+ public synchronized void setRecordReader(LlapBaseRecordReader recordReader) {
+ this.recordReader = recordReader;
+
+ if (recordReader == null) {
+ return;
+ }
+
+ // If any events were queued by the responder, give them to the record reader now.
+ while (!queuedEvents.isEmpty()) {
+ ReaderEvent readerEvent = queuedEvents.poll();
+ LOG.debug("Sending queued event to record reader: " + readerEvent.getEventType());
+ recordReader.handleEvent(readerEvent);
+ }
+ }
+
+ /**
+ * Send the ReaderEvents to the record reader, if it is registered to this responder.
+ * If there is no registered record reader, add them to a list of pending reader events
+ * since we don't want to drop these events.
+ * @param readerEvent
+ */
+ protected synchronized void sendOrQueueEvent(ReaderEvent readerEvent) {
+ LlapBaseRecordReader recordReader = getRecordReader();
+ if (recordReader != null) {
+ recordReader.handleEvent(readerEvent);
+ } else {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("No registered record reader, queueing event " + readerEvent.getEventType()
+ + " with message " + readerEvent.getMessage());
+ }
+
+ try {
+ queuedEvents.put(readerEvent);
+ } catch (Exception err) {
+ throw new RuntimeException("Unexpected exception while queueing reader event", err);
+ }
+ }
+ }
+
+ /**
+ * Clear the list of queued reader events if we are not interested in sending any pending events to any registering record reader.
+ */
+ public void clearQueuedEvents() {
+ queuedEvents.clear();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/e0579097/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
----------------------------------------------------------------------
diff --cc llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
index d8367ce,2bfe3ed..2524dc2
--- 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
@@@ -263,13 -267,12 +267,12 @@@ public class ContainerRunnerImpl extend
new QueryIdentifier(request.getQueryIdentifier().getAppIdentifier(),
request.getQueryIdentifier().getDagIdentifier());
LOG.info("Processing queryComplete notification for {}", queryIdentifier);
- List<QueryFragmentInfo> knownFragments =
- queryTracker
- .queryComplete(queryIdentifier, request.getDeleteDelay());
- LOG.info("Pending fragment count for completed query {} = {}", queryIdentifier,
+ List<QueryFragmentInfo> knownFragments = queryTracker.queryComplete(
+ queryIdentifier, request.getDeleteDelay(), false);
+ LOG.info("DBG: Pending fragment count for completed query {} = {}", queryIdentifier,
knownFragments.size());
for (QueryFragmentInfo fragmentInfo : knownFragments) {
- LOG.info("DBG: Issuing killFragment for completed query {} {}", queryIdentifier,
+ LOG.info("Issuing killFragment for completed query {} {}", queryIdentifier,
fragmentInfo.getFragmentIdentifierString());
executorService.killFragment(fragmentInfo.getFragmentIdentifierString());
}
http://git-wip-us.apache.org/repos/asf/hive/blob/e0579097/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/e0579097/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
----------------------------------------------------------------------
diff --cc llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
index 4a33373,3093de7..8594ee1
--- 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
@@@ -134,15 -135,20 +135,18 @@@ public class TaskRunnerCallable extend
this.memoryAvailable = memoryAvailable;
this.confParams = confParams;
this.jobToken = TokenCache.getSessionToken(credentials);
- this.taskSpec = Converters.getTaskSpecfromProto(request.getFragmentSpec());
+ // TODO: support binary spec here or above
+ this.vertex = request.getWorkSpec().getVertex();
+ this.taskSpec = Converters.getTaskSpecfromProto(
+ vertex, request.getFragmentNumber(), request.getAttemptNumber(), attemptId);
this.amReporter = amReporter;
// Register with the AMReporter when the callable is setup. Unregister once it starts running.
- if (jobToken != null) {
this.amReporter.registerTask(request.getAmHost(), request.getAmPort(),
- request.getUser(), jobToken, fragmentInfo.getQueryInfo().getQueryIdentifier());
+ vertex.getUser(), jobToken, fragmentInfo.getQueryInfo().getQueryIdentifier());
- }
this.metrics = metrics;
- this.requestId = request.getFragmentSpec().getFragmentIdentifierString();
+ this.requestId = taskSpec.getTaskAttemptID().toString();
// TODO Change this to the queryId/Name when that's available.
- this.queryId = request.getFragmentSpec().getDagName();
+ this.queryId = vertex.getDagName();
this.killedTaskHandler = killedTaskHandler;
this.fragmentCompletionHanler = fragmentCompleteHandler;
this.tezHadoopShim = tezHadoopShim;
http://git-wip-us.apache.org/repos/asf/hive/blob/e0579097/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
----------------------------------------------------------------------
[09/10] hive git commit: HIVE-13674: usingTezAm field not required in
LLAP SubmitWorkRequestProto
Posted by jd...@apache.org.
HIVE-13674: usingTezAm field not required in LLAP SubmitWorkRequestProto
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/4847f652
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/4847f652
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/4847f652
Branch: refs/heads/llap
Commit: 4847f652804f476bbc969716fe7643d8b20eba8c
Parents: bc75d72
Author: Jason Dere <jd...@hortonworks.com>
Authored: Tue May 3 18:38:07 2016 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Tue May 3 18:38:07 2016 -0700
----------------------------------------------------------------------
.../ext/LlapTaskUmbilicalExternalClient.java | 4 +-
.../daemon/rpc/LlapDaemonProtocolProtos.java | 230 ++++++-------------
.../src/protobuf/LlapDaemonProtocol.proto | 8 -
.../hadoop/hive/llap/LlapBaseInputFormat.java | 1 -
.../llap/daemon/impl/TaskRunnerCallable.java | 3 -
5 files changed, 71 insertions(+), 175 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/4847f652/llap-client/src/java/org/apache/hadoop/hive/llap/ext/LlapTaskUmbilicalExternalClient.java
----------------------------------------------------------------------
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 8598bc8..fe2fd7c 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
@@ -123,12 +123,10 @@ public class LlapTaskUmbilicalExternalClient extends AbstractService {
/**
- * Submit the work for actual execution. This should always have the usingTezAm flag disabled
+ * Submit the work for actual execution.
* @param submitWorkRequestProto
*/
public void submitWork(final SubmitWorkRequestProto submitWorkRequestProto, String llapHost, int llapPort, List<TezEvent> tezEvents) {
- Preconditions.checkArgument(submitWorkRequestProto.getUsingTezAm() == false);
-
// Register the pending events to be sent for this spec.
String fragmentId = submitWorkRequestProto.getFragmentSpec().getFragmentIdentifierString();
PendingEventData pendingEventData = new PendingEventData(
http://git-wip-us.apache.org/repos/asf/hive/blob/4847f652/llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/daemon/rpc/LlapDaemonProtocolProtos.java
----------------------------------------------------------------------
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 653e7e0..6a20031 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
@@ -1,5 +1,5 @@
// Generated by the protocol buffer compiler. DO NOT EDIT!
-// source: LlapDaemonProtocol.proto
+// source: llap-common/src/protobuf/LlapDaemonProtocol.proto
package org.apache.hadoop.hive.llap.daemon.rpc;
@@ -7334,16 +7334,6 @@ public final class LlapDaemonProtocolProtos {
* <code>optional .FragmentRuntimeInfo fragment_runtime_info = 10;</code>
*/
org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentRuntimeInfoOrBuilder getFragmentRuntimeInfoOrBuilder();
-
- // optional bool usingTezAm = 11 [default = true];
- /**
- * <code>optional bool usingTezAm = 11 [default = true];</code>
- */
- boolean hasUsingTezAm();
- /**
- * <code>optional bool usingTezAm = 11 [default = true];</code>
- */
- boolean getUsingTezAm();
}
/**
* Protobuf type {@code SubmitWorkRequestProto}
@@ -7462,11 +7452,6 @@ public final class LlapDaemonProtocolProtos {
bitField0_ |= 0x00000200;
break;
}
- case 88: {
- bitField0_ |= 0x00000400;
- usingTezAm_ = input.readBool();
- break;
- }
}
}
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -7814,22 +7799,6 @@ public final class LlapDaemonProtocolProtos {
return fragmentRuntimeInfo_;
}
- // optional bool usingTezAm = 11 [default = true];
- public static final int USINGTEZAM_FIELD_NUMBER = 11;
- private boolean usingTezAm_;
- /**
- * <code>optional bool usingTezAm = 11 [default = true];</code>
- */
- public boolean hasUsingTezAm() {
- return ((bitField0_ & 0x00000400) == 0x00000400);
- }
- /**
- * <code>optional bool usingTezAm = 11 [default = true];</code>
- */
- public boolean getUsingTezAm() {
- return usingTezAm_;
- }
-
private void initFields() {
containerIdString_ = "";
amHost_ = "";
@@ -7841,7 +7810,6 @@ public final class LlapDaemonProtocolProtos {
appAttemptNumber_ = 0;
fragmentSpec_ = org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto.getDefaultInstance();
fragmentRuntimeInfo_ = org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentRuntimeInfo.getDefaultInstance();
- usingTezAm_ = true;
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
@@ -7885,9 +7853,6 @@ public final class LlapDaemonProtocolProtos {
if (((bitField0_ & 0x00000200) == 0x00000200)) {
output.writeMessage(10, fragmentRuntimeInfo_);
}
- if (((bitField0_ & 0x00000400) == 0x00000400)) {
- output.writeBool(11, usingTezAm_);
- }
getUnknownFields().writeTo(output);
}
@@ -7937,10 +7902,6 @@ public final class LlapDaemonProtocolProtos {
size += com.google.protobuf.CodedOutputStream
.computeMessageSize(10, fragmentRuntimeInfo_);
}
- if (((bitField0_ & 0x00000400) == 0x00000400)) {
- size += com.google.protobuf.CodedOutputStream
- .computeBoolSize(11, usingTezAm_);
- }
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
return size;
@@ -8014,11 +7975,6 @@ public final class LlapDaemonProtocolProtos {
result = result && getFragmentRuntimeInfo()
.equals(other.getFragmentRuntimeInfo());
}
- result = result && (hasUsingTezAm() == other.hasUsingTezAm());
- if (hasUsingTezAm()) {
- result = result && (getUsingTezAm()
- == other.getUsingTezAm());
- }
result = result &&
getUnknownFields().equals(other.getUnknownFields());
return result;
@@ -8072,10 +8028,6 @@ public final class LlapDaemonProtocolProtos {
hash = (37 * hash) + FRAGMENT_RUNTIME_INFO_FIELD_NUMBER;
hash = (53 * hash) + getFragmentRuntimeInfo().hashCode();
}
- if (hasUsingTezAm()) {
- hash = (37 * hash) + USINGTEZAM_FIELD_NUMBER;
- hash = (53 * hash) + hashBoolean(getUsingTezAm());
- }
hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash;
return hash;
@@ -8215,8 +8167,6 @@ public final class LlapDaemonProtocolProtos {
fragmentRuntimeInfoBuilder_.clear();
}
bitField0_ = (bitField0_ & ~0x00000200);
- usingTezAm_ = true;
- bitField0_ = (bitField0_ & ~0x00000400);
return this;
}
@@ -8293,10 +8243,6 @@ public final class LlapDaemonProtocolProtos {
} else {
result.fragmentRuntimeInfo_ = fragmentRuntimeInfoBuilder_.build();
}
- if (((from_bitField0_ & 0x00000400) == 0x00000400)) {
- to_bitField0_ |= 0x00000400;
- }
- result.usingTezAm_ = usingTezAm_;
result.bitField0_ = to_bitField0_;
onBuilt();
return result;
@@ -8353,9 +8299,6 @@ public final class LlapDaemonProtocolProtos {
if (other.hasFragmentRuntimeInfo()) {
mergeFragmentRuntimeInfo(other.getFragmentRuntimeInfo());
}
- if (other.hasUsingTezAm()) {
- setUsingTezAm(other.getUsingTezAm());
- }
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
@@ -9089,39 +9032,6 @@ public final class LlapDaemonProtocolProtos {
return fragmentRuntimeInfoBuilder_;
}
- // optional bool usingTezAm = 11 [default = true];
- private boolean usingTezAm_ = true;
- /**
- * <code>optional bool usingTezAm = 11 [default = true];</code>
- */
- public boolean hasUsingTezAm() {
- return ((bitField0_ & 0x00000400) == 0x00000400);
- }
- /**
- * <code>optional bool usingTezAm = 11 [default = true];</code>
- */
- public boolean getUsingTezAm() {
- return usingTezAm_;
- }
- /**
- * <code>optional bool usingTezAm = 11 [default = true];</code>
- */
- public Builder setUsingTezAm(boolean value) {
- bitField0_ |= 0x00000400;
- usingTezAm_ = value;
- onChanged();
- return this;
- }
- /**
- * <code>optional bool usingTezAm = 11 [default = true];</code>
- */
- public Builder clearUsingTezAm() {
- bitField0_ = (bitField0_ & ~0x00000400);
- usingTezAm_ = true;
- onChanged();
- return this;
- }
-
// @@protoc_insertion_point(builder_scope:SubmitWorkRequestProto)
}
@@ -14455,74 +14365,74 @@ public final class LlapDaemonProtocolProtos {
descriptor;
static {
java.lang.String[] descriptorData = {
- "\n\030LlapDaemonProtocol.proto\"9\n\020UserPayloa" +
- "dProto\022\024\n\014user_payload\030\001 \001(\014\022\017\n\007version\030" +
- "\002 \001(\005\"j\n\025EntityDescriptorProto\022\022\n\nclass_" +
- "name\030\001 \001(\t\022\'\n\014user_payload\030\002 \001(\0132\021.UserP" +
- "ayloadProto\022\024\n\014history_text\030\003 \001(\014\"x\n\013IOS" +
- "pecProto\022\035\n\025connected_vertex_name\030\001 \001(\t\022" +
- "-\n\rio_descriptor\030\002 \001(\0132\026.EntityDescripto" +
- "rProto\022\033\n\023physical_edge_count\030\003 \001(\005\"z\n\023G" +
- "roupInputSpecProto\022\022\n\ngroup_name\030\001 \001(\t\022\026" +
- "\n\016group_vertices\030\002 \003(\t\0227\n\027merged_input_d",
- "escriptor\030\003 \001(\0132\026.EntityDescriptorProto\"" +
- "\353\002\n\021FragmentSpecProto\022\"\n\032fragment_identi" +
- "fier_string\030\001 \001(\t\022\020\n\010dag_name\030\002 \001(\t\022\016\n\006d" +
- "ag_id\030\013 \001(\005\022\023\n\013vertex_name\030\003 \001(\t\0224\n\024proc" +
- "essor_descriptor\030\004 \001(\0132\026.EntityDescripto" +
- "rProto\022!\n\013input_specs\030\005 \003(\0132\014.IOSpecProt" +
- "o\022\"\n\014output_specs\030\006 \003(\0132\014.IOSpecProto\0221\n" +
- "\023grouped_input_specs\030\007 \003(\0132\024.GroupInputS" +
- "pecProto\022\032\n\022vertex_parallelism\030\010 \001(\005\022\027\n\017" +
- "fragment_number\030\t \001(\005\022\026\n\016attempt_number\030",
- "\n \001(\005\"\344\001\n\023FragmentRuntimeInfo\022#\n\033num_sel" +
- "f_and_upstream_tasks\030\001 \001(\005\022-\n%num_self_a" +
- "nd_upstream_completed_tasks\030\002 \001(\005\022\033\n\023wit" +
- "hin_dag_priority\030\003 \001(\005\022\026\n\016dag_start_time" +
- "\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\"\320\002\n\026SubmitW" +
- "orkRequestProto\022\033\n\023container_id_string\030\001" +
- " \001(\t\022\017\n\007am_host\030\002 \001(\t\022\017\n\007am_port\030\003 \001(\005\022\030",
- "\n\020token_identifier\030\004 \001(\t\022\032\n\022credentials_" +
- "binary\030\005 \001(\014\022\014\n\004user\030\006 \001(\t\022\035\n\025applicatio" +
- "n_id_string\030\007 \001(\t\022\032\n\022app_attempt_number\030" +
- "\010 \001(\005\022)\n\rfragment_spec\030\t \001(\0132\022.FragmentS" +
- "pecProto\0223\n\025fragment_runtime_info\030\n \001(\0132" +
- "\024.FragmentRuntimeInfo\022\030\n\nusingTezAm\030\013 \001(" +
- "\010:\004true\"J\n\027SubmitWorkResponseProto\022/\n\020su" +
- "bmission_state\030\001 \001(\0162\025.SubmissionStatePr" +
- "oto\"\205\001\n\036SourceStateUpdatedRequestProto\022/" +
- "\n\020query_identifier\030\001 \001(\0132\025.QueryIdentifi",
- "erProto\022\020\n\010src_name\030\002 \001(\t\022 \n\005state\030\003 \001(\016" +
- "2\021.SourceStateProto\"!\n\037SourceStateUpdate" +
- "dResponseProto\"w\n\031QueryCompleteRequestPr" +
- "oto\022\020\n\010query_id\030\001 \001(\t\022/\n\020query_identifie" +
- "r\030\002 \001(\0132\025.QueryIdentifierProto\022\027\n\014delete" +
- "_delay\030\004 \001(\003:\0010\"\034\n\032QueryCompleteResponse" +
- "Proto\"t\n\035TerminateFragmentRequestProto\022/" +
- "\n\020query_identifier\030\001 \001(\0132\025.QueryIdentifi" +
- "erProto\022\"\n\032fragment_identifier_string\030\002 " +
- "\001(\t\" \n\036TerminateFragmentResponseProto\"\026\n",
- "\024GetTokenRequestProto\"&\n\025GetTokenRespons" +
- "eProto\022\r\n\005token\030\001 \001(\014*2\n\020SourceStateProt" +
- "o\022\017\n\013S_SUCCEEDED\020\001\022\r\n\tS_RUNNING\020\002*E\n\024Sub" +
- "missionStateProto\022\014\n\010ACCEPTED\020\001\022\014\n\010REJEC" +
- "TED\020\002\022\021\n\rEVICTED_OTHER\020\0032\316\002\n\022LlapDaemonP" +
- "rotocol\022?\n\nsubmitWork\022\027.SubmitWorkReques" +
- "tProto\032\030.SubmitWorkResponseProto\022W\n\022sour" +
- "ceStateUpdated\022\037.SourceStateUpdatedReque" +
- "stProto\032 .SourceStateUpdatedResponseProt" +
- "o\022H\n\rqueryComplete\022\032.QueryCompleteReques",
- "tProto\032\033.QueryCompleteResponseProto\022T\n\021t" +
- "erminateFragment\022\036.TerminateFragmentRequ" +
- "estProto\032\037.TerminateFragmentResponseProt" +
- "o2]\n\026LlapManagementProtocol\022C\n\022getDelega" +
- "tionToken\022\025.GetTokenRequestProto\032\026.GetTo" +
- "kenResponseProtoBH\n&org.apache.hadoop.hi" +
- "ve.llap.daemon.rpcB\030LlapDaemonProtocolPr" +
- "otos\210\001\001\240\001\001"
+ "\n1llap-common/src/protobuf/LlapDaemonPro" +
+ "tocol.proto\"9\n\020UserPayloadProto\022\024\n\014user_" +
+ "payload\030\001 \001(\014\022\017\n\007version\030\002 \001(\005\"j\n\025Entity" +
+ "DescriptorProto\022\022\n\nclass_name\030\001 \001(\t\022\'\n\014u" +
+ "ser_payload\030\002 \001(\0132\021.UserPayloadProto\022\024\n\014" +
+ "history_text\030\003 \001(\014\"x\n\013IOSpecProto\022\035\n\025con" +
+ "nected_vertex_name\030\001 \001(\t\022-\n\rio_descripto" +
+ "r\030\002 \001(\0132\026.EntityDescriptorProto\022\033\n\023physi" +
+ "cal_edge_count\030\003 \001(\005\"z\n\023GroupInputSpecPr" +
+ "oto\022\022\n\ngroup_name\030\001 \001(\t\022\026\n\016group_vertice",
+ "s\030\002 \003(\t\0227\n\027merged_input_descriptor\030\003 \001(\013" +
+ "2\026.EntityDescriptorProto\"\353\002\n\021FragmentSpe" +
+ "cProto\022\"\n\032fragment_identifier_string\030\001 \001" +
+ "(\t\022\020\n\010dag_name\030\002 \001(\t\022\016\n\006dag_id\030\013 \001(\005\022\023\n\013" +
+ "vertex_name\030\003 \001(\t\0224\n\024processor_descripto" +
+ "r\030\004 \001(\0132\026.EntityDescriptorProto\022!\n\013input" +
+ "_specs\030\005 \003(\0132\014.IOSpecProto\022\"\n\014output_spe" +
+ "cs\030\006 \003(\0132\014.IOSpecProto\0221\n\023grouped_input_" +
+ "specs\030\007 \003(\0132\024.GroupInputSpecProto\022\032\n\022ver" +
+ "tex_parallelism\030\010 \001(\005\022\027\n\017fragment_number",
+ "\030\t \001(\005\022\026\n\016attempt_number\030\n \001(\005\"\344\001\n\023Fragm" +
+ "entRuntimeInfo\022#\n\033num_self_and_upstream_" +
+ "tasks\030\001 \001(\005\022-\n%num_self_and_upstream_com" +
+ "pleted_tasks\030\002 \001(\005\022\033\n\023within_dag_priorit" +
+ "y\030\003 \001(\005\022\026\n\016dag_start_time\030\004 \001(\003\022 \n\030first" +
+ "_attempt_start_time\030\005 \001(\003\022\"\n\032current_att" +
+ "empt_start_time\030\006 \001(\003\"F\n\024QueryIdentifier" +
+ "Proto\022\026\n\016app_identifier\030\001 \001(\t\022\026\n\016dag_ide" +
+ "ntifier\030\002 \001(\005\"\266\002\n\026SubmitWorkRequestProto" +
+ "\022\033\n\023container_id_string\030\001 \001(\t\022\017\n\007am_host",
+ "\030\002 \001(\t\022\017\n\007am_port\030\003 \001(\005\022\030\n\020token_identif" +
+ "ier\030\004 \001(\t\022\032\n\022credentials_binary\030\005 \001(\014\022\014\n" +
+ "\004user\030\006 \001(\t\022\035\n\025application_id_string\030\007 \001" +
+ "(\t\022\032\n\022app_attempt_number\030\010 \001(\005\022)\n\rfragme" +
+ "nt_spec\030\t \001(\0132\022.FragmentSpecProto\0223\n\025fra" +
+ "gment_runtime_info\030\n \001(\0132\024.FragmentRunti" +
+ "meInfo\"J\n\027SubmitWorkResponseProto\022/\n\020sub" +
+ "mission_state\030\001 \001(\0162\025.SubmissionStatePro" +
+ "to\"\205\001\n\036SourceStateUpdatedRequestProto\022/\n" +
+ "\020query_identifier\030\001 \001(\0132\025.QueryIdentifie",
+ "rProto\022\020\n\010src_name\030\002 \001(\t\022 \n\005state\030\003 \001(\0162" +
+ "\021.SourceStateProto\"!\n\037SourceStateUpdated" +
+ "ResponseProto\"w\n\031QueryCompleteRequestPro" +
+ "to\022\020\n\010query_id\030\001 \001(\t\022/\n\020query_identifier" +
+ "\030\002 \001(\0132\025.QueryIdentifierProto\022\027\n\014delete_" +
+ "delay\030\004 \001(\003:\0010\"\034\n\032QueryCompleteResponseP" +
+ "roto\"t\n\035TerminateFragmentRequestProto\022/\n" +
+ "\020query_identifier\030\001 \001(\0132\025.QueryIdentifie" +
+ "rProto\022\"\n\032fragment_identifier_string\030\002 \001" +
+ "(\t\" \n\036TerminateFragmentResponseProto\"\026\n\024",
+ "GetTokenRequestProto\"&\n\025GetTokenResponse" +
+ "Proto\022\r\n\005token\030\001 \001(\014*2\n\020SourceStateProto" +
+ "\022\017\n\013S_SUCCEEDED\020\001\022\r\n\tS_RUNNING\020\002*E\n\024Subm" +
+ "issionStateProto\022\014\n\010ACCEPTED\020\001\022\014\n\010REJECT" +
+ "ED\020\002\022\021\n\rEVICTED_OTHER\020\0032\316\002\n\022LlapDaemonPr" +
+ "otocol\022?\n\nsubmitWork\022\027.SubmitWorkRequest" +
+ "Proto\032\030.SubmitWorkResponseProto\022W\n\022sourc" +
+ "eStateUpdated\022\037.SourceStateUpdatedReques" +
+ "tProto\032 .SourceStateUpdatedResponseProto" +
+ "\022H\n\rqueryComplete\022\032.QueryCompleteRequest",
+ "Proto\032\033.QueryCompleteResponseProto\022T\n\021te" +
+ "rminateFragment\022\036.TerminateFragmentReque" +
+ "stProto\032\037.TerminateFragmentResponseProto" +
+ "2]\n\026LlapManagementProtocol\022C\n\022getDelegat" +
+ "ionToken\022\025.GetTokenRequestProto\032\026.GetTok" +
+ "enResponseProtoBH\n&org.apache.hadoop.hiv" +
+ "e.llap.daemon.rpcB\030LlapDaemonProtocolPro" +
+ "tos\210\001\001\240\001\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -14576,7 +14486,7 @@ public final class LlapDaemonProtocolProtos {
internal_static_SubmitWorkRequestProto_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_SubmitWorkRequestProto_descriptor,
- new java.lang.String[] { "ContainerIdString", "AmHost", "AmPort", "TokenIdentifier", "CredentialsBinary", "User", "ApplicationIdString", "AppAttemptNumber", "FragmentSpec", "FragmentRuntimeInfo", "UsingTezAm", });
+ new java.lang.String[] { "ContainerIdString", "AmHost", "AmPort", "TokenIdentifier", "CredentialsBinary", "User", "ApplicationIdString", "AppAttemptNumber", "FragmentSpec", "FragmentRuntimeInfo", });
internal_static_SubmitWorkResponseProto_descriptor =
getDescriptor().getMessageTypes().get(8);
internal_static_SubmitWorkResponseProto_fieldAccessorTable = new
http://git-wip-us.apache.org/repos/asf/hive/blob/4847f652/llap-common/src/protobuf/LlapDaemonProtocol.proto
----------------------------------------------------------------------
diff --git a/llap-common/src/protobuf/LlapDaemonProtocol.proto b/llap-common/src/protobuf/LlapDaemonProtocol.proto
index e964c5f..944c96c 100644
--- a/llap-common/src/protobuf/LlapDaemonProtocol.proto
+++ b/llap-common/src/protobuf/LlapDaemonProtocol.proto
@@ -91,7 +91,6 @@ message SubmitWorkRequestProto {
optional int32 app_attempt_number = 8;
optional FragmentSpecProto fragment_spec = 9;
optional FragmentRuntimeInfo fragment_runtime_info = 10;
- optional bool usingTezAm = 11 [default = true];
}
enum SubmissionStateProto {
@@ -137,18 +136,11 @@ message GetTokenResponseProto {
optional bytes token = 1;
}
-message SendEventsRequestProto {
-}
-
-message SendEventsResponseProto {
-}
-
service LlapDaemonProtocol {
rpc submitWork(SubmitWorkRequestProto) returns (SubmitWorkResponseProto);
rpc sourceStateUpdated(SourceStateUpdatedRequestProto) returns (SourceStateUpdatedResponseProto);
rpc queryComplete(QueryCompleteRequestProto) returns (QueryCompleteResponseProto);
rpc terminateFragment(TerminateFragmentRequestProto) returns (TerminateFragmentResponseProto);
- rpc sendEvents(SendEventsRequestProto) return (SendEventsResponseProto);
}
service LlapManagementProtocol {
http://git-wip-us.apache.org/repos/asf/hive/blob/4847f652/llap-ext-client/src/java/org/apache/hadoop/hive/llap/LlapBaseInputFormat.java
----------------------------------------------------------------------
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 10d14c0..8db2f88 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
@@ -346,7 +346,6 @@ public class LlapBaseInputFormat<V extends WritableComparable> implements InputF
runtimeInfo.setNumSelfAndUpstreamCompletedTasks(0);
- builder.setUsingTezAm(false);
builder.setFragmentRuntimeInfo(runtimeInfo.build());
return builder.build();
}
http://git-wip-us.apache.org/repos/asf/hive/blob/4847f652/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
----------------------------------------------------------------------
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 efd6f0a..4a33373 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
@@ -108,7 +108,6 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
private final String queryId;
private final HadoopShim tezHadoopShim;
private boolean shouldRunTask = true;
- private final boolean withTezAm;
final Stopwatch runtimeWatch = new Stopwatch();
final Stopwatch killtimerWatch = new Stopwatch();
private final AtomicBoolean isStarted = new AtomicBoolean(false);
@@ -137,8 +136,6 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
this.jobToken = TokenCache.getSessionToken(credentials);
this.taskSpec = Converters.getTaskSpecfromProto(request.getFragmentSpec());
this.amReporter = amReporter;
- this.withTezAm = request.getUsingTezAm();
- LOG.warn("ZZZ: DBG: usingTezAm=" + withTezAm);
// Register with the AMReporter when the callable is setup. Unregister once it starts running.
this.amReporter.registerTask(request.getAmHost(), request.getAmPort(),
request.getUser(), jobToken, fragmentInfo.getQueryInfo().getQueryIdentifier());
[06/10] hive git commit: HIVE-13442 : LLAP: refactor submit API to be
amenable to signing (Sergey Shelukhin, reviewed by Siddharth Seth)
Posted by jd...@apache.org.
HIVE-13442 : LLAP: refactor submit API to be amenable to signing (Sergey Shelukhin, reviewed by Siddharth Seth)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/0b5c27fd
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/0b5c27fd
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/0b5c27fd
Branch: refs/heads/llap
Commit: 0b5c27fdd4fbf8861d4eefc207c2da3a6ceac23d
Parents: b70efa4
Author: Sergey Shelukhin <se...@apache.org>
Authored: Tue May 3 15:02:24 2016 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Tue May 3 15:23:39 2016 -0700
----------------------------------------------------------------------
.../daemon/rpc/LlapDaemonProtocolProtos.java | 6836 +++++++++++-------
.../apache/hadoop/hive/llap/tez/Converters.java | 84 +-
.../src/protobuf/LlapDaemonProtocol.proto | 69 +-
.../hadoop/hive/llap/tez/TestConverters.java | 51 +-
.../llap/daemon/impl/ContainerRunnerImpl.java | 88 +-
.../llap/daemon/impl/QueryFragmentInfo.java | 23 +-
.../hadoop/hive/llap/daemon/impl/QueryInfo.java | 9 +-
.../hive/llap/daemon/impl/QueryTracker.java | 18 +-
.../llap/daemon/impl/TaskExecutorService.java | 8 +-
.../llap/daemon/impl/TaskRunnerCallable.java | 77 +-
.../daemon/impl/TaskExecutorTestHelpers.java | 42 +-
.../TestFirstInFirstOutComparator.java | 27 +-
.../llap/tezplugins/LlapTaskCommunicator.java | 31 +-
13 files changed, 4504 insertions(+), 2859 deletions(-)
----------------------------------------------------------------------
[03/10] hive git commit: HIVE-13213 make DbLockManger work for
non-acid resources (Eugene Koifman, reviewed by Alan Gates)
Posted by jd...@apache.org.
HIVE-13213 make DbLockManger work for non-acid resources (Eugene Koifman, reviewed by Alan Gates)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/b70efa44
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/b70efa44
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/b70efa44
Branch: refs/heads/llap
Commit: b70efa447d9ae5883315d88e84ad1262d371213d
Parents: 47bf055
Author: Eugene Koifman <ek...@hortonworks.com>
Authored: Tue May 3 13:38:42 2016 -0700
Committer: Eugene Koifman <ek...@hortonworks.com>
Committed: Tue May 3 13:53:02 2016 -0700
----------------------------------------------------------------------
.../hadoop/hive/metastore/txn/TxnHandler.java | 5 ++
.../hadoop/hive/ql/lockmgr/DbTxnManager.java | 12 +++
.../apache/hadoop/hive/ql/TestTxnCommands2.java | 22 ++++++
.../hive/ql/lockmgr/TestDbTxnManager2.java | 81 ++++++++++++++++++++
4 files changed, 120 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/b70efa44/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
index c32b0b0..c0fa97a 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
@@ -87,6 +87,11 @@ import java.util.regex.Pattern;
* If we ever decide to run remote Derby server, according to
* https://db.apache.org/derby/docs/10.0/manuals/develop/develop78.html all transactions will be
* seriazlied, so that would also work though has not been tested.
+ *
+ * General design note:
+ * It's imperative that any operation on a txn (e.g. commit), ensure (atomically) that this txn is
+ * still valid and active. In the code this is usually achieved at the same time the txn record
+ * is locked for some operation.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
http://git-wip-us.apache.org/repos/asf/hive/blob/b70efa44/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
index e8ebe55..3aec8eb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.lockmgr;
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
import org.apache.hive.common.util.ShutdownHookManager;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -213,6 +214,17 @@ public class DbTxnManager extends HiveTxnManagerImpl {
break;
case INSERT:
+ t = output.getTable();
+ if(t == null) {
+ throw new IllegalStateException("No table info for " + output);
+ }
+ if(AcidUtils.isAcidTable(t)) {
+ compBuilder.setShared();
+ }
+ else {
+ compBuilder.setExclusive();
+ }
+ break;
case DDL_SHARED:
compBuilder.setShared();
break;
http://git-wip-us.apache.org/repos/asf/hive/blob/b70efa44/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
index 04c1d17..1030987 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
@@ -433,6 +433,28 @@ public class TestTxnCommands2 {
}
/**
+ * Test update that hits multiple partitions (i.e. requries dynamic partition insert to process)
+ * @throws Exception
+ */
+ @Test
+ public void updateDeletePartitioned() throws Exception {
+ int[][] tableData = {{1,2},{3,4},{5,6}};
+ runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p=1) (a,b) " + makeValuesClause(tableData));
+ runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p=2) (a,b) " + makeValuesClause(tableData));
+ TxnStore txnHandler = TxnUtils.getTxnStore(hiveConf);
+ txnHandler.compact(new CompactionRequest("default", Table.ACIDTBLPART.name(), CompactionType.MAJOR));
+ runWorker(hiveConf);
+ runCleaner(hiveConf);
+ runStatementOnDriver("update " + Table.ACIDTBLPART + " set b = b + 1 where a = 3");
+ txnHandler.compact(new CompactionRequest("default", Table.ACIDTBLPART.toString(), CompactionType.MAJOR));
+ runWorker(hiveConf);
+ runCleaner(hiveConf);
+ List<String> rs = runStatementOnDriver("select p,a,b from " + Table.ACIDTBLPART + " order by p, a, b");
+ int[][] expectedData = {{1,1,2},{1,3,5},{1,5,6},{2,1,2},{2,3,5},{2,5,6}};
+ Assert.assertEquals("Update " + Table.ACIDTBLPART + " didn't match:", stringifyValues(expectedData), rs);
+ }
+
+ /**
* https://issues.apache.org/jira/browse/HIVE-10151
*/
@Test
http://git-wip-us.apache.org/repos/asf/hive/blob/b70efa44/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
index 6e2cf30..e94af55 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
@@ -536,6 +536,87 @@ public class TestDbTxnManager2 {
Assert.assertEquals(0, count);
}
+ /**
+ * collection of queries where we ensure that we get the locks that are expected
+ * @throws Exception
+ */
+ @Test
+ public void checkExpectedLocks() throws Exception {
+ CommandProcessorResponse cpr = null;
+ cpr = driver.run("create table acidPart(a int, b int) partitioned by (p string) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')");
+ checkCmdOnDriver(cpr);
+ cpr = driver.run("create table nonAcidPart(a int, b int) partitioned by (p string) stored as orc");
+ checkCmdOnDriver(cpr);
+
+ cpr = driver.compileAndRespond("insert into nonAcidPart partition(p) values(1,2,3)");
+ checkCmdOnDriver(cpr);
+ LockState lockState = ((DbTxnManager) txnMgr).acquireLocks(driver.getPlan(), ctx, "Practical", false);
+ List<ShowLocksResponseElement> locks = getLocks();
+ Assert.assertEquals("Unexpected lock count", 2, locks.size());
+ checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "values__tmp__table__1", null, locks.get(0));
+ checkLock(LockType.EXCLUSIVE, LockState.ACQUIRED, "default", "nonAcidPart", null, locks.get(1));
+ List<HiveLock> relLocks = new ArrayList<HiveLock>(2);
+ relLocks.add(new DbLockManager.DbHiveLock(locks.get(0).getLockid()));
+ relLocks.add(new DbLockManager.DbHiveLock(locks.get(1).getLockid()));
+ txnMgr.getLockManager().releaseLocks(relLocks);
+
+ cpr = driver.compileAndRespond("insert into nonAcidPart partition(p=1) values(5,6)");
+ checkCmdOnDriver(cpr);
+ lockState = ((DbTxnManager) txnMgr).acquireLocks(driver.getPlan(), ctx, "Practical", false);
+ locks = getLocks();
+ Assert.assertEquals("Unexpected lock count", 2, locks.size());
+ checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "values__tmp__table__2", null, locks.get(0));
+ checkLock(LockType.EXCLUSIVE, LockState.ACQUIRED, "default", "nonAcidPart", "p=1", locks.get(1));
+ relLocks = new ArrayList<HiveLock>(2);
+ relLocks.add(new DbLockManager.DbHiveLock(locks.get(0).getLockid()));
+ relLocks.add(new DbLockManager.DbHiveLock(locks.get(1).getLockid()));
+ txnMgr.getLockManager().releaseLocks(relLocks);
+
+ cpr = driver.compileAndRespond("insert into acidPart partition(p) values(1,2,3)");
+ checkCmdOnDriver(cpr);
+ lockState = ((DbTxnManager) txnMgr).acquireLocks(driver.getPlan(), ctx, "Practical", false);
+ locks = getLocks();
+ Assert.assertEquals("Unexpected lock count", 2, locks.size());
+ checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "values__tmp__table__3", null, locks.get(0));
+ checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "acidPart", null, locks.get(1));
+ relLocks = new ArrayList<HiveLock>(2);
+ relLocks.add(new DbLockManager.DbHiveLock(locks.get(0).getLockid()));
+ relLocks.add(new DbLockManager.DbHiveLock(locks.get(1).getLockid()));
+ txnMgr.getLockManager().releaseLocks(relLocks);
+
+ cpr = driver.compileAndRespond("insert into acidPart partition(p=1) values(5,6)");
+ checkCmdOnDriver(cpr);
+ lockState = ((DbTxnManager) txnMgr).acquireLocks(driver.getPlan(), ctx, "Practical", false);
+ locks = getLocks();
+ Assert.assertEquals("Unexpected lock count", 2, locks.size());
+ checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "values__tmp__table__4", null, locks.get(0));
+ checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "acidPart", "p=1", locks.get(1));
+ relLocks = new ArrayList<HiveLock>(2);
+ relLocks.add(new DbLockManager.DbHiveLock(locks.get(0).getLockid()));
+ relLocks.add(new DbLockManager.DbHiveLock(locks.get(1).getLockid()));
+ txnMgr.getLockManager().releaseLocks(relLocks);
+
+ cpr = driver.compileAndRespond("update acidPart set b = 17 where a = 1");
+ checkCmdOnDriver(cpr);
+ lockState = ((DbTxnManager) txnMgr).acquireLocks(driver.getPlan(), ctx, "Practical", false);
+ locks = getLocks();
+ Assert.assertEquals("Unexpected lock count", 1, locks.size());
+ checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "acidPart", null, locks.get(0));
+ relLocks = new ArrayList<HiveLock>(2);
+ relLocks.add(new DbLockManager.DbHiveLock(locks.get(0).getLockid()));
+ txnMgr.getLockManager().releaseLocks(relLocks);
+
+ cpr = driver.compileAndRespond("update acidPart set b = 17 where p = 1");
+ checkCmdOnDriver(cpr);
+ lockState = ((DbTxnManager) txnMgr).acquireLocks(driver.getPlan(), ctx, "Practical", false);
+ locks = getLocks();
+ Assert.assertEquals("Unexpected lock count", 1, locks.size());
+ checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "acidPart", null, locks.get(0));//https://issues.apache.org/jira/browse/HIVE-13212
+ relLocks = new ArrayList<HiveLock>(2);
+ relLocks.add(new DbLockManager.DbHiveLock(locks.get(0).getLockid()));
+ txnMgr.getLockManager().releaseLocks(relLocks);
+ }
+
private void checkLock(LockType type, LockState state, String db, String table, String partition, ShowLocksResponseElement l) {
Assert.assertEquals(l.toString(),l.getType(), type);
Assert.assertEquals(l.toString(),l.getState(), state);
[07/10] hive git commit: HIVE-13683 Remove erroneously included patch
file (Alan Gates)
Posted by jd...@apache.org.
HIVE-13683 Remove erroneously included patch file (Alan Gates)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/70fe3108
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/70fe3108
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/70fe3108
Branch: refs/heads/llap
Commit: 70fe31088639ebfdd114e026d8a332540dfbe3b2
Parents: 0b5c27f
Author: Alan Gates <ga...@hortonworks.com>
Authored: Tue May 3 15:51:44 2016 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Tue May 3 15:53:19 2016 -0700
----------------------------------------------------------------------
HIVE-13509.2.patch | 478 ------------------------------------------------
1 file changed, 478 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/70fe3108/HIVE-13509.2.patch
----------------------------------------------------------------------
diff --git a/HIVE-13509.2.patch b/HIVE-13509.2.patch
deleted file mode 100644
index 930b1f7..0000000
--- a/HIVE-13509.2.patch
+++ /dev/null
@@ -1,478 +0,0 @@
-diff --git a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatConstants.java b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatConstants.java
-index 6b03fcb..d165e7e 100644
---- a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatConstants.java
-+++ b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatConstants.java
-@@ -208,4 +208,7 @@ private HCatConstants() { // restrict instantiation
- */
- public static final String HCAT_INPUT_BAD_RECORD_MIN_KEY = "hcat.input.bad.record.min";
- public static final int HCAT_INPUT_BAD_RECORD_MIN_DEFAULT = 2;
-+
-+ public static final String HCAT_INPUT_IGNORE_INVALID_PATH_KEY = "hcat.input.ignore.invalid.path";
-+ public static final boolean HCAT_INPUT_IGNORE_INVALID_PATH_DEFAULT = false;
- }
-diff --git a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/HCatBaseInputFormat.java b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/HCatBaseInputFormat.java
-index adfaf4e..dbbdd61 100644
---- a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/HCatBaseInputFormat.java
-+++ b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/HCatBaseInputFormat.java
-@@ -21,11 +21,11 @@
-
- import java.io.IOException;
- import java.util.ArrayList;
-+import java.util.Iterator;
- import java.util.LinkedList;
- import java.util.Map;
- import java.util.HashMap;
- import java.util.List;
--
- import org.apache.hadoop.conf.Configuration;
- import org.apache.hadoop.fs.FileSystem;
- import org.apache.hadoop.fs.Path;
-@@ -127,7 +127,10 @@ public static void setOutputSchema(Job job, HCatSchema hcatSchema)
- //For each matching partition, call getSplits on the underlying InputFormat
- for (PartInfo partitionInfo : partitionInfoList) {
- jobConf = HCatUtil.getJobConfFromContext(jobContext);
-- setInputPath(jobConf, partitionInfo.getLocation());
-+ List<String> setInputPath = setInputPath(jobConf, partitionInfo.getLocation());
-+ if (setInputPath.isEmpty()) {
-+ continue;
-+ }
- Map<String, String> jobProperties = partitionInfo.getJobProperties();
-
- HCatUtil.copyJobPropertiesToJobConf(jobProperties, jobConf);
-@@ -281,7 +284,7 @@ private static InputJobInfo getJobInfo(Configuration conf)
- return (InputJobInfo) HCatUtil.deserialize(jobString);
- }
-
-- private void setInputPath(JobConf jobConf, String location)
-+ private List<String> setInputPath(JobConf jobConf, String location)
- throws IOException {
-
- // ideally we should just call FileInputFormat.setInputPaths() here - but
-@@ -322,19 +325,33 @@ private void setInputPath(JobConf jobConf, String location)
- }
- pathStrings.add(location.substring(pathStart, length));
-
-- Path[] paths = StringUtils.stringToPath(pathStrings.toArray(new String[0]));
- String separator = "";
- StringBuilder str = new StringBuilder();
-
-- for (Path path : paths) {
-+ boolean ignoreInvalidPath =jobConf.getBoolean(HCatConstants.HCAT_INPUT_IGNORE_INVALID_PATH_KEY,
-+ HCatConstants.HCAT_INPUT_IGNORE_INVALID_PATH_DEFAULT);
-+ Iterator<String> pathIterator = pathStrings.iterator();
-+ while (pathIterator.hasNext()) {
-+ String pathString = pathIterator.next();
-+ if (ignoreInvalidPath && org.apache.commons.lang.StringUtils.isBlank(pathString)) {
-+ continue;
-+ }
-+ Path path = new Path(pathString);
- FileSystem fs = path.getFileSystem(jobConf);
-+ if (ignoreInvalidPath && !fs.exists(path)) {
-+ pathIterator.remove();
-+ continue;
-+ }
- final String qualifiedPath = fs.makeQualified(path).toString();
- str.append(separator)
- .append(StringUtils.escapeString(qualifiedPath));
- separator = StringUtils.COMMA_STR;
- }
-
-- jobConf.set("mapred.input.dir", str.toString());
-+ if (!ignoreInvalidPath || !pathStrings.isEmpty()) {
-+ jobConf.set("mapred.input.dir", str.toString());
-+ }
-+ return pathStrings;
- }
-
- }
-diff --git a/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoader.java b/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoader.java
-index 2440cb5..4e23fa2 100644
---- a/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoader.java
-+++ b/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoader.java
-@@ -66,6 +66,7 @@
- import org.apache.pig.data.Tuple;
- import org.apache.pig.impl.logicalLayer.schema.Schema;
- import org.apache.pig.impl.logicalLayer.schema.Schema.FieldSchema;
-+import org.apache.pig.impl.util.PropertiesUtil;
- import org.joda.time.DateTime;
- import org.junit.After;
- import org.junit.Before;
-@@ -102,6 +103,7 @@
- add("testReadPartitionedBasic");
- add("testProjectionsBasic");
- add("testColumnarStorePushdown2");
-+ add("testReadMissingPartitionBasicNeg");
- }});
- }};
-
-@@ -438,6 +440,59 @@ public void testReadPartitionedBasic() throws IOException, CommandNeedRetryExcep
- }
-
- @Test
-+ public void testReadMissingPartitionBasicNeg() throws IOException, CommandNeedRetryException {
-+ assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
-+ PigServer server = new PigServer(ExecType.LOCAL);
-+
-+ File removedPartitionDir = new File(TEST_WAREHOUSE_DIR + "/" + PARTITIONED_TABLE + "/bkt=0");
-+ if (!removeDirectory(removedPartitionDir)) {
-+ System.out.println("Test did not run because its environment could not be set.");
-+ return;
-+ }
-+ driver.run("select * from " + PARTITIONED_TABLE);
-+ ArrayList<String> valuesReadFromHiveDriver = new ArrayList<String>();
-+ driver.getResults(valuesReadFromHiveDriver);
-+ assertTrue(valuesReadFromHiveDriver.size() == 6);
-+
-+ server.registerQuery("W = load '" + PARTITIONED_TABLE + "' using org.apache.hive.hcatalog.pig.HCatLoader();");
-+ Schema dumpedWSchema = server.dumpSchema("W");
-+ List<FieldSchema> Wfields = dumpedWSchema.getFields();
-+ assertEquals(3, Wfields.size());
-+ assertTrue(Wfields.get(0).alias.equalsIgnoreCase("a"));
-+ assertTrue(Wfields.get(0).type == DataType.INTEGER);
-+ assertTrue(Wfields.get(1).alias.equalsIgnoreCase("b"));
-+ assertTrue(Wfields.get(1).type == DataType.CHARARRAY);
-+ assertTrue(Wfields.get(2).alias.equalsIgnoreCase("bkt"));
-+ assertTrue(Wfields.get(2).type == DataType.CHARARRAY);
-+
-+ try {
-+ Iterator<Tuple> WIter = server.openIterator("W");
-+ fail("Should failed in retriving an invalid partition");
-+ } catch (IOException ioe) {
-+ // expected
-+ }
-+ }
-+
-+ private static boolean removeDirectory(File dir) {
-+ boolean success = false;
-+ if (dir.isDirectory()) {
-+ File[] files = dir.listFiles();
-+ if (files != null && files.length > 0) {
-+ for (File file : files) {
-+ success = removeDirectory(file);
-+ if (!success) {
-+ return false;
-+ }
-+ }
-+ }
-+ success = dir.delete();
-+ } else {
-+ success = dir.delete();
-+ }
-+ return success;
-+ }
-+
-+ @Test
- public void testProjectionsBasic() throws IOException {
- assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
-
-diff --git a/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoaderWithProps.java b/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoaderWithProps.java
-new file mode 100644
-index 0000000..41fe79b
---- /dev/null
-+++ b/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoaderWithProps.java
-@@ -0,0 +1,305 @@
-+/**
-+ * 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.hive.hcatalog.pig;
-+
-+import java.io.File;
-+import java.io.FileWriter;
-+import java.io.IOException;
-+import java.io.PrintWriter;
-+import java.io.RandomAccessFile;
-+import java.sql.Date;
-+import java.sql.Timestamp;
-+import java.util.ArrayList;
-+import java.util.Collection;
-+import java.util.HashMap;
-+import java.util.HashSet;
-+import java.util.Iterator;
-+import java.util.List;
-+import java.util.Map;
-+import java.util.Properties;
-+import java.util.Set;
-+
-+import org.apache.commons.io.FileUtils;
-+import org.apache.hadoop.fs.FileSystem;
-+import org.apache.hadoop.fs.FileUtil;
-+import org.apache.hadoop.fs.Path;
-+import org.apache.hadoop.hive.cli.CliSessionState;
-+import org.apache.hadoop.hive.conf.HiveConf;
-+import org.apache.hadoop.hive.ql.CommandNeedRetryException;
-+import org.apache.hadoop.hive.ql.Driver;
-+import org.apache.hadoop.hive.ql.WindowsPathUtil;
-+import org.apache.hadoop.hive.ql.io.IOConstants;
-+import org.apache.hadoop.hive.ql.io.StorageFormats;
-+import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
-+import org.apache.hadoop.hive.ql.session.SessionState;
-+import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
-+import org.apache.hadoop.mapreduce.Job;
-+import org.apache.hadoop.util.Shell;
-+import org.apache.hive.hcatalog.HcatTestUtils;
-+import org.apache.hive.hcatalog.common.HCatUtil;
-+import org.apache.hive.hcatalog.common.HCatConstants;
-+import org.apache.hive.hcatalog.data.Pair;
-+import org.apache.hive.hcatalog.data.schema.HCatFieldSchema;
-+import org.apache.pig.ExecType;
-+import org.apache.pig.PigRunner;
-+import org.apache.pig.PigServer;
-+import org.apache.pig.ResourceStatistics;
-+import org.apache.pig.tools.pigstats.OutputStats;
-+import org.apache.pig.tools.pigstats.PigStats;
-+import org.apache.pig.data.DataType;
-+import org.apache.pig.data.Tuple;
-+import org.apache.pig.impl.logicalLayer.schema.Schema;
-+import org.apache.pig.impl.logicalLayer.schema.Schema.FieldSchema;
-+import org.apache.pig.impl.util.PropertiesUtil;
-+import org.joda.time.DateTime;
-+import org.junit.After;
-+import org.junit.Before;
-+import org.junit.Test;
-+import org.junit.runner.RunWith;
-+import org.junit.runners.Parameterized;
-+import org.slf4j.Logger;
-+import org.slf4j.LoggerFactory;
-+
-+import static org.junit.Assert.*;
-+import static org.junit.Assume.assumeTrue;
-+
-+@RunWith(Parameterized.class)
-+public class TestHCatLoaderWithProps {
-+ private static final Logger LOG = LoggerFactory.getLogger(TestHCatLoaderWithProps.class);
-+ private static final String TEST_DATA_DIR = HCatUtil.makePathASafeFileName(System.getProperty("java.io.tmpdir") +
-+ File.separator + TestHCatLoaderWithProps.class.getCanonicalName() + "-" + System.currentTimeMillis());
-+ private static final String TEST_WAREHOUSE_DIR = TEST_DATA_DIR + "/warehouse";
-+ private static final String BASIC_FILE_NAME = TEST_DATA_DIR + "/basic.input.data";
-+
-+ private static final String BASIC_TABLE = "junit_unparted_basic";
-+ private static final String PARTITIONED_TABLE = "junit_parted_basic";
-+
-+ private Driver driver;
-+ private Map<Integer, Pair<Integer, String>> basicInputData;
-+
-+ private static final Map<String, Set<String>> DISABLED_STORAGE_FORMATS =
-+ new HashMap<String, Set<String>>() {{
-+ put(IOConstants.PARQUETFILE, new HashSet<String>() {{
-+ add("testReadMissingPartitionBasic");
-+ }});
-+ }};
-+
-+ private final String storageFormat;
-+
-+ @Parameterized.Parameters
-+ public static Collection<Object[]> generateParameters() {
-+ return StorageFormats.names();
-+ }
-+
-+ public TestHCatLoaderWithProps(String storageFormat) {
-+ this.storageFormat = storageFormat;
-+ }
-+
-+ private void dropTable(String tablename) throws IOException, CommandNeedRetryException {
-+ dropTable(tablename, driver);
-+ }
-+
-+ static void dropTable(String tablename, Driver driver) throws IOException, CommandNeedRetryException {
-+ driver.run("drop table if exists " + tablename);
-+ }
-+
-+ private void createTable(String tablename, String schema, String partitionedBy) throws IOException, CommandNeedRetryException {
-+ createTable(tablename, schema, partitionedBy, driver, storageFormat);
-+ }
-+
-+ static void createTable(String tablename, String schema, String partitionedBy, Driver driver, String storageFormat)
-+ throws IOException, CommandNeedRetryException {
-+ String createTable;
-+ createTable = "create table " + tablename + "(" + schema + ") ";
-+ if ((partitionedBy != null) && (!partitionedBy.trim().isEmpty())) {
-+ createTable = createTable + "partitioned by (" + partitionedBy + ") ";
-+ }
-+ createTable = createTable + "stored as " +storageFormat;
-+ executeStatementOnDriver(createTable, driver);
-+ }
-+
-+ private void createTable(String tablename, String schema) throws IOException, CommandNeedRetryException {
-+ createTable(tablename, schema, null);
-+ }
-+
-+ /**
-+ * Execute Hive CLI statement
-+ * @param cmd arbitrary statement to execute
-+ */
-+ static void executeStatementOnDriver(String cmd, Driver driver) throws IOException, CommandNeedRetryException {
-+ LOG.debug("Executing: " + cmd);
-+ CommandProcessorResponse cpr = driver.run(cmd);
-+ if(cpr.getResponseCode() != 0) {
-+ throw new IOException("Failed to execute \"" + cmd + "\". Driver returned " + cpr.getResponseCode() + " Error: " + cpr.getErrorMessage());
-+ }
-+ }
-+
-+ @Before
-+ public void setup() throws Exception {
-+ File f = new File(TEST_WAREHOUSE_DIR);
-+ if (f.exists()) {
-+ FileUtil.fullyDelete(f);
-+ }
-+ if (!(new File(TEST_WAREHOUSE_DIR).mkdirs())) {
-+ throw new RuntimeException("Could not create " + TEST_WAREHOUSE_DIR);
-+ }
-+
-+ HiveConf hiveConf = new HiveConf(this.getClass());
-+ hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, "");
-+ hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, "");
-+ hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false");
-+ hiveConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, TEST_WAREHOUSE_DIR);
-+ hiveConf.setVar(HiveConf.ConfVars.HIVEMAPREDMODE, "nonstrict");
-+
-+ if (Shell.WINDOWS) {
-+ WindowsPathUtil.convertPathsFromWindowsToHdfs(hiveConf);
-+ }
-+
-+ driver = new Driver(hiveConf);
-+ SessionState.start(new CliSessionState(hiveConf));
-+
-+ createTable(BASIC_TABLE, "a int, b string");
-+ createTable(PARTITIONED_TABLE, "a int, b string", "bkt string");
-+
-+ int LOOP_SIZE = 3;
-+ String[] input = new String[LOOP_SIZE * LOOP_SIZE];
-+ basicInputData = new HashMap<Integer, Pair<Integer, String>>();
-+ int k = 0;
-+ for (int i = 1; i <= LOOP_SIZE; i++) {
-+ String si = i + "";
-+ for (int j = 1; j <= LOOP_SIZE; j++) {
-+ String sj = "S" + j + "S";
-+ input[k] = si + "\t" + sj;
-+ basicInputData.put(k, new Pair<Integer, String>(i, sj));
-+ k++;
-+ }
-+ }
-+ HcatTestUtils.createTestDataFile(BASIC_FILE_NAME, input);
-+
-+ PigServer server = new PigServer(ExecType.LOCAL);
-+ server.setBatchOn();
-+ int i = 0;
-+ server.registerQuery("A = load '" + BASIC_FILE_NAME + "' as (a:int, b:chararray);", ++i);
-+
-+ server.registerQuery("store A into '" + BASIC_TABLE + "' using org.apache.hive.hcatalog.pig.HCatStorer();", ++i);
-+ server.registerQuery("B = foreach A generate a,b;", ++i);
-+ server.registerQuery("B2 = filter B by a < 2;", ++i);
-+ server.registerQuery("store B2 into '" + PARTITIONED_TABLE + "' using org.apache.hive.hcatalog.pig.HCatStorer('bkt=0');", ++i);
-+
-+ server.registerQuery("C = foreach A generate a,b;", ++i);
-+ server.registerQuery("C2 = filter C by a >= 2;", ++i);
-+ server.registerQuery("store C2 into '" + PARTITIONED_TABLE + "' using org.apache.hive.hcatalog.pig.HCatStorer('bkt=1');", ++i);
-+
-+ server.executeBatch();
-+ }
-+
-+ @After
-+ public void tearDown() throws Exception {
-+ try {
-+ if (driver != null) {
-+ dropTable(BASIC_TABLE);
-+ dropTable(PARTITIONED_TABLE);
-+ }
-+ } finally {
-+ FileUtils.deleteDirectory(new File(TEST_DATA_DIR));
-+ }
-+ }
-+
-+ @Test
-+ public void testReadMissingPartitionBasic() throws IOException, CommandNeedRetryException {
-+ assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
-+ Properties pigProperties = PropertiesUtil.loadDefaultProperties();
-+ pigProperties.setProperty("hcat.input.ignore.invalid.path", "true");
-+ PigServer server = new PigServer(ExecType.LOCAL, pigProperties);
-+
-+ File removedPartitionDir = new File(TEST_WAREHOUSE_DIR + "/" + PARTITIONED_TABLE + "/bkt=0");
-+ if (!removeDirectory(removedPartitionDir)) {
-+ System.out.println("Test did not run because its environment could not be set.");
-+ return;
-+ }
-+ driver.run("select * from " + PARTITIONED_TABLE);
-+ ArrayList<String> valuesReadFromHiveDriver = new ArrayList<String>();
-+ driver.getResults(valuesReadFromHiveDriver);
-+ assertTrue(valuesReadFromHiveDriver.size() == 6);
-+
-+ server.registerQuery("W = load '" + PARTITIONED_TABLE + "' using org.apache.hive.hcatalog.pig.HCatLoader();");
-+ Schema dumpedWSchema = server.dumpSchema("W");
-+ List<FieldSchema> Wfields = dumpedWSchema.getFields();
-+ assertEquals(3, Wfields.size());
-+ assertTrue(Wfields.get(0).alias.equalsIgnoreCase("a"));
-+ assertTrue(Wfields.get(0).type == DataType.INTEGER);
-+ assertTrue(Wfields.get(1).alias.equalsIgnoreCase("b"));
-+ assertTrue(Wfields.get(1).type == DataType.CHARARRAY);
-+ assertTrue(Wfields.get(2).alias.equalsIgnoreCase("bkt"));
-+ assertTrue(Wfields.get(2).type == DataType.CHARARRAY);
-+
-+ Iterator<Tuple> WIter = server.openIterator("W");
-+ Collection<Pair<Integer, String>> valuesRead = new ArrayList<Pair<Integer, String>>();
-+ while (WIter.hasNext()) {
-+ Tuple t = WIter.next();
-+ assertTrue(t.size() == 3);
-+ assertNotNull(t.get(0));
-+ assertNotNull(t.get(1));
-+ assertNotNull(t.get(2));
-+ assertTrue(t.get(0).getClass() == Integer.class);
-+ assertTrue(t.get(1).getClass() == String.class);
-+ assertTrue(t.get(2).getClass() == String.class);
-+ valuesRead.add(new Pair<Integer, String>((Integer) t.get(0), (String) t.get(1)));
-+ // the returned partition value is always 1
-+ assertEquals("1", t.get(2));
-+ }
-+ assertEquals(valuesReadFromHiveDriver.size(), valuesRead.size());
-+
-+ server.registerQuery("P1 = load '" + PARTITIONED_TABLE + "' using org.apache.hive.hcatalog.pig.HCatLoader();");
-+ server.registerQuery("P1filter = filter P1 by bkt == '0';");
-+ Iterator<Tuple> P1Iter = server.openIterator("P1filter");
-+ assertFalse(P1Iter.hasNext());
-+
-+ server.registerQuery("P2 = load '" + PARTITIONED_TABLE + "' using org.apache.hive.hcatalog.pig.HCatLoader();");
-+ server.registerQuery("P2filter = filter P2 by bkt == '1';");
-+ Iterator<Tuple> P2Iter = server.openIterator("P2filter");
-+ int count2 = 0;
-+ while (P2Iter.hasNext()) {
-+ Tuple t = P2Iter.next();
-+ assertEquals("1", t.get(2));
-+ assertTrue(((Integer) t.get(0)) > 1);
-+ count2++;
-+ }
-+ assertEquals(6, count2);
-+ }
-+
-+ private static boolean removeDirectory(File dir) {
-+ boolean success = false;
-+ if (dir.isDirectory()) {
-+ File[] files = dir.listFiles();
-+ if (files != null && files.length > 0) {
-+ for (File file : files) {
-+ success = removeDirectory(file);
-+ if (!success) {
-+ return false;
-+ }
-+ }
-+ }
-+ success = dir.delete();
-+ } else {
-+ success = dir.delete();
-+ }
-+ return success;
-+ }
-+}
[04/10] hive git commit: HIVE-13442 : LLAP: refactor submit API to be
amenable to signing (Sergey Shelukhin, reviewed by Siddharth Seth)
Posted by jd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/0b5c27fd/llap-common/src/java/org/apache/hadoop/hive/llap/tez/Converters.java
----------------------------------------------------------------------
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 ec6e439..e43b72b 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
@@ -22,9 +22,11 @@ import com.google.protobuf.ByteString;
import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto;
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.FragmentSpecProto;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SignableVertexSpec;
import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SourceStateProto;
import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.UserPayloadProto;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier;
+import org.apache.hadoop.yarn.util.ConverterUtils;
import org.apache.tez.common.TezCommonUtils;
import org.apache.tez.dag.api.EntityDescriptor;
import org.apache.tez.dag.api.InputDescriptor;
@@ -33,7 +35,10 @@ import org.apache.tez.dag.api.ProcessorDescriptor;
import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.dag.api.UserPayload;
import org.apache.tez.dag.api.event.VertexState;
+import org.apache.tez.dag.records.TezDAGID;
import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.dag.records.TezTaskID;
+import org.apache.tez.dag.records.TezVertexID;
import org.apache.tez.runtime.api.impl.GroupInputSpec;
import org.apache.tez.runtime.api.impl.InputSpec;
import org.apache.tez.runtime.api.impl.OutputSpec;
@@ -41,55 +46,88 @@ import org.apache.tez.runtime.api.impl.TaskSpec;
public class Converters {
- public static TaskSpec getTaskSpecfromProto(FragmentSpecProto FragmentSpecProto) {
- TezTaskAttemptID taskAttemptID =
- TezTaskAttemptID.fromString(FragmentSpecProto.getFragmentIdentifierString());
+ public static TaskSpec getTaskSpecfromProto(SignableVertexSpec vectorProto,
+ int fragmentNum, int attemptNum, TezTaskAttemptID attemptId) {
+ VertexIdentifier vertexId = vectorProto.getVertexIdentifier();
+ TezTaskAttemptID taskAttemptID = attemptId != null ? attemptId
+ : createTaskAttemptId(vertexId, fragmentNum, attemptNum);
ProcessorDescriptor processorDescriptor = null;
- if (FragmentSpecProto.hasProcessorDescriptor()) {
+ if (vectorProto.hasProcessorDescriptor()) {
processorDescriptor = convertProcessorDescriptorFromProto(
- FragmentSpecProto.getProcessorDescriptor());
+ vectorProto.getProcessorDescriptor());
}
- List<InputSpec> inputSpecList = new ArrayList<InputSpec>(FragmentSpecProto.getInputSpecsCount());
- if (FragmentSpecProto.getInputSpecsCount() > 0) {
- for (IOSpecProto inputSpecProto : FragmentSpecProto.getInputSpecsList()) {
+ List<InputSpec> inputSpecList = new ArrayList<InputSpec>(vectorProto.getInputSpecsCount());
+ if (vectorProto.getInputSpecsCount() > 0) {
+ for (IOSpecProto inputSpecProto : vectorProto.getInputSpecsList()) {
inputSpecList.add(getInputSpecFromProto(inputSpecProto));
}
}
List<OutputSpec> outputSpecList =
- new ArrayList<OutputSpec>(FragmentSpecProto.getOutputSpecsCount());
- if (FragmentSpecProto.getOutputSpecsCount() > 0) {
- for (IOSpecProto outputSpecProto : FragmentSpecProto.getOutputSpecsList()) {
+ new ArrayList<OutputSpec>(vectorProto.getOutputSpecsCount());
+ if (vectorProto.getOutputSpecsCount() > 0) {
+ for (IOSpecProto outputSpecProto : vectorProto.getOutputSpecsList()) {
outputSpecList.add(getOutputSpecFromProto(outputSpecProto));
}
}
List<GroupInputSpec> groupInputSpecs =
- new ArrayList<GroupInputSpec>(FragmentSpecProto.getGroupedInputSpecsCount());
- if (FragmentSpecProto.getGroupedInputSpecsCount() > 0) {
- for (GroupInputSpecProto groupInputSpecProto : FragmentSpecProto.getGroupedInputSpecsList()) {
+ new ArrayList<GroupInputSpec>(vectorProto.getGroupedInputSpecsCount());
+ if (vectorProto.getGroupedInputSpecsCount() > 0) {
+ for (GroupInputSpecProto groupInputSpecProto : vectorProto.getGroupedInputSpecsList()) {
groupInputSpecs.add(getGroupInputSpecFromProto(groupInputSpecProto));
}
}
TaskSpec taskSpec =
- new TaskSpec(taskAttemptID, FragmentSpecProto.getDagName(), FragmentSpecProto.getVertexName(),
- FragmentSpecProto.getVertexParallelism(), processorDescriptor, inputSpecList,
+ new TaskSpec(taskAttemptID, vectorProto.getDagName(), vectorProto.getVertexName(),
+ vectorProto.getVertexParallelism(), processorDescriptor, inputSpecList,
outputSpecList, groupInputSpecs);
return taskSpec;
}
- public static FragmentSpecProto convertTaskSpecToProto(TaskSpec taskSpec) {
- FragmentSpecProto.Builder builder = FragmentSpecProto.newBuilder();
- builder.setFragmentIdentifierString(taskSpec.getTaskAttemptID().toString());
+ public static TezTaskAttemptID createTaskAttemptId(
+ VertexIdentifier vertexId, int fragmentNum, int attemptNum) {
+ // Come ride the API roller-coaster!
+ return TezTaskAttemptID.getInstance(
+ TezTaskID.getInstance(
+ TezVertexID.getInstance(
+ TezDAGID.getInstance(
+ ConverterUtils.toApplicationId(
+ vertexId.getApplicationIdString()),
+ vertexId.getDagId()),
+ vertexId.getVertexId()),
+ fragmentNum),
+ attemptNum);
+ }
+
+ public static VertexIdentifier createVertexIdentifier(
+ TezTaskAttemptID taId, int appAttemptId) {
+ VertexIdentifier.Builder idBuilder = VertexIdentifier.newBuilder();
+ idBuilder.setApplicationIdString(
+ taId.getTaskID().getVertexID().getDAGId().getApplicationId().toString());
+ idBuilder.setAppAttemptNumber(appAttemptId);
+ idBuilder.setDagId(taId.getTaskID().getVertexID().getDAGId().getId());
+ idBuilder.setVertexId(taId.getTaskID().getVertexID().getId());
+ return idBuilder.build();
+ }
+
+ public static SignableVertexSpec convertTaskSpecToProto(TaskSpec taskSpec,
+ int appAttemptId, String tokenIdentifier, Integer signatureKeyId, String user) {
+ TezTaskAttemptID tId = taskSpec.getTaskAttemptID();
+
+ SignableVertexSpec.Builder builder = SignableVertexSpec.newBuilder();
+ builder.setVertexIdentifier(createVertexIdentifier(tId, appAttemptId));
builder.setDagName(taskSpec.getDAGName());
- builder.setDagId(taskSpec.getDagIdentifier());
builder.setVertexName(taskSpec.getVertexName());
builder.setVertexParallelism(taskSpec.getVertexParallelism());
- builder.setFragmentNumber(taskSpec.getTaskAttemptID().getTaskID().getId());
- builder.setAttemptNumber(taskSpec.getTaskAttemptID().getId());
+ builder.setTokenIdentifier(tokenIdentifier);
+ builder.setUser(user);
+ if (signatureKeyId != null) {
+ builder.setSignatureKeyId(signatureKeyId);
+ }
if (taskSpec.getProcessorDescriptor() != null) {
builder.setProcessorDescriptor(
http://git-wip-us.apache.org/repos/asf/hive/blob/0b5c27fd/llap-common/src/protobuf/LlapDaemonProtocol.proto
----------------------------------------------------------------------
diff --git a/llap-common/src/protobuf/LlapDaemonProtocol.proto b/llap-common/src/protobuf/LlapDaemonProtocol.proto
index 5cdc02e..486ba0a 100644
--- a/llap-common/src/protobuf/LlapDaemonProtocol.proto
+++ b/llap-common/src/protobuf/LlapDaemonProtocol.proto
@@ -46,19 +46,38 @@ message GroupInputSpecProto {
optional EntityDescriptorProto merged_input_descriptor = 3;
}
+message VertexIdentifier {
+ optional string application_id_string = 1;
+ optional int32 app_attempt_number = 2;
+ optional int32 dag_id = 3;
+ optional int32 vertex_id = 4;
+}
+
+// The part of SubmitWork that can be signed
+message SignableVertexSpec
+{
+ optional string user = 1;
+ optional int64 signatureKeyId = 2;
+
+ optional VertexIdentifier vertexIdentifier = 3;
+ // Display names cannot be modified by the client for now. If needed, they should be sent to HS2 who will put them here.
+ optional string dag_name = 4;
+ optional string vertex_name = 5;
+
+ // The core vertex stuff
+ optional string token_identifier = 6;
+ optional EntityDescriptorProto processor_descriptor = 7;
+ repeated IOSpecProto input_specs = 8;
+ repeated IOSpecProto output_specs = 9;
+ repeated GroupInputSpecProto grouped_input_specs = 10;
+
+ optional int32 vertex_parallelism = 11; // An internal field required for Tez.
+}
-message FragmentSpecProto {
- optional string fragment_identifier_string = 1;
- optional string dag_name = 2;
- optional int32 dag_id = 11;
- optional string vertex_name = 3;
- optional EntityDescriptorProto processor_descriptor = 4;
- repeated IOSpecProto input_specs = 5;
- repeated IOSpecProto output_specs = 6;
- repeated GroupInputSpecProto grouped_input_specs = 7;
- optional int32 vertex_parallelism = 8;
- optional int32 fragment_number =9;
- optional int32 attempt_number = 10;
+// Union
+message VertexOrBinary {
+ optional SignableVertexSpec vertex = 1;
+ optional bytes vertexBinary = 2; // SignableVertexSpec
}
message FragmentRuntimeInfo {
@@ -81,18 +100,24 @@ message QueryIdentifierProto {
}
message SubmitWorkRequestProto {
- optional string container_id_string = 1;
- optional string am_host = 2;
- optional int32 am_port = 3;
- optional string token_identifier = 4;
- optional bytes credentials_binary = 5;
- optional string user = 6;
- optional string application_id_string = 7;
- optional int32 app_attempt_number = 8;
- optional FragmentSpecProto fragment_spec = 9;
- optional FragmentRuntimeInfo fragment_runtime_info = 10;
+ optional VertexOrBinary work_spec = 1;
+ optional bytes work_spec_signature = 2;
+
+ optional int32 fragment_number = 3;
+ optional int32 attempt_number = 4;
+
+ optional string container_id_string = 5;
+ optional string am_host = 6;
+ optional int32 am_port = 7;
+
+ // Credentials are not signed - the client can add e.g. his own HDFS tokens.
+ optional bytes credentials_binary = 8;
+
+ // Not supported/honored for external clients right now.
+ optional FragmentRuntimeInfo fragment_runtime_info = 9;
}
+
enum SubmissionStateProto {
ACCEPTED = 1;
REJECTED = 2;
http://git-wip-us.apache.org/repos/asf/hive/blob/0b5c27fd/llap-common/src/test/org/apache/hadoop/hive/llap/tez/TestConverters.java
----------------------------------------------------------------------
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 d4cdac1..349ee14 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
@@ -23,8 +23,8 @@ import java.util.List;
import com.google.common.collect.Lists;
import com.google.protobuf.ByteString;
import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto;
-import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto;
import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SignableVertexSpec;
import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.UserPayloadProto;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.tez.dag.api.InputDescriptor;
@@ -77,28 +77,24 @@ public class TestConverters {
new TaskSpec(tezTaskAttemptId, "dagName", "vertexName", 10, processorDescriptor,
inputSpecList, outputSpecList, null);
+ SignableVertexSpec vertexProto = Converters.convertTaskSpecToProto(taskSpec, 0, "", null, "");
- FragmentSpecProto fragmentSpecProto = Converters.convertTaskSpecToProto(taskSpec);
-
-
- assertEquals("dagName", fragmentSpecProto.getDagName());
- assertEquals("vertexName", fragmentSpecProto.getVertexName());
- assertEquals(tezTaskAttemptId.toString(), fragmentSpecProto.getFragmentIdentifierString());
- assertEquals(tezDagId.getId(), fragmentSpecProto.getDagId());
- assertEquals(tezTaskAttemptId.getId(), fragmentSpecProto.getAttemptNumber());
- assertEquals(tezTaskId.getId(), fragmentSpecProto.getFragmentNumber());
+ assertEquals("dagName", vertexProto.getDagName());
+ assertEquals("vertexName", vertexProto.getVertexName());
+ assertEquals(appId.toString(), vertexProto.getVertexIdentifier().getApplicationIdString());
+ assertEquals(tezDagId.getId(), vertexProto.getVertexIdentifier().getDagId());
assertEquals(processorDescriptor.getClassName(),
- fragmentSpecProto.getProcessorDescriptor().getClassName());
+ vertexProto.getProcessorDescriptor().getClassName());
assertEquals(processorDescriptor.getUserPayload().getPayload(),
- fragmentSpecProto.getProcessorDescriptor().getUserPayload().getUserPayload()
+ vertexProto.getProcessorDescriptor().getUserPayload().getUserPayload()
.asReadOnlyByteBuffer());
- assertEquals(2, fragmentSpecProto.getInputSpecsCount());
- assertEquals(2, fragmentSpecProto.getOutputSpecsCount());
+ assertEquals(2, vertexProto.getInputSpecsCount());
+ assertEquals(2, vertexProto.getOutputSpecsCount());
- verifyInputSpecAndProto(inputSpec1, fragmentSpecProto.getInputSpecs(0));
- verifyInputSpecAndProto(inputSpec2, fragmentSpecProto.getInputSpecs(1));
- verifyOutputSpecAndProto(outputSpec1, fragmentSpecProto.getOutputSpecs(0));
- verifyOutputSpecAndProto(outputSpec2, fragmentSpecProto.getOutputSpecs(1));
+ verifyInputSpecAndProto(inputSpec1, vertexProto.getInputSpecs(0));
+ verifyInputSpecAndProto(inputSpec2, vertexProto.getInputSpecs(1));
+ verifyOutputSpecAndProto(outputSpec1, vertexProto.getOutputSpecs(0));
+ verifyOutputSpecAndProto(outputSpec2, vertexProto.getOutputSpecs(1));
}
@@ -120,11 +116,10 @@ public class TestConverters {
TezTaskID tezTaskId = TezTaskID.getInstance(tezVertexId, 500);
TezTaskAttemptID tezTaskAttemptId = TezTaskAttemptID.getInstance(tezTaskId, 600);
- FragmentSpecProto.Builder builder = FragmentSpecProto.newBuilder();
- builder.setFragmentIdentifierString(tezTaskAttemptId.toString());
+ SignableVertexSpec.Builder builder = SignableVertexSpec.newBuilder();
+ builder.setVertexIdentifier(Converters.createVertexIdentifier(tezTaskAttemptId, 0));
builder.setDagName("dagName");
builder.setVertexName("vertexName");
- builder.setDagId(tezDagId.getId());
builder.setProcessorDescriptor(
EntityDescriptorProto.newBuilder().setClassName("fakeProcessorName").setUserPayload(
UserPayloadProto.newBuilder().setUserPayload(ByteString.copyFrom(procBb))));
@@ -145,9 +140,9 @@ public class TestConverters {
EntityDescriptorProto.newBuilder().setClassName("outputClassName").setUserPayload(
UserPayloadProto.newBuilder().setUserPayload(ByteString.copyFrom(output1Bb)))));
- FragmentSpecProto fragmentSpecProto = builder.build();
+ SignableVertexSpec vertexProto = builder.build();
- TaskSpec taskSpec = Converters.getTaskSpecfromProto(fragmentSpecProto);
+ TaskSpec taskSpec = Converters.getTaskSpecfromProto(vertexProto, 0, 0, null);
assertEquals("dagName", taskSpec.getDAGName());
assertEquals("vertexName", taskSpec.getVertexName());
@@ -160,12 +155,10 @@ public class TestConverters {
assertEquals(2, taskSpec.getInputs().size());
assertEquals(2, taskSpec.getOutputs().size());
- verifyInputSpecAndProto(taskSpec.getInputs().get(0), fragmentSpecProto.getInputSpecs(0));
- verifyInputSpecAndProto(taskSpec.getInputs().get(1), fragmentSpecProto.getInputSpecs(1));
- verifyOutputSpecAndProto(taskSpec.getOutputs().get(0), fragmentSpecProto.getOutputSpecs(0));
- verifyOutputSpecAndProto(taskSpec.getOutputs().get(1), fragmentSpecProto.getOutputSpecs(1));
-
-
+ verifyInputSpecAndProto(taskSpec.getInputs().get(0), vertexProto.getInputSpecs(0));
+ verifyInputSpecAndProto(taskSpec.getInputs().get(1), vertexProto.getInputSpecs(1));
+ verifyOutputSpecAndProto(taskSpec.getOutputs().get(0), vertexProto.getOutputSpecs(0));
+ verifyOutputSpecAndProto(taskSpec.getOutputs().get(1), vertexProto.getOutputSpecs(1));
}
private void verifyInputSpecAndProto(InputSpec inputSpec,
http://git-wip-us.apache.org/repos/asf/hive/blob/0b5c27fd/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
----------------------------------------------------------------------
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 78b37f7..2bfe3ed 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
@@ -33,11 +33,11 @@ import org.apache.hadoop.hive.llap.daemon.HistoryLogger;
import org.apache.hadoop.hive.llap.daemon.KilledTaskHandler;
import org.apache.hadoop.hive.llap.daemon.QueryFailedHandler;
import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentRuntimeInfo;
-import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto;
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.QueryCompleteRequestProto;
import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.QueryCompleteResponseProto;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SignableVertexSpec;
import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SourceStateUpdatedRequestProto;
import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SourceStateUpdatedResponseProto;
import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmissionStateProto;
@@ -45,7 +45,9 @@ import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWor
import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWorkResponseProto;
import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.TerminateFragmentRequestProto;
import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.TerminateFragmentResponseProto;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier;
import org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorMetrics;
+import org.apache.hadoop.hive.llap.tez.Converters;
import org.apache.hadoop.hive.ql.exec.tez.TezProcessor;
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.security.Credentials;
@@ -151,32 +153,35 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
@Override
public SubmitWorkResponseProto submitWork(SubmitWorkRequestProto request) throws IOException {
- HistoryLogger.logFragmentStart(request.getApplicationIdString(), request.getContainerIdString(),
- localAddress.get().getHostName(), request.getFragmentSpec().getDagName(), request.getFragmentSpec().getDagId(),
- request.getFragmentSpec().getVertexName(), request.getFragmentSpec().getFragmentNumber(),
- request.getFragmentSpec().getAttemptNumber());
+ // TODO: also support binary. Actually, we should figure out the binary stuff here and
+ // stop passing the protobuf around. We should pass around some plain objects/values.
+ SignableVertexSpec vertex = request.getWorkSpec().getVertex();
if (LOG.isInfoEnabled()) {
- LOG.info("Queueing container for execution: " + stringifySubmitRequest(request));
+ LOG.info("Queueing container for execution: " + stringifySubmitRequest(request, vertex));
}
+ VertexIdentifier vId = vertex.getVertexIdentifier();
+ TezTaskAttemptID attemptId = Converters.createTaskAttemptId(
+ vId, request.getFragmentNumber(), request.getAttemptNumber());
+ String fragmentIdString = attemptId.toString();
+ HistoryLogger.logFragmentStart(vId.getApplicationIdString(), request.getContainerIdString(),
+ localAddress.get().getHostName(), vertex.getDagName(), vId.getDagId(),
+ vertex.getVertexName(), request.getFragmentNumber(), request.getAttemptNumber());
// This is the start of container-annotated logging.
// TODO Reduce the length of this string. Way too verbose at the moment.
- String ndcContextString = request.getFragmentSpec().getFragmentIdentifierString();
- NDC.push(ndcContextString);
+ NDC.push(fragmentIdString);
Scheduler.SubmissionState submissionState;
SubmitWorkResponseProto.Builder responseBuilder = SubmitWorkResponseProto.newBuilder();
try {
Map<String, String> env = new HashMap<>();
// TODO What else is required in this environment map.
env.putAll(localEnv);
- env.put(ApplicationConstants.Environment.USER.name(), request.getUser());
+ env.put(ApplicationConstants.Environment.USER.name(), vertex.getUser());
- FragmentSpecProto fragmentSpec = request.getFragmentSpec();
- TezTaskAttemptID taskAttemptId = TezTaskAttemptID.fromString(
- fragmentSpec.getFragmentIdentifierString());
+ TezTaskAttemptID taskAttemptId = TezTaskAttemptID.fromString(fragmentIdString);
int dagIdentifier = taskAttemptId.getTaskID().getVertexID().getDAGId().getId();
QueryIdentifier queryIdentifier = new QueryIdentifier(
- request.getApplicationIdString(), dagIdentifier);
+ vId.getApplicationIdString(), dagIdentifier);
Credentials credentials = new Credentials();
DataInputBuffer dib = new DataInputBuffer();
@@ -186,14 +191,10 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
Token<JobTokenIdentifier> jobToken = TokenCache.getSessionToken(credentials);
- QueryFragmentInfo fragmentInfo = queryTracker
- .registerFragment(queryIdentifier, request.getApplicationIdString(),
- fragmentSpec.getDagName(),
- dagIdentifier,
- fragmentSpec.getVertexName(), fragmentSpec.getFragmentNumber(),
- fragmentSpec.getAttemptNumber(), request.getUser(), request.getFragmentSpec(),
- jobToken);
-
+ QueryFragmentInfo fragmentInfo = queryTracker.registerFragment(
+ queryIdentifier, vId.getApplicationIdString(), vertex.getDagName(), dagIdentifier,
+ vertex.getVertexName(), request.getFragmentNumber(), request.getAttemptNumber(),
+ vertex.getUser(), vertex, jobToken, fragmentIdString);
String[] localDirs = fragmentInfo.getLocalDirs();
Preconditions.checkNotNull(localDirs);
@@ -202,14 +203,16 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
}
// May need to setup localDir for re-localization, which is usually setup as Environment.PWD.
// Used for re-localization, to add the user specified configuration (conf_pb_binary_stream)
- TaskRunnerCallable callable = new TaskRunnerCallable(request, fragmentInfo, new Configuration(getConfig()),
+
+ Configuration callableConf = new Configuration(getConfig());
+ TaskRunnerCallable callable = new TaskRunnerCallable(request, fragmentInfo, callableConf,
new LlapExecutionContext(localAddress.get().getHostName(), queryTracker), env,
credentials, memoryPerExecutor, amReporter, confParams, metrics, killedTaskHandler,
- this, tezHadoopShim);
+ this, tezHadoopShim, attemptId);
submissionState = executorService.schedule(callable);
if (LOG.isInfoEnabled()) {
- LOG.info("SubmissionState for {} : {} ", ndcContextString, submissionState);
+ LOG.info("SubmissionState for {} : {} ", fragmentIdString, submissionState);
}
if (submissionState.equals(Scheduler.SubmissionState.REJECTED)) {
@@ -300,24 +303,25 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
return sb.toString();
}
- public static String stringifySubmitRequest(SubmitWorkRequestProto request) {
+ public static String stringifySubmitRequest(
+ SubmitWorkRequestProto request, SignableVertexSpec vertex) {
StringBuilder sb = new StringBuilder();
- FragmentSpecProto fragmentSpec = request.getFragmentSpec();
sb.append("am_details=").append(request.getAmHost()).append(":").append(request.getAmPort());
- sb.append(", taskInfo=").append(fragmentSpec.getFragmentIdentifierString());
- sb.append(", user=").append(request.getUser());
- sb.append(", appIdString=").append(request.getApplicationIdString());
- sb.append(", appAttemptNum=").append(request.getAppAttemptNumber());
+ sb.append(", taskInfo=").append(vertex.getVertexIdentifier()).append(" fragment ")
+ .append(request.getFragmentNumber()).append(" attempt ").append(request.getAttemptNumber());
+ sb.append(", user=").append(vertex.getUser());
+ sb.append(", appIdString=").append(vertex.getVertexIdentifier().getApplicationIdString());
+ sb.append(", appAttemptNum=").append(vertex.getVertexIdentifier().getAppAttemptNumber());
sb.append(", containerIdString=").append(request.getContainerIdString());
- sb.append(", dagName=").append(fragmentSpec.getDagName());
- sb.append(", vertexName=").append(fragmentSpec.getVertexName());
- sb.append(", processor=").append(fragmentSpec.getProcessorDescriptor().getClassName());
- sb.append(", numInputs=").append(fragmentSpec.getInputSpecsCount());
- sb.append(", numOutputs=").append(fragmentSpec.getOutputSpecsCount());
- sb.append(", numGroupedInputs=").append(fragmentSpec.getGroupedInputSpecsCount());
+ sb.append(", dagName=").append(vertex.getDagName());
+ sb.append(", vertexName=").append(vertex.getVertexName());
+ sb.append(", processor=").append(vertex.getProcessorDescriptor().getClassName());
+ sb.append(", numInputs=").append(vertex.getInputSpecsCount());
+ sb.append(", numOutputs=").append(vertex.getOutputSpecsCount());
+ sb.append(", numGroupedInputs=").append(vertex.getGroupedInputSpecsCount());
sb.append(", Inputs={");
- if (fragmentSpec.getInputSpecsCount() > 0) {
- for (IOSpecProto ioSpec : fragmentSpec.getInputSpecsList()) {
+ if (vertex.getInputSpecsCount() > 0) {
+ for (IOSpecProto ioSpec : vertex.getInputSpecsList()) {
sb.append("{").append(ioSpec.getConnectedVertexName()).append(",")
.append(ioSpec.getIoDescriptor().getClassName()).append(",")
.append(ioSpec.getPhysicalEdgeCount()).append("}");
@@ -325,8 +329,8 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
}
sb.append("}");
sb.append(", Outputs={");
- if (fragmentSpec.getOutputSpecsCount() > 0) {
- for (IOSpecProto ioSpec : fragmentSpec.getOutputSpecsList()) {
+ if (vertex.getOutputSpecsCount() > 0) {
+ for (IOSpecProto ioSpec : vertex.getOutputSpecsList()) {
sb.append("{").append(ioSpec.getConnectedVertexName()).append(",")
.append(ioSpec.getIoDescriptor().getClassName()).append(",")
.append(ioSpec.getPhysicalEdgeCount()).append("}");
@@ -334,8 +338,8 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
}
sb.append("}");
sb.append(", GroupedInputs={");
- if (fragmentSpec.getGroupedInputSpecsCount() > 0) {
- for (GroupInputSpecProto group : fragmentSpec.getGroupedInputSpecsList()) {
+ if (vertex.getGroupedInputSpecsCount() > 0) {
+ for (GroupInputSpecProto group : vertex.getGroupedInputSpecsList()) {
sb.append("{").append("groupName=").append(group.getGroupName()).append(", elements=")
.append(group.getGroupVerticesList()).append("}");
sb.append(group.getGroupVerticesList());
http://git-wip-us.apache.org/repos/asf/hive/blob/0b5c27fd/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryFragmentInfo.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryFragmentInfo.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryFragmentInfo.java
index 480a394..195775e 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryFragmentInfo.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryFragmentInfo.java
@@ -21,8 +21,8 @@ import java.util.List;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hive.llap.daemon.FinishableStateUpdateHandler;
import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos;
-import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto;
import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SignableVertexSpec;
import org.apache.hadoop.hive.llap.tezplugins.LlapTezUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -35,19 +35,20 @@ public class QueryFragmentInfo {
private final String vertexName;
private final int fragmentNumber;
private final int attemptNumber;
- private final FragmentSpecProto fragmentSpec;
+ private final SignableVertexSpec vertexSpec;
+ private final String fragmentIdString;
public QueryFragmentInfo(QueryInfo queryInfo, String vertexName, int fragmentNumber,
- int attemptNumber,
- FragmentSpecProto fragmentSpec) {
+ int attemptNumber, SignableVertexSpec vertexSpec, String fragmentIdString) {
Preconditions.checkNotNull(queryInfo);
Preconditions.checkNotNull(vertexName);
- Preconditions.checkNotNull(fragmentSpec);
+ Preconditions.checkNotNull(vertexSpec);
this.queryInfo = queryInfo;
this.vertexName = vertexName;
this.fragmentNumber = fragmentNumber;
this.attemptNumber = attemptNumber;
- this.fragmentSpec = fragmentSpec;
+ this.vertexSpec = vertexSpec;
+ this.fragmentIdString = fragmentIdString;
}
// Only meant for use by the QueryTracker
@@ -55,8 +56,8 @@ public class QueryFragmentInfo {
return this.queryInfo;
}
- public FragmentSpecProto getFragmentSpec() {
- return fragmentSpec;
+ public SignableVertexSpec getVertexSpec() {
+ return vertexSpec;
}
public String getVertexName() {
@@ -72,7 +73,7 @@ public class QueryFragmentInfo {
}
public String getFragmentIdentifierString() {
- return fragmentSpec.getFragmentIdentifierString();
+ return fragmentIdString;
}
/**
@@ -85,7 +86,7 @@ public class QueryFragmentInfo {
* @return true if the task can finish, false otherwise
*/
public boolean canFinish() {
- List<IOSpecProto> inputSpecList = fragmentSpec.getInputSpecsList();
+ List<IOSpecProto> inputSpecList = vertexSpec.getInputSpecsList();
boolean canFinish = true;
if (inputSpecList != null && !inputSpecList.isEmpty()) {
for (IOSpecProto inputSpec : inputSpecList) {
@@ -126,7 +127,7 @@ public class QueryFragmentInfo {
public boolean registerForFinishableStateUpdates(FinishableStateUpdateHandler handler,
boolean lastFinishableState) {
List<String> sourcesOfInterest = new LinkedList<>();
- List<IOSpecProto> inputSpecList = fragmentSpec.getInputSpecsList();
+ List<IOSpecProto> inputSpecList = vertexSpec.getInputSpecsList();
if (inputSpecList != null && !inputSpecList.isEmpty()) {
for (IOSpecProto inputSpec : inputSpecList) {
if (LlapTezUtils.isSourceOfInterest(inputSpec.getIoDescriptor().getClassName())) {
http://git-wip-us.apache.org/repos/asf/hive/blob/0b5c27fd/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryInfo.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryInfo.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryInfo.java
index 8daef9e..6914134 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryInfo.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryInfo.java
@@ -35,7 +35,7 @@ import com.google.common.collect.Multimap;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.llap.daemon.FinishableStateUpdateHandler;
-import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SignableVertexSpec;
import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SourceStateProto;
public class QueryInfo {
@@ -92,9 +92,10 @@ public class QueryInfo {
return sourceStateMap;
}
- public QueryFragmentInfo registerFragment(String vertexName, int fragmentNumber, int attemptNumber, FragmentSpecProto fragmentSpec) {
- QueryFragmentInfo fragmentInfo = new QueryFragmentInfo(this, vertexName, fragmentNumber, attemptNumber,
- fragmentSpec);
+ public QueryFragmentInfo registerFragment(String vertexName, int fragmentNumber,
+ int attemptNumber, SignableVertexSpec vertexSpec, String fragmentIdString) {
+ QueryFragmentInfo fragmentInfo = new QueryFragmentInfo(
+ this, vertexName, fragmentNumber, attemptNumber, vertexSpec, fragmentIdString);
knownFragments.add(fragmentInfo);
return fragmentInfo;
}
http://git-wip-us.apache.org/repos/asf/hive/blob/0b5c27fd/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryTracker.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryTracker.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryTracker.java
index cb3be2b..8abd198 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryTracker.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryTracker.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SignableVertexSpec;
import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SourceStateProto;
import org.apache.hadoop.hive.llap.shufflehandler.ShuffleHandler;
import org.apache.hadoop.hive.ql.exec.ObjectCacheFactory;
@@ -113,20 +113,11 @@ public class QueryTracker extends AbstractService {
/**
* Register a new fragment for a specific query
- * @param queryIdentifier
- * @param appIdString
- * @param dagName
- * @param dagIdentifier
- * @param vertexName
- * @param fragmentNumber
- * @param attemptNumber
- * @param user
- * @throws IOException
*/
QueryFragmentInfo registerFragment(QueryIdentifier queryIdentifier, String appIdString,
String dagName, int dagIdentifier, String vertexName, int fragmentNumber, int attemptNumber,
- String user, FragmentSpecProto fragmentSpec, Token<JobTokenIdentifier> appToken)
- throws IOException {
+ String user, SignableVertexSpec vertex, Token<JobTokenIdentifier> appToken,
+ String fragmentIdString) throws IOException {
ReadWriteLock dagLock = getDagLock(queryIdentifier);
dagLock.readLock().lock();
try {
@@ -166,7 +157,8 @@ public class QueryTracker extends AbstractService {
.registerDag(appIdString, dagIdentifier, appToken,
user, queryInfo.getLocalDirs());
- return queryInfo.registerFragment(vertexName, fragmentNumber, attemptNumber, fragmentSpec);
+ return queryInfo.registerFragment(
+ vertexName, fragmentNumber, attemptNumber, vertex, fragmentIdString);
} finally {
dagLock.readLock().unlock();
}
http://git-wip-us.apache.org/repos/asf/hive/blob/0b5c27fd/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorService.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorService.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorService.java
index 1933eb1..eac0e8f 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorService.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorService.java
@@ -43,7 +43,7 @@ import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.lang3.exception.ExceptionUtils;
import org.apache.hadoop.hive.llap.daemon.FinishableStateUpdateHandler;
import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentRuntimeInfo;
-import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SignableVertexSpec;
import org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorMetrics;
import org.apache.hadoop.service.AbstractService;
import org.apache.tez.runtime.task.EndReason;
@@ -191,8 +191,8 @@ public class TaskExecutorService extends AbstractService implements Scheduler<Ta
TaskWrapper task = e.getValue();
boolean isFirst = true;
TaskRunnerCallable c = task.getTaskRunnerCallable();
- if (c != null && c.getRequest() != null && c.getRequest().getFragmentSpec() != null) {
- FragmentSpecProto fs = c.getRequest().getFragmentSpec();
+ if (c != null && c.getVertexSpec() != null) {
+ SignableVertexSpec fs = c.getVertexSpec();
value.append(isFirst ? " (" : ", ").append(fs.getDagName())
.append("/").append(fs.getVertexName());
isFirst = false;
@@ -781,7 +781,7 @@ public class TaskExecutorService extends AbstractService implements Scheduler<Ta
", firstAttemptStartTime=" + taskRunnerCallable.getFragmentRuntimeInfo().getFirstAttemptStartTime() +
", dagStartTime=" + taskRunnerCallable.getFragmentRuntimeInfo().getDagStartTime() +
", withinDagPriority=" + taskRunnerCallable.getFragmentRuntimeInfo().getWithinDagPriority() +
- ", vertexParallelism= " + taskRunnerCallable.getFragmentSpec().getVertexParallelism() +
+ ", vertexParallelism= " + taskRunnerCallable.getVertexSpec().getVertexParallelism() +
", selfAndUpstreamParallelism= " + taskRunnerCallable.getFragmentRuntimeInfo().getNumSelfAndUpstreamTasks() +
", selfAndUpstreamComplete= " + taskRunnerCallable.getFragmentRuntimeInfo().getNumSelfAndUpstreamCompletedTasks() +
'}';
http://git-wip-us.apache.org/repos/asf/hive/blob/0b5c27fd/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
----------------------------------------------------------------------
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 fcfa940..3093de7 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
@@ -33,8 +33,8 @@ import org.apache.hadoop.hive.llap.daemon.FragmentCompletionHandler;
import org.apache.hadoop.hive.llap.daemon.HistoryLogger;
import org.apache.hadoop.hive.llap.daemon.KilledTaskHandler;
import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentRuntimeInfo;
-import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto;
import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto;
+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.metrics.LlapDaemonExecutorMetrics;
import org.apache.hadoop.hive.llap.protocol.LlapTaskUmbilicalProtocol;
@@ -113,6 +113,7 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
private final AtomicBoolean isStarted = new AtomicBoolean(false);
private final AtomicBoolean isCompleted = new AtomicBoolean(false);
private final AtomicBoolean killInvoked = new AtomicBoolean(false);
+ private final SignableVertexSpec vertex;
@VisibleForTesting
public TaskRunnerCallable(SubmitWorkRequestProto request, QueryFragmentInfo fragmentInfo,
@@ -123,7 +124,7 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
ConfParams confParams, LlapDaemonExecutorMetrics metrics,
KilledTaskHandler killedTaskHandler,
FragmentCompletionHandler fragmentCompleteHandler,
- HadoopShim tezHadoopShim) {
+ HadoopShim tezHadoopShim, TezTaskAttemptID attemptId) {
this.request = request;
this.fragmentInfo = fragmentInfo;
this.conf = conf;
@@ -134,17 +135,20 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
this.memoryAvailable = memoryAvailable;
this.confParams = confParams;
this.jobToken = TokenCache.getSessionToken(credentials);
- this.taskSpec = Converters.getTaskSpecfromProto(request.getFragmentSpec());
+ // TODO: support binary spec here or above
+ this.vertex = request.getWorkSpec().getVertex();
+ this.taskSpec = Converters.getTaskSpecfromProto(
+ vertex, request.getFragmentNumber(), request.getAttemptNumber(), attemptId);
this.amReporter = amReporter;
// Register with the AMReporter when the callable is setup. Unregister once it starts running.
if (jobToken != null) {
this.amReporter.registerTask(request.getAmHost(), request.getAmPort(),
- request.getUser(), jobToken, fragmentInfo.getQueryInfo().getQueryIdentifier());
+ vertex.getUser(), jobToken, fragmentInfo.getQueryInfo().getQueryIdentifier());
}
this.metrics = metrics;
- this.requestId = request.getFragmentSpec().getFragmentIdentifierString();
+ this.requestId = taskSpec.getTaskAttemptID().toString();
// TODO Change this to the queryId/Name when that's available.
- this.queryId = request.getFragmentSpec().getDagName();
+ this.queryId = vertex.getDagName();
this.killedTaskHandler = killedTaskHandler;
this.fragmentCompletionHanler = fragmentCompleteHandler;
this.tezHadoopShim = tezHadoopShim;
@@ -184,16 +188,16 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
// TODO Consolidate this code with TezChild.
runtimeWatch.start();
- UserGroupInformation taskUgi = UserGroupInformation.createRemoteUser(request.getUser());
+ UserGroupInformation taskUgi = UserGroupInformation.createRemoteUser(vertex.getUser());
taskUgi.addCredentials(credentials);
Map<String, ByteBuffer> serviceConsumerMetadata = new HashMap<>();
serviceConsumerMetadata.put(TezConstants.TEZ_SHUFFLE_HANDLER_SERVICE_ID,
TezCommonUtils.convertJobTokenToBytes(jobToken));
- Multimap<String, String> startedInputsMap = createStartedInputMap(request.getFragmentSpec());
+ Multimap<String, String> startedInputsMap = createStartedInputMap(vertex);
UserGroupInformation taskOwner =
- UserGroupInformation.createRemoteUser(request.getTokenIdentifier());
+ UserGroupInformation.createRemoteUser(vertex.getTokenIdentifier());
final InetSocketAddress address =
NetUtils.createSocketAddrForHost(request.getAmHost(), request.getAmPort());
SecurityUtil.setTokenService(jobToken, address);
@@ -228,7 +232,7 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
if (shouldRunTask) {
taskRunner = new TezTaskRunner2(conf, taskUgi, fragmentInfo.getLocalDirs(),
taskSpec,
- request.getAppAttemptNumber(),
+ vertex.getVertexIdentifier().getAppAttemptNumber(),
serviceConsumerMetadata, envMap, startedInputsMap, taskReporter, executor,
objectRegistry,
pid,
@@ -313,7 +317,7 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
*/
public void reportTaskKilled() {
killedTaskHandler
- .taskKilled(request.getAmHost(), request.getAmPort(), request.getUser(), jobToken,
+ .taskKilled(request.getAmHost(), request.getAmPort(), vertex.getUser(), jobToken,
fragmentInfo.getQueryInfo().getQueryIdentifier(), taskSpec.getTaskAttemptID());
}
@@ -321,15 +325,15 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
return fragmentInfo.canFinish();
}
- private Multimap<String, String> createStartedInputMap(FragmentSpecProto fragmentSpec) {
+ private static Multimap<String, String> createStartedInputMap(SignableVertexSpec vertex) {
Multimap<String, String> startedInputMap = HashMultimap.create();
// Let the Processor control start for Broadcast inputs.
// TODO For now, this affects non broadcast unsorted cases as well. Make use of the edge
// property when it's available.
- for (IOSpecProto inputSpec : fragmentSpec.getInputSpecsList()) {
+ for (IOSpecProto inputSpec : vertex.getInputSpecsList()) {
if (inputSpec.getIoDescriptor().getClassName().equals(UnorderedKVInput.class.getName())) {
- startedInputMap.put(fragmentSpec.getVertexName(), inputSpec.getConnectedVertexName());
+ startedInputMap.put(vertex.getVertexName(), inputSpec.getConnectedVertexName());
}
}
return startedInputMap;
@@ -350,7 +354,7 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
@Override
public String toString() {
return requestId + " {canFinish: " + canFinish() +
- ", vertexParallelism: " + request.getFragmentSpec().getVertexParallelism() +
+ ", vertexParallelism: " + vertex.getVertexParallelism() +
", selfAndUpstreamParallelism: " + request.getFragmentRuntimeInfo().getNumSelfAndUpstreamTasks() +
", selfAndUpstreamComplete: " + request.getFragmentRuntimeInfo().getNumSelfAndUpstreamCompletedTasks() +
", firstAttemptStartTime: " + getFragmentRuntimeInfo().getFirstAttemptStartTime() +
@@ -454,14 +458,7 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
fragmentCompletionHanler.fragmentComplete(fragmentInfo);
taskRunnerCallable.shutdown();
- HistoryLogger
- .logFragmentEnd(request.getApplicationIdString(), request.getContainerIdString(),
- executionContext.getHostName(), request.getFragmentSpec().getDagName(),
- fragmentInfo.getQueryInfo().getDagIdentifier(),
- request.getFragmentSpec().getVertexName(),
- request.getFragmentSpec().getFragmentNumber(),
- request.getFragmentSpec().getAttemptNumber(), taskRunnerCallable.threadName,
- taskRunnerCallable.startTime, true);
+ logFragmentEnd(true);
}
@Override
@@ -471,14 +468,15 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
fragmentCompletionHanler.fragmentComplete(fragmentInfo);
// TODO HIVE-10236 Report a fatal error over the umbilical
taskRunnerCallable.shutdown();
- HistoryLogger
- .logFragmentEnd(request.getApplicationIdString(), request.getContainerIdString(),
- executionContext.getHostName(), request.getFragmentSpec().getDagName(),
- fragmentInfo.getQueryInfo().getDagIdentifier(),
- request.getFragmentSpec().getVertexName(),
- request.getFragmentSpec().getFragmentNumber(),
- request.getFragmentSpec().getAttemptNumber(), taskRunnerCallable.threadName,
- taskRunnerCallable.startTime, false);
+ logFragmentEnd(false);
+ }
+
+ protected void logFragmentEnd(boolean success) {
+ HistoryLogger.logFragmentEnd(vertex.getVertexIdentifier().getApplicationIdString(),
+ request.getContainerIdString(), executionContext.getHostName(), vertex.getDagName(),
+ fragmentInfo.getQueryInfo().getDagIdentifier(), vertex.getVertexName(),
+ request.getFragmentNumber(), request.getAttemptNumber(), taskRunnerCallable.threadName,
+ taskRunnerCallable.startTime, success);
}
}
@@ -498,12 +496,14 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
public static String getTaskIdentifierString(
SubmitWorkRequestProto request) {
StringBuilder sb = new StringBuilder();
- sb.append("AppId=").append(request.getApplicationIdString())
+ // TODO: also support the binary version
+ SignableVertexSpec vertex = request.getWorkSpec().getVertex();
+ sb.append("AppId=").append(vertex.getVertexIdentifier().getApplicationIdString())
.append(", containerId=").append(request.getContainerIdString())
- .append(", Dag=").append(request.getFragmentSpec().getDagName())
- .append(", Vertex=").append(request.getFragmentSpec().getVertexName())
- .append(", FragmentNum=").append(request.getFragmentSpec().getFragmentNumber())
- .append(", Attempt=").append(request.getFragmentSpec().getAttemptNumber());
+ .append(", Dag=").append(vertex.getDagName())
+ .append(", Vertex=").append(vertex.getVertexName())
+ .append(", FragmentNum=").append(request.getFragmentNumber())
+ .append(", Attempt=").append(request.getAttemptNumber());
return sb.toString();
}
@@ -511,7 +511,8 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
return request.getFragmentRuntimeInfo();
}
- public FragmentSpecProto getFragmentSpec() {
- return request.getFragmentSpec();
+ public SignableVertexSpec getVertexSpec() {
+ // TODO: support for binary spec? presumably we'd parse it somewhere earlier
+ return vertex;
}
}
http://git-wip-us.apache.org/repos/asf/hive/blob/0b5c27fd/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorTestHelpers.java
----------------------------------------------------------------------
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 c6ba14e..d699f20 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
@@ -26,9 +26,11 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hive.llap.daemon.FragmentCompletionHandler;
import org.apache.hadoop.hive.llap.daemon.KilledTaskHandler;
import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos;
-import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto;
+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.VertexOrBinary;
import org.apache.hadoop.hive.llap.metrics.LlapDaemonExecutorMetrics;
+import org.apache.hadoop.hive.llap.tez.Converters;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.tez.dag.records.TezDAGID;
@@ -51,26 +53,25 @@ public class TaskExecutorTestHelpers {
SubmitWorkRequestProto
requestProto = createSubmitWorkRequestProto(fragmentNum, parallelism,
startTime);
- QueryFragmentInfo queryFragmentInfo = createQueryFragmentInfo(requestProto.getFragmentSpec());
+ QueryFragmentInfo queryFragmentInfo = createQueryFragmentInfo(
+ requestProto.getWorkSpec().getVertex(), requestProto.getFragmentNumber());
MockRequest mockRequest = new MockRequest(requestProto, queryFragmentInfo, canFinish, workTime);
return mockRequest;
}
public static TaskExecutorService.TaskWrapper createTaskWrapper(
SubmitWorkRequestProto request, boolean canFinish, int workTime) {
- QueryFragmentInfo queryFragmentInfo = createQueryFragmentInfo(request.getFragmentSpec());
+ 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;
}
- public static QueryFragmentInfo createQueryFragmentInfo(FragmentSpecProto fragmentSpecProto) {
- QueryInfo queryInfo = createQueryInfo();
- QueryFragmentInfo fragmentInfo =
- new QueryFragmentInfo(queryInfo, "fakeVertexName", fragmentSpecProto.getFragmentNumber(), 0,
- fragmentSpecProto);
- return fragmentInfo;
+ public static QueryFragmentInfo createQueryFragmentInfo(
+ SignableVertexSpec vertex, int fragmentNum) {
+ return new QueryFragmentInfo(createQueryInfo(), "fakeVertexName", fragmentNum, 0, vertex, "");
}
public static QueryInfo createQueryInfo() {
@@ -100,20 +101,23 @@ public class TaskExecutorTestHelpers {
TezTaskAttemptID taId = TezTaskAttemptID.getInstance(tId, fragmentNumber);
return SubmitWorkRequestProto
.newBuilder()
- .setFragmentSpec(
- FragmentSpecProto
- .newBuilder()
- .setAttemptNumber(0)
+ .setAttemptNumber(0)
+ .setFragmentNumber(fragmentNumber)
+ .setWorkSpec(
+ VertexOrBinary.newBuilder().setVertex(
+ SignableVertexSpec.newBuilder()
.setDagName("MockDag")
- .setFragmentNumber(fragmentNumber)
+ .setUser("MockUser")
+ .setTokenIdentifier("MockToken_1")
+ .setVertexIdentifier(Converters.createVertexIdentifier(taId, 0))
.setVertexName("MockVertex")
.setProcessorDescriptor(
LlapDaemonProtocolProtos.EntityDescriptorProto.newBuilder()
.setClassName("MockProcessor").build())
- .setFragmentIdentifierString(taId.toString()).build()).setAmHost("localhost")
- .setAmPort(12345).setAppAttemptNumber(0).setApplicationIdString("MockApp_1")
- .setContainerIdString("MockContainer_1").setUser("MockUser")
- .setTokenIdentifier("MockToken_1")
+ .build()).build())
+ .setAmHost("localhost")
+ .setAmPort(12345)
+ .setContainerIdString("MockContainer_1")
.setFragmentRuntimeInfo(LlapDaemonProtocolProtos
.FragmentRuntimeInfo
.newBuilder()
@@ -146,7 +150,7 @@ public class TaskExecutorTestHelpers {
new ExecutionContextImpl("localhost"), null, new Credentials(), 0, null, null, mock(
LlapDaemonExecutorMetrics.class),
mock(KilledTaskHandler.class), mock(
- FragmentCompletionHandler.class), new DefaultHadoopShim());
+ FragmentCompletionHandler.class), new DefaultHadoopShim(), null);
this.workTime = workTime;
this.canFinish = canFinish;
}
http://git-wip-us.apache.org/repos/asf/hive/blob/0b5c27fd/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/comparator/TestFirstInFirstOutComparator.java
----------------------------------------------------------------------
diff --git a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/comparator/TestFirstInFirstOutComparator.java b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/comparator/TestFirstInFirstOutComparator.java
index 08ee769..a250882 100644
--- a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/comparator/TestFirstInFirstOutComparator.java
+++ b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/comparator/TestFirstInFirstOutComparator.java
@@ -31,8 +31,11 @@ import org.apache.hadoop.hive.llap.daemon.impl.TaskExecutorService.TaskWrapper;
import org.apache.hadoop.hive.llap.daemon.impl.TaskRunnerCallable;
import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos;
import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto;
-import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto;
+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.tez.Converters;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.tez.dag.records.TezDAGID;
@@ -59,7 +62,7 @@ public class TestFirstInFirstOutComparator {
super(requestProto, mock(QueryFragmentInfo.class), conf,
new ExecutionContextImpl("localhost"), null, cred, 0, null, null, null,
mock(KilledTaskHandler.class), mock(
- FragmentCompletionHandler.class), new DefaultHadoopShim());
+ FragmentCompletionHandler.class), new DefaultHadoopShim(), null);
this.workTime = workTime;
this.canFinish = canFinish;
}
@@ -102,19 +105,23 @@ public class TestFirstInFirstOutComparator {
TezTaskAttemptID taId = TezTaskAttemptID.getInstance(tId, fragmentNumber);
return SubmitWorkRequestProto
.newBuilder()
- .setFragmentSpec(
- FragmentSpecProto
+ .setAttemptNumber(0)
+ .setFragmentNumber(fragmentNumber)
+ .setWorkSpec(
+ VertexOrBinary.newBuilder().setVertex(
+ SignableVertexSpec
.newBuilder()
- .setAttemptNumber(0)
+ .setVertexIdentifier(Converters.createVertexIdentifier(taId, 0))
.setDagName("MockDag")
- .setFragmentNumber(fragmentNumber)
.setVertexName("MockVertex")
+ .setUser("MockUser")
+ .setTokenIdentifier("MockToken_1")
.setProcessorDescriptor(
EntityDescriptorProto.newBuilder().setClassName("MockProcessor").build())
- .setFragmentIdentifierString(taId.toString()).build()).setAmHost("localhost")
- .setAmPort(12345).setAppAttemptNumber(0).setApplicationIdString("MockApp_1")
- .setContainerIdString("MockContainer_1").setUser("MockUser")
- .setTokenIdentifier("MockToken_1")
+ .build()).build())
+ .setAmHost("localhost")
+ .setAmPort(12345)
+ .setContainerIdString("MockContainer_1")
.setFragmentRuntimeInfo(LlapDaemonProtocolProtos
.FragmentRuntimeInfo
.newBuilder()
http://git-wip-us.apache.org/repos/asf/hive/blob/0b5c27fd/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
----------------------------------------------------------------------
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 b4b041a..a3f2eb8 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
@@ -46,6 +46,7 @@ import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWor
import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWorkResponseProto;
import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.TerminateFragmentRequestProto;
import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.TerminateFragmentResponseProto;
+import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexOrBinary;
import org.apache.hadoop.hive.llap.protocol.LlapTaskUmbilicalProtocol;
import org.apache.hadoop.hive.llap.registry.ServiceInstance;
import org.apache.hadoop.hive.llap.registry.impl.LlapRegistryService;
@@ -89,10 +90,7 @@ public class LlapTaskCommunicator extends TezTaskCommunicatorImpl {
private static final Logger LOG = LoggerFactory.getLogger(LlapTaskCommunicator.class);
private static final boolean isInfoEnabled = LOG.isInfoEnabled();
- private static final boolean isDebugEnabed = LOG.isDebugEnabled();
-
- private final SubmitWorkRequestProto BASE_SUBMIT_WORK_REQUEST;
-
+
private final ConcurrentMap<QueryIdentifierProto, ByteBuffer> credentialMap;
// Tracks containerIds and taskAttemptIds, so can be kept independent of the running DAG.
@@ -105,6 +103,8 @@ public class LlapTaskCommunicator extends TezTaskCommunicatorImpl {
private long deleteDelayOnDagComplete;
private final LlapTaskUmbilicalProtocol umbilical;
private final Token<LlapTokenIdentifier> token;
+ private final int appAttemptId;
+ private final String user;
// These two structures track the list of known nodes, and the list of nodes which are sending in keep-alive heartbeats.
// Primarily for debugging purposes a.t.m, since there's some unexplained TASK_TIMEOUTS which are currently being observed.
@@ -113,8 +113,6 @@ public class LlapTaskCommunicator extends TezTaskCommunicatorImpl {
private final LlapRegistryService serviceRegistry;
-
- private volatile int currentDagId;
private volatile QueryIdentifierProto currentQueryIdentifierProto;
public LlapTaskCommunicator(
@@ -138,17 +136,10 @@ public class LlapTaskCommunicator extends TezTaskCommunicatorImpl {
serviceRegistry = LlapRegistryService.getClient(conf);
umbilical = new LlapTaskUmbilicalProtocolImpl(getUmbilical());
- SubmitWorkRequestProto.Builder baseBuilder = SubmitWorkRequestProto.newBuilder();
// TODO Avoid reading this from the environment
- baseBuilder.setUser(System.getenv(ApplicationConstants.Environment.USER.name()));
- baseBuilder.setApplicationIdString(
- taskCommunicatorContext.getApplicationAttemptId().getApplicationId().toString());
- baseBuilder
- .setAppAttemptNumber(taskCommunicatorContext.getApplicationAttemptId().getAttemptId());
- baseBuilder.setTokenIdentifier(getTokenIdentifier());
-
- BASE_SUBMIT_WORK_REQUEST = baseBuilder.build();
+ user = System.getenv(ApplicationConstants.Environment.USER.name());
+ appAttemptId = taskCommunicatorContext.getApplicationAttemptId().getAttemptId();
credentialMap = new ConcurrentHashMap<>();
sourceStateTracker = new SourceStateTracker(getContext(), this);
@@ -316,7 +307,6 @@ public class LlapTaskCommunicator extends TezTaskCommunicatorImpl {
t = se.getCause();
}
if (t instanceof RemoteException) {
- RemoteException re = (RemoteException) t;
// All others from the remote service cause the task to FAIL.
LOG.info(
"Failed to run task: " + taskSpec.getTaskAttemptID() + " on containerId: " +
@@ -591,8 +581,9 @@ public class LlapTaskCommunicator extends TezTaskCommunicatorImpl {
TaskSpec taskSpec,
FragmentRuntimeInfo fragmentRuntimeInfo) throws
IOException {
- SubmitWorkRequestProto.Builder builder =
- SubmitWorkRequestProto.newBuilder(BASE_SUBMIT_WORK_REQUEST);
+ SubmitWorkRequestProto.Builder builder = SubmitWorkRequestProto.newBuilder();
+ builder.setFragmentNumber(taskSpec.getTaskAttemptID().getTaskID().getId());
+ builder.setAttemptNumber(taskSpec.getTaskAttemptID().getId());
builder.setContainerIdString(containerId.toString());
builder.setAmHost(getAddress().getHostName());
builder.setAmPort(getAddress().getPort());
@@ -607,7 +598,9 @@ public class LlapTaskCommunicator extends TezTaskCommunicatorImpl {
credentialsBinary = credentialsBinary.duplicate();
}
builder.setCredentialsBinary(ByteString.copyFrom(credentialsBinary));
- builder.setFragmentSpec(Converters.convertTaskSpecToProto(taskSpec));
+ builder.setWorkSpec(VertexOrBinary.newBuilder().setVertex(Converters.convertTaskSpecToProto(
+ taskSpec, appAttemptId, getTokenIdentifier(), null, user)).build());
+ // Don't call builder.setWorkSpecSignature() - Tez doesn't sign fragments
builder.setFragmentRuntimeInfo(fragmentRuntimeInfo);
return builder.build();
}
[08/10] hive git commit: HIVE-13646 make
hive.optimize.sort.dynamic.partition compatible with ACID tables (Eugene
Koifman, reviewed by Wei Zheng)
Posted by jd...@apache.org.
HIVE-13646 make hive.optimize.sort.dynamic.partition compatible with ACID tables (Eugene Koifman, reviewed by Wei Zheng)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/87299662
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/87299662
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/87299662
Branch: refs/heads/llap
Commit: 8729966296a041b7ea952ba67f148d2c48c27749
Parents: 70fe310
Author: Eugene Koifman <ek...@hortonworks.com>
Authored: Tue May 3 17:11:47 2016 -0700
Committer: Eugene Koifman <ek...@hortonworks.com>
Committed: Tue May 3 17:11:47 2016 -0700
----------------------------------------------------------------------
.../hadoop/hive/ql/parse/SemanticAnalyzer.java | 1 -
.../dynpart_sort_optimization_acid.q.out | 120 +++++++++++++++----
2 files changed, 100 insertions(+), 21 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/87299662/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 06db7f9..2983d38 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -7030,7 +7030,6 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
conf.setBoolVar(ConfVars.HIVEOPTREDUCEDEDUPLICATION, true);
conf.setIntVar(ConfVars.HIVEOPTREDUCEDEDUPLICATIONMINREDUCER, 1);
conf.set(AcidUtils.CONF_ACID_KEY, "true");
- conf.setBoolVar(ConfVars.HIVEOPTSORTDYNAMICPARTITION, false);
if (table.getNumBuckets() < 1) {
throw new SemanticException(ErrorMsg.ACID_OP_ON_NONACID_TABLE, table.getTableName());
http://git-wip-us.apache.org/repos/asf/hive/blob/87299662/ql/src/test/results/clientpositive/dynpart_sort_optimization_acid.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/dynpart_sort_optimization_acid.q.out b/ql/src/test/results/clientpositive/dynpart_sort_optimization_acid.q.out
index eca29df..62399e3 100644
--- a/ql/src/test/results/clientpositive/dynpart_sort_optimization_acid.q.out
+++ b/ql/src/test/results/clientpositive/dynpart_sort_optimization_acid.q.out
@@ -380,8 +380,9 @@ POSTHOOK: query: explain update acid set value = 'bar' where key = 'foo' and ds
POSTHOOK: type: QUERY
STAGE DEPENDENCIES:
Stage-1 is a root stage
- Stage-0 depends on stages: Stage-1
- Stage-2 depends on stages: Stage-0
+ Stage-2 depends on stages: Stage-1
+ Stage-0 depends on stages: Stage-2
+ Stage-3 depends on stages: Stage-0
STAGE PLANS:
Stage: Stage-1
@@ -397,12 +398,31 @@ STAGE PLANS:
Reduce Output Operator
key expressions: _col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>)
sort order: +
- Map-reduce partition columns: UDFToInteger(_col0) (type: int)
value expressions: _col3 (type: string)
Reduce Operator Tree:
Select Operator
- expressions: KEY.reducesinkkey0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>), 'foo' (type: string), 'bar' (type: string), VALUE._col2 (type: string)
- outputColumnNames: _col0, _col1, _col2, _col3
+ expressions: KEY.reducesinkkey0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>), VALUE._col2 (type: string)
+ outputColumnNames: _col0, _col3
+ File Output Operator
+ compressed: false
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+ Stage: Stage-2
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ Reduce Output Operator
+ key expressions: _col3 (type: string), '_bucket_number' (type: string), _col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>)
+ sort order: +++
+ Map-reduce partition columns: _col3 (type: string)
+ value expressions: 'foo' (type: string), 'bar' (type: string)
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY._col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>), VALUE._col1 (type: string), VALUE._col2 (type: string), KEY._col3 (type: string), KEY.'_bucket_number' (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3, '_bucket_number'
File Output Operator
compressed: false
table:
@@ -423,7 +443,7 @@ STAGE PLANS:
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid
- Stage: Stage-2
+ Stage: Stage-3
Stats-Aggr Operator
PREHOOK: query: update acid set value = 'bar' where key = 'foo' and ds in ('2008-04-08')
@@ -875,8 +895,9 @@ POSTHOOK: query: explain update acid set value = 'bar' where key = 'foo' and ds=
POSTHOOK: type: QUERY
STAGE DEPENDENCIES:
Stage-1 is a root stage
- Stage-0 depends on stages: Stage-1
- Stage-2 depends on stages: Stage-0
+ Stage-2 depends on stages: Stage-1
+ Stage-0 depends on stages: Stage-2
+ Stage-3 depends on stages: Stage-0
STAGE PLANS:
Stage: Stage-1
@@ -892,12 +913,31 @@ STAGE PLANS:
Reduce Output Operator
key expressions: _col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>)
sort order: +
- Map-reduce partition columns: UDFToInteger(_col0) (type: int)
value expressions: _col4 (type: int)
Reduce Operator Tree:
Select Operator
- expressions: KEY.reducesinkkey0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>), 'foo' (type: string), 'bar' (type: string), '2008-04-08' (type: string), VALUE._col3 (type: int)
- outputColumnNames: _col0, _col1, _col2, _col3, _col4
+ expressions: KEY.reducesinkkey0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>), VALUE._col3 (type: int)
+ outputColumnNames: _col0, _col4
+ File Output Operator
+ compressed: false
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+ Stage: Stage-2
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ Reduce Output Operator
+ key expressions: '2008-04-08' (type: string), _col4 (type: int), '_bucket_number' (type: string), _col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>)
+ sort order: ++++
+ Map-reduce partition columns: '2008-04-08' (type: string), _col4 (type: int)
+ value expressions: 'foo' (type: string), 'bar' (type: string)
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY._col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>), VALUE._col1 (type: string), VALUE._col2 (type: string), KEY._col3 (type: string), KEY._col4 (type: int), KEY.'_bucket_number' (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, '_bucket_number'
File Output Operator
compressed: false
table:
@@ -919,7 +959,7 @@ STAGE PLANS:
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid
- Stage: Stage-2
+ Stage: Stage-3
Stats-Aggr Operator
PREHOOK: query: update acid set value = 'bar' where key = 'foo' and ds='2008-04-08' and hr>=11
@@ -1053,8 +1093,9 @@ POSTHOOK: query: explain update acid set value = 'bar' where key = 'foo' and ds=
POSTHOOK: type: QUERY
STAGE DEPENDENCIES:
Stage-1 is a root stage
- Stage-0 depends on stages: Stage-1
- Stage-2 depends on stages: Stage-0
+ Stage-2 depends on stages: Stage-1
+ Stage-0 depends on stages: Stage-2
+ Stage-3 depends on stages: Stage-0
STAGE PLANS:
Stage: Stage-1
@@ -1070,7 +1111,6 @@ STAGE PLANS:
Reduce Output Operator
key expressions: _col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>)
sort order: +
- Map-reduce partition columns: UDFToInteger(_col0) (type: int)
value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: int)
Reduce Operator Tree:
Select Operator
@@ -1079,6 +1119,26 @@ STAGE PLANS:
File Output Operator
compressed: false
table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+ Stage: Stage-2
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ Reduce Output Operator
+ key expressions: _col3 (type: string), _col4 (type: int), '_bucket_number' (type: string), _col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>)
+ sort order: ++++
+ Map-reduce partition columns: _col3 (type: string), _col4 (type: int)
+ value expressions: _col1 (type: string), _col2 (type: string)
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY._col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>), VALUE._col1 (type: string), VALUE._col2 (type: string), KEY._col3 (type: string), KEY._col4 (type: int), KEY.'_bucket_number' (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, '_bucket_number'
+ File Output Operator
+ compressed: false
+ table:
input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
@@ -1097,7 +1157,7 @@ STAGE PLANS:
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid
- Stage: Stage-2
+ Stage: Stage-3
Stats-Aggr Operator
PREHOOK: query: update acid set value = 'bar' where key = 'foo' and ds='2008-04-08' and hr=11
@@ -1127,8 +1187,9 @@ POSTHOOK: query: explain update acid set value = 'bar' where key = 'foo' and ds=
POSTHOOK: type: QUERY
STAGE DEPENDENCIES:
Stage-1 is a root stage
- Stage-0 depends on stages: Stage-1
- Stage-2 depends on stages: Stage-0
+ Stage-2 depends on stages: Stage-1
+ Stage-0 depends on stages: Stage-2
+ Stage-3 depends on stages: Stage-0
STAGE PLANS:
Stage: Stage-1
@@ -1144,7 +1205,6 @@ STAGE PLANS:
Reduce Output Operator
key expressions: _col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>)
sort order: +
- Map-reduce partition columns: UDFToInteger(_col0) (type: int)
value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: int)
Reduce Operator Tree:
Select Operator
@@ -1153,6 +1213,26 @@ STAGE PLANS:
File Output Operator
compressed: false
table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+ Stage: Stage-2
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ Reduce Output Operator
+ key expressions: _col3 (type: string), _col4 (type: int), '_bucket_number' (type: string), _col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>)
+ sort order: ++++
+ Map-reduce partition columns: _col3 (type: string), _col4 (type: int)
+ value expressions: _col1 (type: string), _col2 (type: string)
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY._col0 (type: struct<transactionid:bigint,bucketid:int,rowid:bigint>), VALUE._col1 (type: string), VALUE._col2 (type: string), KEY._col3 (type: string), KEY._col4 (type: int), KEY.'_bucket_number' (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, '_bucket_number'
+ File Output Operator
+ compressed: false
+ table:
input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
@@ -1171,7 +1251,7 @@ STAGE PLANS:
serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
name: default.acid
- Stage: Stage-2
+ Stage: Stage-3
Stats-Aggr Operator
PREHOOK: query: update acid set value = 'bar' where key = 'foo' and ds='2008-04-08' and hr>=11
[05/10] hive git commit: HIVE-13442 : LLAP: refactor submit API to be
amenable to signing (Sergey Shelukhin, reviewed by Siddharth Seth)
Posted by jd...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/0b5c27fd/llap-common/src/gen/protobuf/gen-java/org/apache/hadoop/hive/llap/daemon/rpc/LlapDaemonProtocolProtos.java
----------------------------------------------------------------------
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 820f6be..16531e3 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
@@ -3212,202 +3212,73 @@ public final class LlapDaemonProtocolProtos {
// @@protoc_insertion_point(class_scope:GroupInputSpecProto)
}
- public interface FragmentSpecProtoOrBuilder
+ public interface VertexIdentifierOrBuilder
extends com.google.protobuf.MessageOrBuilder {
- // optional string fragment_identifier_string = 1;
+ // optional string application_id_string = 1;
/**
- * <code>optional string fragment_identifier_string = 1;</code>
+ * <code>optional string application_id_string = 1;</code>
*/
- boolean hasFragmentIdentifierString();
+ boolean hasApplicationIdString();
/**
- * <code>optional string fragment_identifier_string = 1;</code>
+ * <code>optional string application_id_string = 1;</code>
*/
- java.lang.String getFragmentIdentifierString();
+ java.lang.String getApplicationIdString();
/**
- * <code>optional string fragment_identifier_string = 1;</code>
+ * <code>optional string application_id_string = 1;</code>
*/
com.google.protobuf.ByteString
- getFragmentIdentifierStringBytes();
+ getApplicationIdStringBytes();
- // optional string dag_name = 2;
- /**
- * <code>optional string dag_name = 2;</code>
- */
- boolean hasDagName();
+ // optional int32 app_attempt_number = 2;
/**
- * <code>optional string dag_name = 2;</code>
+ * <code>optional int32 app_attempt_number = 2;</code>
*/
- java.lang.String getDagName();
+ boolean hasAppAttemptNumber();
/**
- * <code>optional string dag_name = 2;</code>
+ * <code>optional int32 app_attempt_number = 2;</code>
*/
- com.google.protobuf.ByteString
- getDagNameBytes();
+ int getAppAttemptNumber();
- // optional int32 dag_id = 11;
+ // optional int32 dag_id = 3;
/**
- * <code>optional int32 dag_id = 11;</code>
+ * <code>optional int32 dag_id = 3;</code>
*/
boolean hasDagId();
/**
- * <code>optional int32 dag_id = 11;</code>
+ * <code>optional int32 dag_id = 3;</code>
*/
int getDagId();
- // optional string vertex_name = 3;
- /**
- * <code>optional string vertex_name = 3;</code>
- */
- boolean hasVertexName();
- /**
- * <code>optional string vertex_name = 3;</code>
- */
- java.lang.String getVertexName();
- /**
- * <code>optional string vertex_name = 3;</code>
- */
- com.google.protobuf.ByteString
- getVertexNameBytes();
-
- // optional .EntityDescriptorProto processor_descriptor = 4;
- /**
- * <code>optional .EntityDescriptorProto processor_descriptor = 4;</code>
- */
- boolean hasProcessorDescriptor();
- /**
- * <code>optional .EntityDescriptorProto processor_descriptor = 4;</code>
- */
- org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto getProcessorDescriptor();
- /**
- * <code>optional .EntityDescriptorProto processor_descriptor = 4;</code>
- */
- org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProtoOrBuilder getProcessorDescriptorOrBuilder();
-
- // repeated .IOSpecProto input_specs = 5;
- /**
- * <code>repeated .IOSpecProto input_specs = 5;</code>
- */
- java.util.List<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto>
- getInputSpecsList();
- /**
- * <code>repeated .IOSpecProto input_specs = 5;</code>
- */
- org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto getInputSpecs(int index);
- /**
- * <code>repeated .IOSpecProto input_specs = 5;</code>
- */
- int getInputSpecsCount();
- /**
- * <code>repeated .IOSpecProto input_specs = 5;</code>
- */
- java.util.List<? extends org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProtoOrBuilder>
- getInputSpecsOrBuilderList();
- /**
- * <code>repeated .IOSpecProto input_specs = 5;</code>
- */
- org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProtoOrBuilder getInputSpecsOrBuilder(
- int index);
-
- // repeated .IOSpecProto output_specs = 6;
- /**
- * <code>repeated .IOSpecProto output_specs = 6;</code>
- */
- java.util.List<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto>
- getOutputSpecsList();
- /**
- * <code>repeated .IOSpecProto output_specs = 6;</code>
- */
- org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto getOutputSpecs(int index);
- /**
- * <code>repeated .IOSpecProto output_specs = 6;</code>
- */
- int getOutputSpecsCount();
- /**
- * <code>repeated .IOSpecProto output_specs = 6;</code>
- */
- java.util.List<? extends org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProtoOrBuilder>
- getOutputSpecsOrBuilderList();
- /**
- * <code>repeated .IOSpecProto output_specs = 6;</code>
- */
- org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProtoOrBuilder getOutputSpecsOrBuilder(
- int index);
-
- // repeated .GroupInputSpecProto grouped_input_specs = 7;
- /**
- * <code>repeated .GroupInputSpecProto grouped_input_specs = 7;</code>
- */
- java.util.List<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProto>
- getGroupedInputSpecsList();
- /**
- * <code>repeated .GroupInputSpecProto grouped_input_specs = 7;</code>
- */
- org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProto getGroupedInputSpecs(int index);
- /**
- * <code>repeated .GroupInputSpecProto grouped_input_specs = 7;</code>
- */
- int getGroupedInputSpecsCount();
- /**
- * <code>repeated .GroupInputSpecProto grouped_input_specs = 7;</code>
- */
- java.util.List<? extends org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProtoOrBuilder>
- getGroupedInputSpecsOrBuilderList();
- /**
- * <code>repeated .GroupInputSpecProto grouped_input_specs = 7;</code>
- */
- org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProtoOrBuilder getGroupedInputSpecsOrBuilder(
- int index);
-
- // optional int32 vertex_parallelism = 8;
- /**
- * <code>optional int32 vertex_parallelism = 8;</code>
- */
- boolean hasVertexParallelism();
- /**
- * <code>optional int32 vertex_parallelism = 8;</code>
- */
- int getVertexParallelism();
-
- // optional int32 fragment_number = 9;
- /**
- * <code>optional int32 fragment_number = 9;</code>
- */
- boolean hasFragmentNumber();
- /**
- * <code>optional int32 fragment_number = 9;</code>
- */
- int getFragmentNumber();
-
- // optional int32 attempt_number = 10;
+ // optional int32 vertex_id = 4;
/**
- * <code>optional int32 attempt_number = 10;</code>
+ * <code>optional int32 vertex_id = 4;</code>
*/
- boolean hasAttemptNumber();
+ boolean hasVertexId();
/**
- * <code>optional int32 attempt_number = 10;</code>
+ * <code>optional int32 vertex_id = 4;</code>
*/
- int getAttemptNumber();
+ int getVertexId();
}
/**
- * Protobuf type {@code FragmentSpecProto}
+ * Protobuf type {@code VertexIdentifier}
*/
- public static final class FragmentSpecProto extends
+ public static final class VertexIdentifier extends
com.google.protobuf.GeneratedMessage
- implements FragmentSpecProtoOrBuilder {
- // Use FragmentSpecProto.newBuilder() to construct.
- private FragmentSpecProto(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+ implements VertexIdentifierOrBuilder {
+ // Use VertexIdentifier.newBuilder() to construct.
+ private VertexIdentifier(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
super(builder);
this.unknownFields = builder.getUnknownFields();
}
- private FragmentSpecProto(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+ private VertexIdentifier(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
- private static final FragmentSpecProto defaultInstance;
- public static FragmentSpecProto getDefaultInstance() {
+ private static final VertexIdentifier defaultInstance;
+ public static VertexIdentifier getDefaultInstance() {
return defaultInstance;
}
- public FragmentSpecProto getDefaultInstanceForType() {
+ public VertexIdentifier getDefaultInstanceForType() {
return defaultInstance;
}
@@ -3417,7 +3288,7 @@ public final class LlapDaemonProtocolProtos {
getUnknownFields() {
return this.unknownFields;
}
- private FragmentSpecProto(
+ private VertexIdentifier(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException {
@@ -3442,76 +3313,24 @@ public final class LlapDaemonProtocolProtos {
}
case 10: {
bitField0_ |= 0x00000001;
- fragmentIdentifierString_ = input.readBytes();
+ applicationIdString_ = input.readBytes();
break;
}
- case 18: {
+ case 16: {
bitField0_ |= 0x00000002;
- dagName_ = input.readBytes();
- break;
- }
- case 26: {
- bitField0_ |= 0x00000008;
- vertexName_ = input.readBytes();
- break;
- }
- case 34: {
- org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.Builder subBuilder = null;
- if (((bitField0_ & 0x00000010) == 0x00000010)) {
- subBuilder = processorDescriptor_.toBuilder();
- }
- processorDescriptor_ = input.readMessage(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.PARSER, extensionRegistry);
- if (subBuilder != null) {
- subBuilder.mergeFrom(processorDescriptor_);
- processorDescriptor_ = subBuilder.buildPartial();
- }
- bitField0_ |= 0x00000010;
- break;
- }
- case 42: {
- if (!((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
- inputSpecs_ = new java.util.ArrayList<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto>();
- mutable_bitField0_ |= 0x00000020;
- }
- inputSpecs_.add(input.readMessage(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto.PARSER, extensionRegistry));
- break;
- }
- case 50: {
- if (!((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
- outputSpecs_ = new java.util.ArrayList<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto>();
- mutable_bitField0_ |= 0x00000040;
- }
- outputSpecs_.add(input.readMessage(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto.PARSER, extensionRegistry));
- break;
- }
- case 58: {
- if (!((mutable_bitField0_ & 0x00000080) == 0x00000080)) {
- groupedInputSpecs_ = new java.util.ArrayList<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProto>();
- mutable_bitField0_ |= 0x00000080;
- }
- groupedInputSpecs_.add(input.readMessage(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProto.PARSER, extensionRegistry));
- break;
- }
- case 64: {
- bitField0_ |= 0x00000020;
- vertexParallelism_ = input.readInt32();
- break;
- }
- case 72: {
- bitField0_ |= 0x00000040;
- fragmentNumber_ = input.readInt32();
- break;
- }
- case 80: {
- bitField0_ |= 0x00000080;
- attemptNumber_ = input.readInt32();
+ appAttemptNumber_ = input.readInt32();
break;
}
- case 88: {
+ case 24: {
bitField0_ |= 0x00000004;
dagId_ = input.readInt32();
break;
}
+ case 32: {
+ bitField0_ |= 0x00000008;
+ vertexId_ = input.readInt32();
+ break;
+ }
}
}
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -3520,61 +3339,52 @@ public final class LlapDaemonProtocolProtos {
throw new com.google.protobuf.InvalidProtocolBufferException(
e.getMessage()).setUnfinishedMessage(this);
} finally {
- if (((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
- inputSpecs_ = java.util.Collections.unmodifiableList(inputSpecs_);
- }
- if (((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
- outputSpecs_ = java.util.Collections.unmodifiableList(outputSpecs_);
- }
- if (((mutable_bitField0_ & 0x00000080) == 0x00000080)) {
- groupedInputSpecs_ = java.util.Collections.unmodifiableList(groupedInputSpecs_);
- }
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_FragmentSpecProto_descriptor;
+ return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_VertexIdentifier_descriptor;
}
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
- return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_FragmentSpecProto_fieldAccessorTable
+ return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_VertexIdentifier_fieldAccessorTable
.ensureFieldAccessorsInitialized(
- org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto.class, org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto.Builder.class);
+ org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier.class, org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier.Builder.class);
}
- public static com.google.protobuf.Parser<FragmentSpecProto> PARSER =
- new com.google.protobuf.AbstractParser<FragmentSpecProto>() {
- public FragmentSpecProto parsePartialFrom(
+ public static com.google.protobuf.Parser<VertexIdentifier> PARSER =
+ new com.google.protobuf.AbstractParser<VertexIdentifier>() {
+ public VertexIdentifier parsePartialFrom(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException {
- return new FragmentSpecProto(input, extensionRegistry);
+ return new VertexIdentifier(input, extensionRegistry);
}
};
@java.lang.Override
- public com.google.protobuf.Parser<FragmentSpecProto> getParserForType() {
+ public com.google.protobuf.Parser<VertexIdentifier> getParserForType() {
return PARSER;
}
private int bitField0_;
- // optional string fragment_identifier_string = 1;
- public static final int FRAGMENT_IDENTIFIER_STRING_FIELD_NUMBER = 1;
- private java.lang.Object fragmentIdentifierString_;
+ // optional string application_id_string = 1;
+ public static final int APPLICATION_ID_STRING_FIELD_NUMBER = 1;
+ private java.lang.Object applicationIdString_;
/**
- * <code>optional string fragment_identifier_string = 1;</code>
+ * <code>optional string application_id_string = 1;</code>
*/
- public boolean hasFragmentIdentifierString() {
+ public boolean hasApplicationIdString() {
return ((bitField0_ & 0x00000001) == 0x00000001);
}
/**
- * <code>optional string fragment_identifier_string = 1;</code>
+ * <code>optional string application_id_string = 1;</code>
*/
- public java.lang.String getFragmentIdentifierString() {
- java.lang.Object ref = fragmentIdentifierString_;
+ public java.lang.String getApplicationIdString() {
+ java.lang.Object ref = applicationIdString_;
if (ref instanceof java.lang.String) {
return (java.lang.String) ref;
} else {
@@ -3582,320 +3392,81 @@ public final class LlapDaemonProtocolProtos {
(com.google.protobuf.ByteString) ref;
java.lang.String s = bs.toStringUtf8();
if (bs.isValidUtf8()) {
- fragmentIdentifierString_ = s;
+ applicationIdString_ = s;
}
return s;
}
}
/**
- * <code>optional string fragment_identifier_string = 1;</code>
+ * <code>optional string application_id_string = 1;</code>
*/
public com.google.protobuf.ByteString
- getFragmentIdentifierStringBytes() {
- java.lang.Object ref = fragmentIdentifierString_;
+ getApplicationIdStringBytes() {
+ java.lang.Object ref = applicationIdString_;
if (ref instanceof java.lang.String) {
com.google.protobuf.ByteString b =
com.google.protobuf.ByteString.copyFromUtf8(
(java.lang.String) ref);
- fragmentIdentifierString_ = b;
+ applicationIdString_ = b;
return b;
} else {
return (com.google.protobuf.ByteString) ref;
}
}
- // optional string dag_name = 2;
- public static final int DAG_NAME_FIELD_NUMBER = 2;
- private java.lang.Object dagName_;
+ // optional int32 app_attempt_number = 2;
+ public static final int APP_ATTEMPT_NUMBER_FIELD_NUMBER = 2;
+ private int appAttemptNumber_;
/**
- * <code>optional string dag_name = 2;</code>
+ * <code>optional int32 app_attempt_number = 2;</code>
*/
- public boolean hasDagName() {
+ public boolean hasAppAttemptNumber() {
return ((bitField0_ & 0x00000002) == 0x00000002);
}
/**
- * <code>optional string dag_name = 2;</code>
- */
- public java.lang.String getDagName() {
- java.lang.Object ref = dagName_;
- 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()) {
- dagName_ = s;
- }
- return s;
- }
- }
- /**
- * <code>optional string dag_name = 2;</code>
+ * <code>optional int32 app_attempt_number = 2;</code>
*/
- public com.google.protobuf.ByteString
- getDagNameBytes() {
- java.lang.Object ref = dagName_;
- if (ref instanceof java.lang.String) {
- com.google.protobuf.ByteString b =
- com.google.protobuf.ByteString.copyFromUtf8(
- (java.lang.String) ref);
- dagName_ = b;
- return b;
- } else {
- return (com.google.protobuf.ByteString) ref;
- }
+ public int getAppAttemptNumber() {
+ return appAttemptNumber_;
}
- // optional int32 dag_id = 11;
- public static final int DAG_ID_FIELD_NUMBER = 11;
+ // optional int32 dag_id = 3;
+ public static final int DAG_ID_FIELD_NUMBER = 3;
private int dagId_;
/**
- * <code>optional int32 dag_id = 11;</code>
+ * <code>optional int32 dag_id = 3;</code>
*/
public boolean hasDagId() {
return ((bitField0_ & 0x00000004) == 0x00000004);
}
/**
- * <code>optional int32 dag_id = 11;</code>
+ * <code>optional int32 dag_id = 3;</code>
*/
public int getDagId() {
return dagId_;
}
- // optional string vertex_name = 3;
- public static final int VERTEX_NAME_FIELD_NUMBER = 3;
- private java.lang.Object vertexName_;
+ // optional int32 vertex_id = 4;
+ public static final int VERTEX_ID_FIELD_NUMBER = 4;
+ private int vertexId_;
/**
- * <code>optional string vertex_name = 3;</code>
+ * <code>optional int32 vertex_id = 4;</code>
*/
- public boolean hasVertexName() {
+ public boolean hasVertexId() {
return ((bitField0_ & 0x00000008) == 0x00000008);
}
/**
- * <code>optional string vertex_name = 3;</code>
- */
- 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;
- }
- }
- /**
- * <code>optional string vertex_name = 3;</code>
+ * <code>optional int32 vertex_id = 4;</code>
*/
- 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;
- }
- }
-
- // optional .EntityDescriptorProto processor_descriptor = 4;
- public static final int PROCESSOR_DESCRIPTOR_FIELD_NUMBER = 4;
- private org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto processorDescriptor_;
- /**
- * <code>optional .EntityDescriptorProto processor_descriptor = 4;</code>
- */
- public boolean hasProcessorDescriptor() {
- return ((bitField0_ & 0x00000010) == 0x00000010);
- }
- /**
- * <code>optional .EntityDescriptorProto processor_descriptor = 4;</code>
- */
- public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto getProcessorDescriptor() {
- return processorDescriptor_;
- }
- /**
- * <code>optional .EntityDescriptorProto processor_descriptor = 4;</code>
- */
- public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProtoOrBuilder getProcessorDescriptorOrBuilder() {
- return processorDescriptor_;
- }
-
- // repeated .IOSpecProto input_specs = 5;
- public static final int INPUT_SPECS_FIELD_NUMBER = 5;
- private java.util.List<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto> inputSpecs_;
- /**
- * <code>repeated .IOSpecProto input_specs = 5;</code>
- */
- public java.util.List<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto> getInputSpecsList() {
- return inputSpecs_;
- }
- /**
- * <code>repeated .IOSpecProto input_specs = 5;</code>
- */
- public java.util.List<? extends org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProtoOrBuilder>
- getInputSpecsOrBuilderList() {
- return inputSpecs_;
- }
- /**
- * <code>repeated .IOSpecProto input_specs = 5;</code>
- */
- public int getInputSpecsCount() {
- return inputSpecs_.size();
- }
- /**
- * <code>repeated .IOSpecProto input_specs = 5;</code>
- */
- public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto getInputSpecs(int index) {
- return inputSpecs_.get(index);
- }
- /**
- * <code>repeated .IOSpecProto input_specs = 5;</code>
- */
- public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProtoOrBuilder getInputSpecsOrBuilder(
- int index) {
- return inputSpecs_.get(index);
- }
-
- // repeated .IOSpecProto output_specs = 6;
- public static final int OUTPUT_SPECS_FIELD_NUMBER = 6;
- private java.util.List<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto> outputSpecs_;
- /**
- * <code>repeated .IOSpecProto output_specs = 6;</code>
- */
- public java.util.List<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto> getOutputSpecsList() {
- return outputSpecs_;
- }
- /**
- * <code>repeated .IOSpecProto output_specs = 6;</code>
- */
- public java.util.List<? extends org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProtoOrBuilder>
- getOutputSpecsOrBuilderList() {
- return outputSpecs_;
- }
- /**
- * <code>repeated .IOSpecProto output_specs = 6;</code>
- */
- public int getOutputSpecsCount() {
- return outputSpecs_.size();
- }
- /**
- * <code>repeated .IOSpecProto output_specs = 6;</code>
- */
- public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto getOutputSpecs(int index) {
- return outputSpecs_.get(index);
- }
- /**
- * <code>repeated .IOSpecProto output_specs = 6;</code>
- */
- public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProtoOrBuilder getOutputSpecsOrBuilder(
- int index) {
- return outputSpecs_.get(index);
- }
-
- // repeated .GroupInputSpecProto grouped_input_specs = 7;
- public static final int GROUPED_INPUT_SPECS_FIELD_NUMBER = 7;
- private java.util.List<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProto> groupedInputSpecs_;
- /**
- * <code>repeated .GroupInputSpecProto grouped_input_specs = 7;</code>
- */
- public java.util.List<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProto> getGroupedInputSpecsList() {
- return groupedInputSpecs_;
- }
- /**
- * <code>repeated .GroupInputSpecProto grouped_input_specs = 7;</code>
- */
- public java.util.List<? extends org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProtoOrBuilder>
- getGroupedInputSpecsOrBuilderList() {
- return groupedInputSpecs_;
- }
- /**
- * <code>repeated .GroupInputSpecProto grouped_input_specs = 7;</code>
- */
- public int getGroupedInputSpecsCount() {
- return groupedInputSpecs_.size();
- }
- /**
- * <code>repeated .GroupInputSpecProto grouped_input_specs = 7;</code>
- */
- public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProto getGroupedInputSpecs(int index) {
- return groupedInputSpecs_.get(index);
- }
- /**
- * <code>repeated .GroupInputSpecProto grouped_input_specs = 7;</code>
- */
- public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProtoOrBuilder getGroupedInputSpecsOrBuilder(
- int index) {
- return groupedInputSpecs_.get(index);
- }
-
- // optional int32 vertex_parallelism = 8;
- public static final int VERTEX_PARALLELISM_FIELD_NUMBER = 8;
- private int vertexParallelism_;
- /**
- * <code>optional int32 vertex_parallelism = 8;</code>
- */
- public boolean hasVertexParallelism() {
- return ((bitField0_ & 0x00000020) == 0x00000020);
- }
- /**
- * <code>optional int32 vertex_parallelism = 8;</code>
- */
- public int getVertexParallelism() {
- return vertexParallelism_;
- }
-
- // optional int32 fragment_number = 9;
- public static final int FRAGMENT_NUMBER_FIELD_NUMBER = 9;
- private int fragmentNumber_;
- /**
- * <code>optional int32 fragment_number = 9;</code>
- */
- public boolean hasFragmentNumber() {
- return ((bitField0_ & 0x00000040) == 0x00000040);
- }
- /**
- * <code>optional int32 fragment_number = 9;</code>
- */
- public int getFragmentNumber() {
- return fragmentNumber_;
- }
-
- // optional int32 attempt_number = 10;
- public static final int ATTEMPT_NUMBER_FIELD_NUMBER = 10;
- private int attemptNumber_;
- /**
- * <code>optional int32 attempt_number = 10;</code>
- */
- public boolean hasAttemptNumber() {
- return ((bitField0_ & 0x00000080) == 0x00000080);
- }
- /**
- * <code>optional int32 attempt_number = 10;</code>
- */
- public int getAttemptNumber() {
- return attemptNumber_;
+ public int getVertexId() {
+ return vertexId_;
}
private void initFields() {
- fragmentIdentifierString_ = "";
- dagName_ = "";
+ applicationIdString_ = "";
+ appAttemptNumber_ = 0;
dagId_ = 0;
- vertexName_ = "";
- processorDescriptor_ = org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.getDefaultInstance();
- inputSpecs_ = java.util.Collections.emptyList();
- outputSpecs_ = java.util.Collections.emptyList();
- groupedInputSpecs_ = java.util.Collections.emptyList();
- vertexParallelism_ = 0;
- fragmentNumber_ = 0;
- attemptNumber_ = 0;
+ vertexId_ = 0;
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
@@ -3910,37 +3481,16 @@ public final class LlapDaemonProtocolProtos {
throws java.io.IOException {
getSerializedSize();
if (((bitField0_ & 0x00000001) == 0x00000001)) {
- output.writeBytes(1, getFragmentIdentifierStringBytes());
+ output.writeBytes(1, getApplicationIdStringBytes());
}
if (((bitField0_ & 0x00000002) == 0x00000002)) {
- output.writeBytes(2, getDagNameBytes());
- }
- if (((bitField0_ & 0x00000008) == 0x00000008)) {
- output.writeBytes(3, getVertexNameBytes());
- }
- if (((bitField0_ & 0x00000010) == 0x00000010)) {
- output.writeMessage(4, processorDescriptor_);
- }
- for (int i = 0; i < inputSpecs_.size(); i++) {
- output.writeMessage(5, inputSpecs_.get(i));
- }
- for (int i = 0; i < outputSpecs_.size(); i++) {
- output.writeMessage(6, outputSpecs_.get(i));
- }
- for (int i = 0; i < groupedInputSpecs_.size(); i++) {
- output.writeMessage(7, groupedInputSpecs_.get(i));
- }
- if (((bitField0_ & 0x00000020) == 0x00000020)) {
- output.writeInt32(8, vertexParallelism_);
- }
- if (((bitField0_ & 0x00000040) == 0x00000040)) {
- output.writeInt32(9, fragmentNumber_);
- }
- if (((bitField0_ & 0x00000080) == 0x00000080)) {
- output.writeInt32(10, attemptNumber_);
+ output.writeInt32(2, appAttemptNumber_);
}
if (((bitField0_ & 0x00000004) == 0x00000004)) {
- output.writeInt32(11, dagId_);
+ output.writeInt32(3, dagId_);
+ }
+ if (((bitField0_ & 0x00000008) == 0x00000008)) {
+ output.writeInt32(4, vertexId_);
}
getUnknownFields().writeTo(output);
}
@@ -3953,47 +3503,19 @@ public final class LlapDaemonProtocolProtos {
size = 0;
if (((bitField0_ & 0x00000001) == 0x00000001)) {
size += com.google.protobuf.CodedOutputStream
- .computeBytesSize(1, getFragmentIdentifierStringBytes());
+ .computeBytesSize(1, getApplicationIdStringBytes());
}
if (((bitField0_ & 0x00000002) == 0x00000002)) {
size += com.google.protobuf.CodedOutputStream
- .computeBytesSize(2, getDagNameBytes());
- }
- if (((bitField0_ & 0x00000008) == 0x00000008)) {
- size += com.google.protobuf.CodedOutputStream
- .computeBytesSize(3, getVertexNameBytes());
- }
- if (((bitField0_ & 0x00000010) == 0x00000010)) {
- size += com.google.protobuf.CodedOutputStream
- .computeMessageSize(4, processorDescriptor_);
+ .computeInt32Size(2, appAttemptNumber_);
}
- for (int i = 0; i < inputSpecs_.size(); i++) {
- size += com.google.protobuf.CodedOutputStream
- .computeMessageSize(5, inputSpecs_.get(i));
- }
- for (int i = 0; i < outputSpecs_.size(); i++) {
- size += com.google.protobuf.CodedOutputStream
- .computeMessageSize(6, outputSpecs_.get(i));
- }
- for (int i = 0; i < groupedInputSpecs_.size(); i++) {
- size += com.google.protobuf.CodedOutputStream
- .computeMessageSize(7, groupedInputSpecs_.get(i));
- }
- if (((bitField0_ & 0x00000020) == 0x00000020)) {
- size += com.google.protobuf.CodedOutputStream
- .computeInt32Size(8, vertexParallelism_);
- }
- if (((bitField0_ & 0x00000040) == 0x00000040)) {
- size += com.google.protobuf.CodedOutputStream
- .computeInt32Size(9, fragmentNumber_);
- }
- if (((bitField0_ & 0x00000080) == 0x00000080)) {
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
size += com.google.protobuf.CodedOutputStream
- .computeInt32Size(10, attemptNumber_);
+ .computeInt32Size(3, dagId_);
}
- if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ if (((bitField0_ & 0x00000008) == 0x00000008)) {
size += com.google.protobuf.CodedOutputStream
- .computeInt32Size(11, dagId_);
+ .computeInt32Size(4, vertexId_);
}
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
@@ -4012,57 +3534,31 @@ public final class LlapDaemonProtocolProtos {
if (obj == this) {
return true;
}
- if (!(obj instanceof org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto)) {
+ if (!(obj instanceof org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier)) {
return super.equals(obj);
}
- org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto other = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto) obj;
+ org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier other = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier) obj;
boolean result = true;
- result = result && (hasFragmentIdentifierString() == other.hasFragmentIdentifierString());
- if (hasFragmentIdentifierString()) {
- result = result && getFragmentIdentifierString()
- .equals(other.getFragmentIdentifierString());
+ result = result && (hasApplicationIdString() == other.hasApplicationIdString());
+ if (hasApplicationIdString()) {
+ result = result && getApplicationIdString()
+ .equals(other.getApplicationIdString());
}
- result = result && (hasDagName() == other.hasDagName());
- if (hasDagName()) {
- result = result && getDagName()
- .equals(other.getDagName());
+ result = result && (hasAppAttemptNumber() == other.hasAppAttemptNumber());
+ if (hasAppAttemptNumber()) {
+ result = result && (getAppAttemptNumber()
+ == other.getAppAttemptNumber());
}
result = result && (hasDagId() == other.hasDagId());
if (hasDagId()) {
result = result && (getDagId()
== other.getDagId());
}
- result = result && (hasVertexName() == other.hasVertexName());
- if (hasVertexName()) {
- result = result && getVertexName()
- .equals(other.getVertexName());
- }
- result = result && (hasProcessorDescriptor() == other.hasProcessorDescriptor());
- if (hasProcessorDescriptor()) {
- result = result && getProcessorDescriptor()
- .equals(other.getProcessorDescriptor());
- }
- result = result && getInputSpecsList()
- .equals(other.getInputSpecsList());
- result = result && getOutputSpecsList()
- .equals(other.getOutputSpecsList());
- result = result && getGroupedInputSpecsList()
- .equals(other.getGroupedInputSpecsList());
- result = result && (hasVertexParallelism() == other.hasVertexParallelism());
- if (hasVertexParallelism()) {
- result = result && (getVertexParallelism()
- == other.getVertexParallelism());
- }
- result = result && (hasFragmentNumber() == other.hasFragmentNumber());
- if (hasFragmentNumber()) {
- result = result && (getFragmentNumber()
- == other.getFragmentNumber());
- }
- result = result && (hasAttemptNumber() == other.hasAttemptNumber());
- if (hasAttemptNumber()) {
- result = result && (getAttemptNumber()
- == other.getAttemptNumber());
+ result = result && (hasVertexId() == other.hasVertexId());
+ if (hasVertexId()) {
+ result = result && (getVertexId()
+ == other.getVertexId());
}
result = result &&
getUnknownFields().equals(other.getUnknownFields());
@@ -4077,102 +3573,74 @@ public final class LlapDaemonProtocolProtos {
}
int hash = 41;
hash = (19 * hash) + getDescriptorForType().hashCode();
- if (hasFragmentIdentifierString()) {
- hash = (37 * hash) + FRAGMENT_IDENTIFIER_STRING_FIELD_NUMBER;
- hash = (53 * hash) + getFragmentIdentifierString().hashCode();
+ if (hasApplicationIdString()) {
+ hash = (37 * hash) + APPLICATION_ID_STRING_FIELD_NUMBER;
+ hash = (53 * hash) + getApplicationIdString().hashCode();
}
- if (hasDagName()) {
- hash = (37 * hash) + DAG_NAME_FIELD_NUMBER;
- hash = (53 * hash) + getDagName().hashCode();
+ if (hasAppAttemptNumber()) {
+ hash = (37 * hash) + APP_ATTEMPT_NUMBER_FIELD_NUMBER;
+ hash = (53 * hash) + getAppAttemptNumber();
}
if (hasDagId()) {
hash = (37 * hash) + DAG_ID_FIELD_NUMBER;
hash = (53 * hash) + getDagId();
}
- if (hasVertexName()) {
- hash = (37 * hash) + VERTEX_NAME_FIELD_NUMBER;
- hash = (53 * hash) + getVertexName().hashCode();
- }
- if (hasProcessorDescriptor()) {
- hash = (37 * hash) + PROCESSOR_DESCRIPTOR_FIELD_NUMBER;
- hash = (53 * hash) + getProcessorDescriptor().hashCode();
- }
- if (getInputSpecsCount() > 0) {
- hash = (37 * hash) + INPUT_SPECS_FIELD_NUMBER;
- hash = (53 * hash) + getInputSpecsList().hashCode();
- }
- if (getOutputSpecsCount() > 0) {
- hash = (37 * hash) + OUTPUT_SPECS_FIELD_NUMBER;
- hash = (53 * hash) + getOutputSpecsList().hashCode();
- }
- if (getGroupedInputSpecsCount() > 0) {
- hash = (37 * hash) + GROUPED_INPUT_SPECS_FIELD_NUMBER;
- hash = (53 * hash) + getGroupedInputSpecsList().hashCode();
- }
- if (hasVertexParallelism()) {
- hash = (37 * hash) + VERTEX_PARALLELISM_FIELD_NUMBER;
- hash = (53 * hash) + getVertexParallelism();
- }
- if (hasFragmentNumber()) {
- hash = (37 * hash) + FRAGMENT_NUMBER_FIELD_NUMBER;
- hash = (53 * hash) + getFragmentNumber();
- }
- if (hasAttemptNumber()) {
- hash = (37 * hash) + ATTEMPT_NUMBER_FIELD_NUMBER;
- hash = (53 * hash) + getAttemptNumber();
+ if (hasVertexId()) {
+ hash = (37 * hash) + VERTEX_ID_FIELD_NUMBER;
+ hash = (53 * hash) + getVertexId();
}
hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash;
return hash;
}
- public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto parseFrom(
+ public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier 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.FragmentSpecProto parseFrom(
+ public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier 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.FragmentSpecProto parseFrom(byte[] data)
+ public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier parseFrom(byte[] data)
throws com.google.protobuf.InvalidProtocolBufferException {
return PARSER.parseFrom(data);
}
- public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto parseFrom(
+ public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier 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.FragmentSpecProto parseFrom(java.io.InputStream input)
+ public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier parseFrom(java.io.InputStream input)
throws java.io.IOException {
return PARSER.parseFrom(input);
}
- public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto parseFrom(
+ public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier 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.FragmentSpecProto parseDelimitedFrom(java.io.InputStream input)
+ public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier parseDelimitedFrom(java.io.InputStream input)
throws java.io.IOException {
return PARSER.parseDelimitedFrom(input);
}
- public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto parseDelimitedFrom(
+ public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier 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.FragmentSpecProto parseFrom(
+ public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier parseFrom(
com.google.protobuf.CodedInputStream input)
throws java.io.IOException {
return PARSER.parseFrom(input);
}
- public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto parseFrom(
+ public static org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier parseFrom(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException {
@@ -4181,7 +3649,7 @@ public final class LlapDaemonProtocolProtos {
public static Builder newBuilder() { return Builder.create(); }
public Builder newBuilderForType() { return newBuilder(); }
- public static Builder newBuilder(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto prototype) {
+ public static Builder newBuilder(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier prototype) {
return newBuilder().mergeFrom(prototype);
}
public Builder toBuilder() { return newBuilder(this); }
@@ -4193,24 +3661,24 @@ public final class LlapDaemonProtocolProtos {
return builder;
}
/**
- * Protobuf type {@code FragmentSpecProto}
+ * Protobuf type {@code VertexIdentifier}
*/
public static final class Builder extends
com.google.protobuf.GeneratedMessage.Builder<Builder>
- implements org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProtoOrBuilder {
+ implements org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifierOrBuilder {
public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
- return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_FragmentSpecProto_descriptor;
+ return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_VertexIdentifier_descriptor;
}
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
- return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_FragmentSpecProto_fieldAccessorTable
+ return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_VertexIdentifier_fieldAccessorTable
.ensureFieldAccessorsInitialized(
- org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto.class, org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto.Builder.class);
+ org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier.class, org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier.Builder.class);
}
- // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto.newBuilder()
+ // Construct using org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier.newBuilder()
private Builder() {
maybeForceBuilderInitialization();
}
@@ -4222,10 +3690,6 @@ public final class LlapDaemonProtocolProtos {
}
private void maybeForceBuilderInitialization() {
if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
- getProcessorDescriptorFieldBuilder();
- getInputSpecsFieldBuilder();
- getOutputSpecsFieldBuilder();
- getGroupedInputSpecsFieldBuilder();
}
}
private static Builder create() {
@@ -4234,44 +3698,14 @@ public final class LlapDaemonProtocolProtos {
public Builder clear() {
super.clear();
- fragmentIdentifierString_ = "";
+ applicationIdString_ = "";
bitField0_ = (bitField0_ & ~0x00000001);
- dagName_ = "";
+ appAttemptNumber_ = 0;
bitField0_ = (bitField0_ & ~0x00000002);
dagId_ = 0;
bitField0_ = (bitField0_ & ~0x00000004);
- vertexName_ = "";
+ vertexId_ = 0;
bitField0_ = (bitField0_ & ~0x00000008);
- if (processorDescriptorBuilder_ == null) {
- processorDescriptor_ = org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.getDefaultInstance();
- } else {
- processorDescriptorBuilder_.clear();
- }
- bitField0_ = (bitField0_ & ~0x00000010);
- if (inputSpecsBuilder_ == null) {
- inputSpecs_ = java.util.Collections.emptyList();
- bitField0_ = (bitField0_ & ~0x00000020);
- } else {
- inputSpecsBuilder_.clear();
- }
- if (outputSpecsBuilder_ == null) {
- outputSpecs_ = java.util.Collections.emptyList();
- bitField0_ = (bitField0_ & ~0x00000040);
- } else {
- outputSpecsBuilder_.clear();
- }
- if (groupedInputSpecsBuilder_ == null) {
- groupedInputSpecs_ = java.util.Collections.emptyList();
- bitField0_ = (bitField0_ & ~0x00000080);
- } else {
- groupedInputSpecsBuilder_.clear();
- }
- vertexParallelism_ = 0;
- bitField0_ = (bitField0_ & ~0x00000100);
- fragmentNumber_ = 0;
- bitField0_ = (bitField0_ & ~0x00000200);
- attemptNumber_ = 0;
- bitField0_ = (bitField0_ & ~0x00000400);
return this;
}
@@ -4281,33 +3715,33 @@ public final class LlapDaemonProtocolProtos {
public com.google.protobuf.Descriptors.Descriptor
getDescriptorForType() {
- return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_FragmentSpecProto_descriptor;
+ return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.internal_static_VertexIdentifier_descriptor;
}
- public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto getDefaultInstanceForType() {
- return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto.getDefaultInstance();
+ public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier getDefaultInstanceForType() {
+ return org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier.getDefaultInstance();
}
- public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto build() {
- org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto result = buildPartial();
+ public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier build() {
+ org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier result = buildPartial();
if (!result.isInitialized()) {
throw newUninitializedMessageException(result);
}
return result;
}
- public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto buildPartial() {
- org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto result = new org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto(this);
+ public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier buildPartial() {
+ org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier result = new org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier(this);
int from_bitField0_ = bitField0_;
int to_bitField0_ = 0;
if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
to_bitField0_ |= 0x00000001;
}
- result.fragmentIdentifierString_ = fragmentIdentifierString_;
+ result.applicationIdString_ = applicationIdString_;
if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
to_bitField0_ |= 0x00000002;
}
- result.dagName_ = dagName_;
+ result.appAttemptNumber_ = appAttemptNumber_;
if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
to_bitField0_ |= 0x00000004;
}
@@ -4315,177 +3749,36 @@ public final class LlapDaemonProtocolProtos {
if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
to_bitField0_ |= 0x00000008;
}
- result.vertexName_ = vertexName_;
- if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
- to_bitField0_ |= 0x00000010;
- }
- if (processorDescriptorBuilder_ == null) {
- result.processorDescriptor_ = processorDescriptor_;
- } else {
- result.processorDescriptor_ = processorDescriptorBuilder_.build();
- }
- if (inputSpecsBuilder_ == null) {
- if (((bitField0_ & 0x00000020) == 0x00000020)) {
- inputSpecs_ = java.util.Collections.unmodifiableList(inputSpecs_);
- bitField0_ = (bitField0_ & ~0x00000020);
- }
- result.inputSpecs_ = inputSpecs_;
- } else {
- result.inputSpecs_ = inputSpecsBuilder_.build();
- }
- if (outputSpecsBuilder_ == null) {
- if (((bitField0_ & 0x00000040) == 0x00000040)) {
- outputSpecs_ = java.util.Collections.unmodifiableList(outputSpecs_);
- bitField0_ = (bitField0_ & ~0x00000040);
- }
- result.outputSpecs_ = outputSpecs_;
- } else {
- result.outputSpecs_ = outputSpecsBuilder_.build();
- }
- if (groupedInputSpecsBuilder_ == null) {
- if (((bitField0_ & 0x00000080) == 0x00000080)) {
- groupedInputSpecs_ = java.util.Collections.unmodifiableList(groupedInputSpecs_);
- bitField0_ = (bitField0_ & ~0x00000080);
- }
- result.groupedInputSpecs_ = groupedInputSpecs_;
- } else {
- result.groupedInputSpecs_ = groupedInputSpecsBuilder_.build();
- }
- if (((from_bitField0_ & 0x00000100) == 0x00000100)) {
- to_bitField0_ |= 0x00000020;
- }
- result.vertexParallelism_ = vertexParallelism_;
- if (((from_bitField0_ & 0x00000200) == 0x00000200)) {
- to_bitField0_ |= 0x00000040;
- }
- result.fragmentNumber_ = fragmentNumber_;
- if (((from_bitField0_ & 0x00000400) == 0x00000400)) {
- to_bitField0_ |= 0x00000080;
- }
- result.attemptNumber_ = attemptNumber_;
+ result.vertexId_ = vertexId_;
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.FragmentSpecProto) {
- return mergeFrom((org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto)other);
+ if (other instanceof org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier) {
+ return mergeFrom((org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier)other);
} else {
super.mergeFrom(other);
return this;
}
}
- public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto other) {
- if (other == org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto.getDefaultInstance()) return this;
- if (other.hasFragmentIdentifierString()) {
+ public Builder mergeFrom(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier other) {
+ if (other == org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier.getDefaultInstance()) return this;
+ if (other.hasApplicationIdString()) {
bitField0_ |= 0x00000001;
- fragmentIdentifierString_ = other.fragmentIdentifierString_;
+ applicationIdString_ = other.applicationIdString_;
onChanged();
}
- if (other.hasDagName()) {
- bitField0_ |= 0x00000002;
- dagName_ = other.dagName_;
- onChanged();
+ if (other.hasAppAttemptNumber()) {
+ setAppAttemptNumber(other.getAppAttemptNumber());
}
if (other.hasDagId()) {
setDagId(other.getDagId());
}
- if (other.hasVertexName()) {
- bitField0_ |= 0x00000008;
- vertexName_ = other.vertexName_;
- onChanged();
- }
- if (other.hasProcessorDescriptor()) {
- mergeProcessorDescriptor(other.getProcessorDescriptor());
- }
- if (inputSpecsBuilder_ == null) {
- if (!other.inputSpecs_.isEmpty()) {
- if (inputSpecs_.isEmpty()) {
- inputSpecs_ = other.inputSpecs_;
- bitField0_ = (bitField0_ & ~0x00000020);
- } else {
- ensureInputSpecsIsMutable();
- inputSpecs_.addAll(other.inputSpecs_);
- }
- onChanged();
- }
- } else {
- if (!other.inputSpecs_.isEmpty()) {
- if (inputSpecsBuilder_.isEmpty()) {
- inputSpecsBuilder_.dispose();
- inputSpecsBuilder_ = null;
- inputSpecs_ = other.inputSpecs_;
- bitField0_ = (bitField0_ & ~0x00000020);
- inputSpecsBuilder_ =
- com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
- getInputSpecsFieldBuilder() : null;
- } else {
- inputSpecsBuilder_.addAllMessages(other.inputSpecs_);
- }
- }
- }
- if (outputSpecsBuilder_ == null) {
- if (!other.outputSpecs_.isEmpty()) {
- if (outputSpecs_.isEmpty()) {
- outputSpecs_ = other.outputSpecs_;
- bitField0_ = (bitField0_ & ~0x00000040);
- } else {
- ensureOutputSpecsIsMutable();
- outputSpecs_.addAll(other.outputSpecs_);
- }
- onChanged();
- }
- } else {
- if (!other.outputSpecs_.isEmpty()) {
- if (outputSpecsBuilder_.isEmpty()) {
- outputSpecsBuilder_.dispose();
- outputSpecsBuilder_ = null;
- outputSpecs_ = other.outputSpecs_;
- bitField0_ = (bitField0_ & ~0x00000040);
- outputSpecsBuilder_ =
- com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
- getOutputSpecsFieldBuilder() : null;
- } else {
- outputSpecsBuilder_.addAllMessages(other.outputSpecs_);
- }
- }
- }
- if (groupedInputSpecsBuilder_ == null) {
- if (!other.groupedInputSpecs_.isEmpty()) {
- if (groupedInputSpecs_.isEmpty()) {
- groupedInputSpecs_ = other.groupedInputSpecs_;
- bitField0_ = (bitField0_ & ~0x00000080);
- } else {
- ensureGroupedInputSpecsIsMutable();
- groupedInputSpecs_.addAll(other.groupedInputSpecs_);
- }
- onChanged();
- }
- } else {
- if (!other.groupedInputSpecs_.isEmpty()) {
- if (groupedInputSpecsBuilder_.isEmpty()) {
- groupedInputSpecsBuilder_.dispose();
- groupedInputSpecsBuilder_ = null;
- groupedInputSpecs_ = other.groupedInputSpecs_;
- bitField0_ = (bitField0_ & ~0x00000080);
- groupedInputSpecsBuilder_ =
- com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
- getGroupedInputSpecsFieldBuilder() : null;
- } else {
- groupedInputSpecsBuilder_.addAllMessages(other.groupedInputSpecs_);
- }
- }
- }
- if (other.hasVertexParallelism()) {
- setVertexParallelism(other.getVertexParallelism());
- }
- if (other.hasFragmentNumber()) {
- setFragmentNumber(other.getFragmentNumber());
- }
- if (other.hasAttemptNumber()) {
- setAttemptNumber(other.getAttemptNumber());
+ if (other.hasVertexId()) {
+ setVertexId(other.getVertexId());
}
this.mergeUnknownFields(other.getUnknownFields());
return this;
@@ -4499,11 +3792,11 @@ public final class LlapDaemonProtocolProtos {
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException {
- org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto parsedMessage = null;
+ org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier parsedMessage = null;
try {
parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
- parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto) e.getUnfinishedMessage();
+ parsedMessage = (org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier) e.getUnfinishedMessage();
throw e;
} finally {
if (parsedMessage != null) {
@@ -4514,170 +3807,129 @@ public final class LlapDaemonProtocolProtos {
}
private int bitField0_;
- // optional string fragment_identifier_string = 1;
- private java.lang.Object fragmentIdentifierString_ = "";
+ // optional string application_id_string = 1;
+ private java.lang.Object applicationIdString_ = "";
/**
- * <code>optional string fragment_identifier_string = 1;</code>
+ * <code>optional string application_id_string = 1;</code>
*/
- public boolean hasFragmentIdentifierString() {
+ public boolean hasApplicationIdString() {
return ((bitField0_ & 0x00000001) == 0x00000001);
}
/**
- * <code>optional string fragment_identifier_string = 1;</code>
+ * <code>optional string application_id_string = 1;</code>
*/
- public java.lang.String getFragmentIdentifierString() {
- java.lang.Object ref = fragmentIdentifierString_;
+ public java.lang.String getApplicationIdString() {
+ java.lang.Object ref = applicationIdString_;
if (!(ref instanceof java.lang.String)) {
java.lang.String s = ((com.google.protobuf.ByteString) ref)
.toStringUtf8();
- fragmentIdentifierString_ = s;
+ applicationIdString_ = s;
return s;
} else {
return (java.lang.String) ref;
}
}
/**
- * <code>optional string fragment_identifier_string = 1;</code>
+ * <code>optional string application_id_string = 1;</code>
*/
public com.google.protobuf.ByteString
- getFragmentIdentifierStringBytes() {
- java.lang.Object ref = fragmentIdentifierString_;
+ getApplicationIdStringBytes() {
+ java.lang.Object ref = applicationIdString_;
if (ref instanceof String) {
com.google.protobuf.ByteString b =
com.google.protobuf.ByteString.copyFromUtf8(
(java.lang.String) ref);
- fragmentIdentifierString_ = b;
+ applicationIdString_ = b;
return b;
} else {
return (com.google.protobuf.ByteString) ref;
}
}
/**
- * <code>optional string fragment_identifier_string = 1;</code>
+ * <code>optional string application_id_string = 1;</code>
*/
- public Builder setFragmentIdentifierString(
+ public Builder setApplicationIdString(
java.lang.String value) {
if (value == null) {
throw new NullPointerException();
}
bitField0_ |= 0x00000001;
- fragmentIdentifierString_ = value;
+ applicationIdString_ = value;
onChanged();
return this;
}
/**
- * <code>optional string fragment_identifier_string = 1;</code>
+ * <code>optional string application_id_string = 1;</code>
*/
- public Builder clearFragmentIdentifierString() {
+ public Builder clearApplicationIdString() {
bitField0_ = (bitField0_ & ~0x00000001);
- fragmentIdentifierString_ = getDefaultInstance().getFragmentIdentifierString();
+ applicationIdString_ = getDefaultInstance().getApplicationIdString();
onChanged();
return this;
}
/**
- * <code>optional string fragment_identifier_string = 1;</code>
+ * <code>optional string application_id_string = 1;</code>
*/
- public Builder setFragmentIdentifierStringBytes(
+ public Builder setApplicationIdStringBytes(
com.google.protobuf.ByteString value) {
if (value == null) {
throw new NullPointerException();
}
bitField0_ |= 0x00000001;
- fragmentIdentifierString_ = value;
+ applicationIdString_ = value;
onChanged();
return this;
}
- // optional string dag_name = 2;
- private java.lang.Object dagName_ = "";
+ // optional int32 app_attempt_number = 2;
+ private int appAttemptNumber_ ;
/**
- * <code>optional string dag_name = 2;</code>
+ * <code>optional int32 app_attempt_number = 2;</code>
*/
- public boolean hasDagName() {
+ public boolean hasAppAttemptNumber() {
return ((bitField0_ & 0x00000002) == 0x00000002);
}
/**
- * <code>optional string dag_name = 2;</code>
+ * <code>optional int32 app_attempt_number = 2;</code>
*/
- public java.lang.String getDagName() {
- java.lang.Object ref = dagName_;
- if (!(ref instanceof java.lang.String)) {
- java.lang.String s = ((com.google.protobuf.ByteString) ref)
- .toStringUtf8();
- dagName_ = s;
- return s;
- } else {
- return (java.lang.String) ref;
- }
+ public int getAppAttemptNumber() {
+ return appAttemptNumber_;
}
/**
- * <code>optional string dag_name = 2;</code>
+ * <code>optional int32 app_attempt_number = 2;</code>
*/
- public com.google.protobuf.ByteString
- getDagNameBytes() {
- java.lang.Object ref = dagName_;
- if (ref instanceof String) {
- com.google.protobuf.ByteString b =
- com.google.protobuf.ByteString.copyFromUtf8(
- (java.lang.String) ref);
- dagName_ = b;
- return b;
- } else {
- return (com.google.protobuf.ByteString) ref;
- }
- }
- /**
- * <code>optional string dag_name = 2;</code>
- */
- public Builder setDagName(
- java.lang.String value) {
- if (value == null) {
- throw new NullPointerException();
- }
- bitField0_ |= 0x00000002;
- dagName_ = value;
+ public Builder setAppAttemptNumber(int value) {
+ bitField0_ |= 0x00000002;
+ appAttemptNumber_ = value;
onChanged();
return this;
}
/**
- * <code>optional string dag_name = 2;</code>
+ * <code>optional int32 app_attempt_number = 2;</code>
*/
- public Builder clearDagName() {
+ public Builder clearAppAttemptNumber() {
bitField0_ = (bitField0_ & ~0x00000002);
- dagName_ = getDefaultInstance().getDagName();
- onChanged();
- return this;
- }
- /**
- * <code>optional string dag_name = 2;</code>
- */
- public Builder setDagNameBytes(
- com.google.protobuf.ByteString value) {
- if (value == null) {
- throw new NullPointerException();
- }
- bitField0_ |= 0x00000002;
- dagName_ = value;
+ appAttemptNumber_ = 0;
onChanged();
return this;
}
- // optional int32 dag_id = 11;
+ // optional int32 dag_id = 3;
private int dagId_ ;
/**
- * <code>optional int32 dag_id = 11;</code>
+ * <code>optional int32 dag_id = 3;</code>
*/
public boolean hasDagId() {
return ((bitField0_ & 0x00000004) == 0x00000004);
}
/**
- * <code>optional int32 dag_id = 11;</code>
+ * <code>optional int32 dag_id = 3;</code>
*/
public int getDagId() {
return dagId_;
}
/**
- * <code>optional int32 dag_id = 11;</code>
+ * <code>optional int32 dag_id = 3;</code>
*/
public Builder setDagId(int value) {
bitField0_ |= 0x00000004;
@@ -4686,7 +3938,7 @@ public final class LlapDaemonProtocolProtos {
return this;
}
/**
- * <code>optional int32 dag_id = 11;</code>
+ * <code>optional int32 dag_id = 3;</code>
*/
public Builder clearDagId() {
bitField0_ = (bitField0_ & ~0x00000004);
@@ -4695,1025 +3947,3552 @@ public final class LlapDaemonProtocolProtos {
return this;
}
- // optional string vertex_name = 3;
- private java.lang.Object vertexName_ = "";
+ // optional int32 vertex_id = 4;
+ private int vertexId_ ;
/**
- * <code>optional string vertex_name = 3;</code>
+ * <code>optional int32 vertex_id = 4;</code>
*/
- public boolean hasVertexName() {
+ public boolean hasVertexId() {
return ((bitField0_ & 0x00000008) == 0x00000008);
}
/**
- * <code>optional string vertex_name = 3;</code>
- */
- 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;
- }
- }
- /**
- * <code>optional string vertex_name = 3;</code>
+ * <code>optional int32 vertex_id = 4;</code>
*/
- 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;
- }
+ public int getVertexId() {
+ return vertexId_;
}
/**
- * <code>optional string vertex_name = 3;</code>
+ * <code>optional int32 vertex_id = 4;</code>
*/
- public Builder setVertexName(
- java.lang.String value) {
- if (value == null) {
- throw new NullPointerException();
- }
- bitField0_ |= 0x00000008;
- vertexName_ = value;
+ public Builder setVertexId(int value) {
+ bitField0_ |= 0x00000008;
+ vertexId_ = value;
onChanged();
return this;
}
/**
- * <code>optional string vertex_name = 3;</code>
+ * <code>optional int32 vertex_id = 4;</code>
*/
- public Builder clearVertexName() {
+ public Builder clearVertexId() {
bitField0_ = (bitField0_ & ~0x00000008);
- vertexName_ = getDefaultInstance().getVertexName();
+ vertexId_ = 0;
onChanged();
return this;
}
- /**
- * <code>optional string vertex_name = 3;</code>
- */
- public Builder setVertexNameBytes(
- com.google.protobuf.ByteString value) {
- if (value == null) {
- throw new NullPointerException();
+
+ // @@protoc_insertion_point(builder_scope:VertexIdentifier)
+ }
+
+ static {
+ defaultInstance = new VertexIdentifier(true);
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:VertexIdentifier)
}
- bitField0_ |= 0x00000008;
- vertexName_ = value;
- onChanged();
- return this;
- }
- // optional .EntityDescriptorProto processor_descriptor = 4;
- private org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto processorDescriptor_ = org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.getDefaultInstance();
- private com.google.protobuf.SingleFieldBuilder<
- org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto, org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.Builder, org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProtoOrBuilder> processorDescriptorBuilder_;
- /**
- * <code>optional .EntityDescriptorProto processor_descriptor = 4;</code>
- */
- public boolean hasProcessorDescriptor() {
- return ((bitField0_ & 0x00000010) == 0x00000010);
- }
- /**
- * <code>optional .EntityDescriptorProto processor_descriptor = 4;</code>
- */
- public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto getProcessorDescriptor() {
- if (processorDescriptorBuilder_ == null) {
- return processorDescriptor_;
- } else {
- return processorDescriptorBuilder_.getMessage();
- }
- }
- /**
- * <code>optional .EntityDescriptorProto processor_descriptor = 4;</code>
- */
- public Builder setProcessorDescriptor(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto value) {
- if (processorDescriptorBuilder_ == null) {
- if (value == null) {
- throw new NullPointerException();
- }
- processorDescriptor_ = value;
- onChanged();
- } else {
- processorDescriptorBuilder_.setMessage(value);
- }
- bitField0_ |= 0x00000010;
- return this;
- }
- /**
- * <code>optional .EntityDescriptorProto processor_descriptor = 4;</code>
- */
- public Builder setProcessorDescriptor(
- org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.Builder builderForValue) {
- if (processorDescriptorBuilder_ == null) {
- processorDescriptor_ = builderForValue.build();
- onChanged();
- } else {
- processorDescriptorBuilder_.setMessage(builderForValue.build());
- }
- bitField0_ |= 0x00000010;
- return this;
- }
- /**
- * <code>optional .EntityDescriptorProto processor_descriptor = 4;</code>
- */
- public Builder mergeProcessorDescriptor(org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto value) {
- if (processorDescriptorBuilder_ == null) {
- if (((bitField0_ & 0x00000010) == 0x00000010) &&
- processorDescriptor_ != org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.getDefaultInstance()) {
- processorDescriptor_ =
- org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.newBuilder(processorDescriptor_).mergeFrom(value).buildPartial();
- } else {
- processorDescriptor_ = value;
- }
- onChanged();
- } else {
- processorDescriptorBuilder_.mergeFrom(value);
- }
- bitField0_ |= 0x00000010;
- return this;
- }
- /**
- * <code>optional .EntityDescriptorProto processor_descriptor = 4;</code>
- */
- public Builder clearProcessorDescriptor() {
- if (processorDescriptorBuilder_ == null) {
- processorDescriptor_ = org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.getDefaultInstance();
- onChanged();
- } else {
- processorDescriptorBuilder_.clear();
- }
- bitField0_ = (bitField0_ & ~0x00000010);
- return this;
- }
- /**
- * <code>optional .EntityDescriptorProto processor_descriptor = 4;</code>
- */
- public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.Builder getProcessorDescriptorBuilder() {
- bitField0_ |= 0x00000010;
- onChanged();
- return getProcessorDescriptorFieldBuilder().getBuilder();
- }
- /**
- * <code>optional .EntityDescriptorProto processor_descriptor = 4;</code>
- */
- public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProtoOrBuilder getProcessorDescriptorOrBuilder() {
- if (processorDescriptorBuilder_ != null) {
- return processorDescriptorBuilder_.getMessageOrBuilder();
- } else {
- return processorDescriptor_;
- }
- }
- /**
- * <code>optional .EntityDescriptorProto processor_descriptor = 4;</code>
- */
- private com.google.protobuf.SingleFieldBuilder<
- org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto, org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.Builder, org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProtoOrBuilder>
- getProcessorDescriptorFieldBuilder() {
- if (processorDescriptorBuilder_ == null) {
- processorDescriptorBuilder_ = new com.google.protobuf.SingleFieldBuilder<
- org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto, org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto.Builder, org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProtoOrBuilder>(
- processorDescriptor_,
- getParentForChildren(),
- isClean());
- processorDescriptor_ = null;
- }
- return processorDescriptorBuilder_;
- }
+ public interface SignableVertexSpecOrBuilder
+ extends com.google.protobuf.MessageOrBuilder {
- // repeated .IOSpecProto input_specs = 5;
- private java.util.List<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto> inputSpecs_ =
- java.util.Collections.emptyList();
- private void ensureInputSpecsIsMutable() {
- if (!((bitField0_ & 0x00000020) == 0x00000020)) {
- inputSpecs_ = new java.util.ArrayList<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto>(inputSpecs_);
- bitField0_ |= 0x00000020;
- }
- }
+ // optional string user = 1;
+ /**
+ * <code>optional string user = 1;</code>
+ */
+ boolean hasUser();
+ /**
+ * <code>optional string user = 1;</code>
+ */
+ java.lang.String getUser();
+ /**
+ * <code>optional string user = 1;</code>
+ */
+ com.google.protobuf.ByteString
+ getUserBytes();
- private com.google.protobuf.RepeatedFieldBuilder<
- org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto, org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto.Builder, org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProtoOrBuilder> inputSpecsBuilder_;
+ // optional int64 signatureKeyId = 2;
+ /**
+ * <code>optional int64 signatureKeyId = 2;</code>
+ */
+ boolean hasSignatureKeyId();
+ /**
+ * <code>optional int64 signatureKeyId = 2;</code>
+ */
+ long getSignatureKeyId();
- /**
- * <code>repeated .IOSpecProto input_specs = 5;</code>
- */
- public java.util.List<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto> getInputSpecsList() {
- if (inputSpecsBuilder_ == null) {
- return java.util.Collections.unmodifiableList(inputSpecs_);
- } else {
- return inputSpecsBuilder_.getMessageList();
- }
- }
- /**
- * <code>repeated .IOSpecProto input_specs = 5;</code>
- */
- public int getInputSpecsCount() {
- if (inputSpecsBuilder_ == null) {
- return inputSpecs_.size();
- } else {
- return inputSpecsBuilder_.getCount();
- }
- }
- /**
- * <code>repeated .IOSpecProto input_specs = 5;</code>
- */
- public org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto getInputSpecs(int index) {
- if (inputSpecsBuilder_ == null) {
- return inputSpecs_.get(index);
- } else {
- return inputSpecsBuilder_.getMessage(index);
- }
- }
- /**
- * <code>repeated .IOSpecProto input_specs = 5;</code>
- */
- public Builder setInputSpecs(
- int index, org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto value) {
- if (inputSpecsBuilder_ == null) {
+ // optional .VertexIdentifier vertexIdentifier = 3;
+ /**
+ * <code>optional .VertexIdentifier vertexIdentifier = 3;</code>
+ */
+ boolean hasVertexIdentifier();
+ /**
+ * <code>optional .VertexIdentifier vertexIdentifier = 3;</code>
+ */
+ org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifier getVertexIdentifier();
+ /**
+ * <code>optional .VertexIdentifier vertexIdentifier = 3;</code>
+ */
+ org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.VertexIdentifierOrBuilder getVertexIdentifierOrBuilder();
+
+ // optional string dag_name = 4;
+ /**
+ * <code>optional string dag_name = 4;</code>
+ *
+ * <pre>
+ * Display names cannot be modified by the client for now. If needed, they should be sent to HS2 who will put them here.
+ * </pre>
+ */
+ boolean hasDagName();
+ /**
+ * <code>optional string dag_name = 4;</code>
+ *
+ * <pre>
+ * Display names cannot be modified by the client for now. If needed, they should be sent to HS2 who will put them here.
+ * </pre>
+ */
+ java.lang.String getDagName();
+ /**
+ * <code>optional string dag_name = 4;</code>
+ *
+ * <pre>
+ * Display names cannot be modified by the client for now. If needed, they should be sent to HS2 who will put them here.
+ * </pre>
+ */
+ com.google.protobuf.ByteString
+ getDagNameBytes();
+
+ // optional string vertex_name = 5;
+ /**
+ * <code>optional string vertex_name = 5;</code>
+ */
+ boolean hasVertexName();
+ /**
+ * <code>optional string vertex_name = 5;</code>
+ */
+ java.lang.String getVertexName();
+ /**
+ * <code>optional string vertex_name = 5;</code>
+ */
+ com.google.protobuf.ByteString
+ getVertexNameBytes();
+
+ // optional string token_identifier = 6;
+ /**
+ * <code>optional string token_identifier = 6;</code>
+ *
+ * <pre>
+ * The core vertex stuff
+ * </pre>
+ */
+ boolean hasTokenIdentifier();
+ /**
+ * <code>optional string token_identifier = 6;</code>
+ *
+ * <pre>
+ * The core vertex stuff
+ * </pre>
+ */
+ java.lang.String getTokenIdentifier();
+ /**
+ * <code>optional string token_identifier = 6;</code>
+ *
+ * <pre>
+ * The core vertex stuff
+ * </pre>
+ */
+ com.google.protobuf.ByteString
+ getTokenIdentifierBytes();
+
+ // optional .EntityDescriptorProto processor_descriptor = 7;
+ /**
+ * <code>optional .EntityDescriptorProto processor_descriptor = 7;</code>
+ */
+ boolean hasProcessorDescriptor();
+ /**
+ * <code>optional .EntityDescriptorProto processor_descriptor = 7;</code>
+ */
+ org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto getProcessorDescriptor();
+ /**
+ * <code>optional .EntityDescriptorProto processor_descriptor = 7;</code>
+ */
+ org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProtoOrBuilder getProcessorDescriptorOrBuilder();
+
+ // repeated .IOSpecProto input_specs = 8;
+ /**
+ * <code>repeated .IOSpecProto input_specs = 8;</code>
+ */
+ java.util.List<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto>
+ getInputSpecsList();
+ /**
+ * <code>repeated .IOSpecProto input_specs = 8;</code>
+ */
+ org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto getInputSpecs(int index);
+ /**
+ * <code>repeated .IOSpecProto input_specs = 8;</code>
+ */
+ int getInputSpecsCount();
+ /**
+ * <code>repeated .IOSpecProto input_specs = 8;</code>
+ */
+ java.util.List<? extends org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProtoOrBuilder>
+ getInputSpecsOrBuilderList();
+ /**
+ * <code>repeated .IOSpecProto input_specs = 8;</code>
+ */
+ org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProtoOrBuilder getInputSpecsOrBuilder(
+ int index);
+
+ // repeated .IOSpecProto output_specs = 9;
+ /**
+ * <code>repeated .IOSpecProto output_specs = 9;</code>
+ */
+ java.util.List<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto>
+ getOutputSpecsList();
+ /**
+ * <code>repeated .IOSpecProto output_specs = 9;</code>
+ */
+ org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto getOutputSpecs(int index);
+ /**
+ * <code>repeated .IOSpecProto output_specs = 9;</code>
+ */
+ int getOutputSpecsCount();
+ /**
+ * <code>repeated .IOSpecProto output_specs = 9;</code>
+ */
+ java.util.List<? extends org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProtoOrBuilder>
+ getOutputSpecsOrBuilderList();
+ /**
+ * <code>repeated .IOSpecProto output_specs = 9;</code>
+ */
+ org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProtoOrBuilder getOutputSpecsOrBuilder(
+ int index);
+
+ // repeated .GroupInputSpecProto grouped_input_specs = 10;
+ /**
+ * <code>repeated .GroupInputSpecProto grouped_input_specs = 10;</code>
+ */
+ java.util.List<org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProto>
+ getGroupedInputSpecsList();
+ /**
+ * <code>repeated .GroupInputSpecProto grouped_input_specs = 10;</code>
+ */
+ org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProto getGroupedInputSpecs(int index);
+ /**
+ * <code>repeated .GroupInputSpecProto grouped_input_specs = 10;</code>
+ */
+ int getGroupedInputSpecsCount();
+ /**
+ * <code>repeated .GroupInputSpecProto grouped_input_specs = 10;</code>
+ */
+ java.util.List<? extends org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProtoOrBuilder>
+ getGroupedInputSpecsOrBuilderList();
+ /**
+ * <code>repeated .GroupInputSpecProto grouped_input_specs = 10;</code>
+ */
+ org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.GroupInputSpecProtoOrBuilder getGroupedInputSpecsOrBuilder(
+ int index);
+
+ // optional int32 vertex_parallelism = 11;
+ /**
+ * <code>optional int32 vertex_parallelism = 11;</code>
+ *
+ * <pre>
+ * An internal field required for Tez.
+ * </pre>
+ */
+ boolean hasVertexParallelism();
+ /**
+ * <code>optional int32 vertex_parallelism = 11;</code>
+ *
+ * <pre>
+ * An internal field required for Tez.
+ * </pre>
+ */
+ int getVertexParallelism();
+ }
+ /**
+ * Protobuf type {@code SignableVertexSpec}
+ *
+ * <pre>
+ * The part of SubmitWork that can be signed
+ * </pre>
+ */
+ public static final class SignableVertexSpec extends
+ com.google.protobuf.GeneratedMessage
+ implements SignableVertexSpecOrBuilder {
+ // Use SignableVertexSpec.newBuilder() to construct.
+ private SignableVertexSpec(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+ super(builder);
+ this.unknownFields = builder.getUnknownFields();
+ }
+ private SignableVertexSpec(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+ private static final SignableVertexSpec defaultInstance;
+ public static SignableVertexSpec getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public SignableVertexSpec getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ private final com.google.protobuf.UnknownFieldSet unknownFields;
+ @java.lang.Override
+ public final com.google.protobuf.UnknownFieldSet
+ getUnknownFields() {
+ return this.unknownFields;
+ }
+ private SignableVertexSpec(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ initFields();
+ int mutable_bitField0_ = 0;
+ c
<TRUNCATED>
[02/10] hive git commit: HIVE-11848 - tables in subqueries don't get
locked (Eugene Koifman, reviewed by Wei Zheng)
Posted by jd...@apache.org.
HIVE-11848 - tables in subqueries don't get locked (Eugene Koifman, reviewed by Wei Zheng)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/47bf055c
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/47bf055c
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/47bf055c
Branch: refs/heads/llap
Commit: 47bf055c02990272753105b917b487c5bbfe9208
Parents: 868e5e1
Author: Eugene Koifman <ek...@hortonworks.com>
Authored: Tue May 3 13:33:42 2016 -0700
Committer: Eugene Koifman <ek...@hortonworks.com>
Committed: Tue May 3 13:53:02 2016 -0700
----------------------------------------------------------------------
.../ql/parse/UpdateDeleteSemanticAnalyzer.java | 16 +++++++++-
.../hive/ql/lockmgr/TestDbTxnManager2.java | 33 ++++++++++++++++++++
2 files changed, 48 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/47bf055c/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
index b8771d2..33fbffe 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
@@ -329,7 +329,9 @@ public class UpdateDeleteSemanticAnalyzer extends SemanticAnalyzer {
// Walk through all our inputs and set them to note that this read is part of an update or a
// delete.
for (ReadEntity input : inputs) {
- input.setUpdateOrDelete(true);
+ if(isWritten(input)) {
+ input.setUpdateOrDelete(true);
+ }
}
if (inputIsPartitioned(inputs)) {
@@ -377,6 +379,18 @@ public class UpdateDeleteSemanticAnalyzer extends SemanticAnalyzer {
}
}
+ /**
+ * Check that {@code readEntity} is also being written
+ */
+ private boolean isWritten(Entity readEntity) {
+ for(Entity writeEntity : outputs) {
+ //make sure to compare them as Entity, i.e. that it's the same table or partition, etc
+ if(writeEntity.toString().equalsIgnoreCase(readEntity.toString())) {
+ return true;
+ }
+ }
+ return false;
+ }
private String operation() {
if (updating()) return "update";
else if (deleting()) return "delete";
http://git-wip-us.apache.org/repos/asf/hive/blob/47bf055c/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
index 836b507..6e2cf30 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
@@ -71,6 +71,39 @@ public class TestDbTxnManager2 {
TxnDbUtil.prepDb();
}
@Test
+ public void testLocksInSubquery() throws Exception {
+ checkCmdOnDriver(driver.run("create table if not exists T (a int, b int)"));
+ checkCmdOnDriver(driver.run("create table if not exists S (a int, b int) clustered by(b) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')"));
+ checkCmdOnDriver(driver.run("create table if not exists R (a int, b int) clustered by(b) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')"));
+
+ checkCmdOnDriver(driver.compileAndRespond("delete from S where a in (select a from T where b = 1)"));
+ txnMgr.openTxn("one");
+ txnMgr.acquireLocks(driver.getPlan(), ctx, "one");
+ List<ShowLocksResponseElement> locks = getLocks();
+ Assert.assertEquals("Unexpected lock count", 2, locks.size());
+ checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "T", null, locks.get(0));
+ checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "S", null, locks.get(1));
+ txnMgr.rollbackTxn();
+
+ checkCmdOnDriver(driver.compileAndRespond("update S set a = 7 where a in (select a from T where b = 1)"));
+ txnMgr.openTxn("one");
+ txnMgr.acquireLocks(driver.getPlan(), ctx, "one");
+ locks = getLocks();
+ Assert.assertEquals("Unexpected lock count", 2, locks.size());
+ checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "T", null, locks.get(0));
+ checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "S", null, locks.get(1));
+ txnMgr.rollbackTxn();
+
+ checkCmdOnDriver(driver.compileAndRespond("insert into R select * from S where a in (select a from T where b = 1)"));
+ txnMgr.openTxn("three");
+ txnMgr.acquireLocks(driver.getPlan(), ctx, "three");
+ locks = getLocks();
+ Assert.assertEquals("Unexpected lock count", 3, locks.size());
+ checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "T", null, locks.get(0));
+ checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "S", null, locks.get(1));
+ checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "R", null, locks.get(2));
+ }
+ @Test
public void createTable() throws Exception {
CommandProcessorResponse cpr = driver.compileAndRespond("create table if not exists T (a int, b int)");
checkCmdOnDriver(cpr);